From a184077e463417947a27c979214d53ab01e800be Mon Sep 17 00:00:00 2001 From: hacheigriega Date: Thu, 7 Nov 2024 18:21:17 -0500 Subject: [PATCH] refactor(batching): tree entries refactor and check previous batch for public keys --- app/abci/expected_keepers.go | 3 +- app/abci/handlers.go | 49 ++- app/abci/testutil/expected_keepers_mock.go | 39 +- app/abci/vote_extension_test.go | 48 ++- app/app.go | 4 +- app/utils/seda_signer.go | 3 - proto/sedachain/batching/v1/batching.proto | 8 +- proto/sedachain/batching/v1/genesis.proto | 16 +- scripts/local_multi_setup.sh | 3 - x/batching/keeper/batch.go | 122 ++++-- x/batching/keeper/endblock.go | 11 +- .../keeper/{abci_test.go => endblock_test.go} | 10 +- x/batching/keeper/genesis.go | 8 +- x/batching/keeper/integration_test.go | 6 +- x/batching/keeper/keeper.go | 4 +- x/batching/keeper/querier.go | 2 +- x/batching/types/batching.pb.go | 142 ++++--- x/batching/types/genesis.go | 6 +- x/batching/types/genesis.pb.go | 394 ++++++++++++++++-- x/pubkey/types/pubkey.pb.go | 26 +- 20 files changed, 710 insertions(+), 194 deletions(-) rename x/batching/keeper/{abci_test.go => endblock_test.go} (99%) diff --git a/app/abci/expected_keepers.go b/app/abci/expected_keepers.go index 8c38b937..b47fd470 100644 --- a/app/abci/expected_keepers.go +++ b/app/abci/expected_keepers.go @@ -14,7 +14,8 @@ import ( type BatchingKeeper interface { GetBatchForHeight(ctx context.Context, height int64) (batchingtypes.Batch, error) - SetBatchSignatures(ctx context.Context, batchNum uint64, sigs batchingtypes.BatchSignatures) error + SetBatchSignatures(ctx context.Context, sigs batchingtypes.BatchSignatures) error + GetValidatorTreeEntry(ctx context.Context, batchNum uint64, valAddress sdk.ValAddress) ([]byte, error) } type PubKeyKeeper interface { diff --git a/app/abci/handlers.go b/app/abci/handlers.go index e1ea3c24..5ce6a95f 100644 --- a/app/abci/handlers.go +++ b/app/abci/handlers.go @@ -13,6 +13,7 @@ import ( "cosmossdk.io/collections" addresscodec "cosmossdk.io/core/address" "cosmossdk.io/log" + "github.com/cosmos/cosmos-sdk/baseapp" sdk "github.com/cosmos/cosmos-sdk/types" @@ -75,7 +76,7 @@ func (h *Handlers) SetSEDASigner(signer utils.SEDASigner) { // ExtendVoteHandler handles the ExtendVote ABCI to sign a batch created // from the previous block. func (h *Handlers) ExtendVoteHandler() sdk.ExtendVoteHandler { - return func(ctx sdk.Context, req *abcitypes.RequestExtendVote) (*abcitypes.ResponseExtendVote, error) { + return func(ctx sdk.Context, _ *abcitypes.RequestExtendVote) (*abcitypes.ResponseExtendVote, error) { h.logger.Debug("start extend vote handler") // Check if there is a batch to sign at this block height. @@ -140,7 +141,7 @@ func (h *Handlers) VerifyVoteExtensionHandler() sdk.VerifyVoteExtensionHandler { return nil, err } - err = h.verifyBatchSignatures(ctx, batch.BatchId, req.VoteExtension, req.ValidatorAddress) + err = h.verifyBatchSignatures(ctx, batch.BatchNumber, batch.BatchId, req.VoteExtension, req.ValidatorAddress) if err != nil { h.logger.Error("failed to verify batch signature", "req", req, "err", err) return nil, err @@ -245,7 +246,7 @@ func (h *Handlers) ProcessProposalHandler() sdk.ProcessProposalHandler { for _, vote := range extendedVotes.Votes { // Only consider extensions with pre-commit votes. if vote.BlockIdFlag == cmttypes.BlockIDFlagCommit { - err = h.verifyBatchSignatures(ctx, batch.BatchId, vote.VoteExtension, vote.Validator.Address) + err = h.verifyBatchSignatures(ctx, batch.BatchNumber, batch.BatchId, vote.VoteExtension, vote.Validator.Address) if err != nil { h.logger.Error("proposal contains an invalid vote extension", "vote", vote) return nil, err @@ -306,11 +307,12 @@ func (h *Handlers) PreBlocker() sdk.PreBlocker { return nil, err } batchSigs := batchingtypes.BatchSignatures{ + BatchNumber: batchNum, ValidatorAddr: validator.OperatorAddress, Signatures: vote.VoteExtension, } - err = h.batchingKeeper.SetBatchSignatures(ctx, batchNum, batchSigs) + err = h.batchingKeeper.SetBatchSignatures(ctx, batchSigs) if err != nil { return nil, err } @@ -325,7 +327,7 @@ func (h *Handlers) PreBlocker() sdk.PreBlocker { // against the validator's public key registered at the key index // in the pubkey module. It returns an error unless the verification // succeeds. -func (h *Handlers) verifyBatchSignatures(ctx sdk.Context, batchID, voteExtension, consAddr []byte) error { +func (h *Handlers) verifyBatchSignatures(ctx sdk.Context, batchNum uint64, batchID, voteExtension, consAddr []byte) error { if len(voteExtension) == 0 || len(voteExtension) > MaxVoteExtensionLength { h.logger.Error("invalid vote extension length", "len", len(voteExtension)) return ErrInvalidVoteExtensionLength @@ -341,15 +343,46 @@ func (h *Handlers) verifyBatchSignatures(ctx sdk.Context, batchID, voteExtension } // Recover and verify secp256k1 public key. - pubKey, err := h.pubKeyKeeper.GetValidatorKeyAtIndex(ctx, valOper, utils.SEDAKeyIndexSecp256k1) + var expAddr []byte + if batchNum == collections.DefaultSequenceStart { + pubKey, err := h.pubKeyKeeper.GetValidatorKeyAtIndex(ctx, valOper, utils.SEDAKeyIndexSecp256k1) + if err != nil { + return err + } + expAddr, err = utils.PubKeyToEthAddress(pubKey) + if err != nil { + return err + } + } else { + entry, err := h.batchingKeeper.GetValidatorTreeEntry(ctx, batchNum-1, valOper) + if err != nil { + if errors.Is(err, collections.ErrNotFound) { + pubKey, err := h.pubKeyKeeper.GetValidatorKeyAtIndex(ctx, valOper, utils.SEDAKeyIndexSecp256k1) + if err != nil { + return err + } + expAddr, err = utils.PubKeyToEthAddress(pubKey) + if err != nil { + return err + } + } else { + return err + } + } else { + expAddr = entry[:20] + } + } + + sigPubKey, err := crypto.Ecrecover(batchID, voteExtension[:65]) if err != nil { return err } - sigPubKey, err := crypto.Ecrecover(batchID, voteExtension[:65]) + sigAddr, err := utils.PubKeyToEthAddress(sigPubKey) if err != nil { return err } - if !bytes.Equal(pubKey, sigPubKey) { + + if !bytes.Equal(expAddr, sigAddr) { return ErrInvalidBatchSignature } return nil diff --git a/app/abci/testutil/expected_keepers_mock.go b/app/abci/testutil/expected_keepers_mock.go index cdd72e3e..3af64099 100644 --- a/app/abci/testutil/expected_keepers_mock.go +++ b/app/abci/testutil/expected_keepers_mock.go @@ -18,6 +18,7 @@ import ( types0 "github.com/cosmos/cosmos-sdk/x/staking/types" utils "github.com/sedaprotocol/seda-chain/app/utils" types1 "github.com/sedaprotocol/seda-chain/x/batching/types" + types2 "github.com/sedaprotocol/seda-chain/x/pubkey/types" gomock "go.uber.org/mock/gomock" ) @@ -59,18 +60,33 @@ func (mr *MockBatchingKeeperMockRecorder) GetBatchForHeight(ctx, height any) *go return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "GetBatchForHeight", reflect.TypeOf((*MockBatchingKeeper)(nil).GetBatchForHeight), ctx, height) } +// GetValidatorTreeEntry mocks base method. +func (m *MockBatchingKeeper) GetValidatorTreeEntry(ctx context.Context, batchNum uint64, valAddress types.ValAddress) ([]byte, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "GetValidatorTreeEntry", ctx, batchNum, valAddress) + ret0, _ := ret[0].([]byte) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// GetValidatorTreeEntry indicates an expected call of GetValidatorTreeEntry. +func (mr *MockBatchingKeeperMockRecorder) GetValidatorTreeEntry(ctx, batchNum, valAddress any) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "GetValidatorTreeEntry", reflect.TypeOf((*MockBatchingKeeper)(nil).GetValidatorTreeEntry), ctx, batchNum, valAddress) +} + // SetBatchSignatures mocks base method. -func (m *MockBatchingKeeper) SetBatchSignatures(ctx context.Context, batchNum uint64, sigs types1.BatchSignatures) error { +func (m *MockBatchingKeeper) SetBatchSignatures(ctx context.Context, sigs types1.BatchSignatures) error { m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "SetBatchSignatures", ctx, batchNum, sigs) + ret := m.ctrl.Call(m, "SetBatchSignatures", ctx, sigs) ret0, _ := ret[0].(error) return ret0 } // SetBatchSignatures indicates an expected call of SetBatchSignatures. -func (mr *MockBatchingKeeperMockRecorder) SetBatchSignatures(ctx, batchNum, sigs any) *gomock.Call { +func (mr *MockBatchingKeeperMockRecorder) SetBatchSignatures(ctx, sigs any) *gomock.Call { mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "SetBatchSignatures", reflect.TypeOf((*MockBatchingKeeper)(nil).SetBatchSignatures), ctx, batchNum, sigs) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "SetBatchSignatures", reflect.TypeOf((*MockBatchingKeeper)(nil).SetBatchSignatures), ctx, sigs) } // MockPubKeyKeeper is a mock of PubKeyKeeper interface. @@ -111,6 +127,21 @@ func (mr *MockPubKeyKeeperMockRecorder) GetValidatorKeyAtIndex(ctx, validatorAdd return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "GetValidatorKeyAtIndex", reflect.TypeOf((*MockPubKeyKeeper)(nil).GetValidatorKeyAtIndex), ctx, validatorAddr, index) } +// GetValidatorKeys mocks base method. +func (m *MockPubKeyKeeper) GetValidatorKeys(ctx context.Context, validatorAddr string) (types2.ValidatorPubKeys, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "GetValidatorKeys", ctx, validatorAddr) + ret0, _ := ret[0].(types2.ValidatorPubKeys) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// GetValidatorKeys indicates an expected call of GetValidatorKeys. +func (mr *MockPubKeyKeeperMockRecorder) GetValidatorKeys(ctx, validatorAddr any) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "GetValidatorKeys", reflect.TypeOf((*MockPubKeyKeeper)(nil).GetValidatorKeys), ctx, validatorAddr) +} + // MockStakingKeeper is a mock of StakingKeeper interface. type MockStakingKeeper struct { ctrl *gomock.Controller diff --git a/app/abci/vote_extension_test.go b/app/abci/vote_extension_test.go index 25a50758..886f94c5 100644 --- a/app/abci/vote_extension_test.go +++ b/app/abci/vote_extension_test.go @@ -2,8 +2,7 @@ package abci import ( "bytes" - "crypto/ecdsa" - "crypto/elliptic" + "path/filepath" "testing" "github.com/ethereum/go-ethereum/crypto" @@ -13,6 +12,8 @@ import ( "golang.org/x/crypto/sha3" cometabci "github.com/cometbft/cometbft/abci/types" + "github.com/cometbft/cometbft/config" + "github.com/cometbft/cometbft/privval" "cosmossdk.io/log" @@ -26,23 +27,24 @@ import ( "github.com/sedaprotocol/seda-chain/app/params" "github.com/sedaprotocol/seda-chain/app/utils" batchingtypes "github.com/sedaprotocol/seda-chain/x/batching/types" + pubkeytypes "github.com/sedaprotocol/seda-chain/x/pubkey/types" ) -var _ utils.SEDASigner = &mockSigner{} +func TestExtendVerifyVoteHandlers(t *testing.T) { + // Set up SEDA signer. + tmpDir := t.TempDir() -type mockSigner struct { - PrivKey *ecdsa.PrivateKey -} + privValKeyPath := filepath.Join(tmpDir, config.DefaultPrivValKeyName) + filePV := privval.GenFilePV(privValKeyPath, "") + filePV.Key.Save() -func (m *mockSigner) Sign(input []byte, _ utils.SEDAKeyIndex) ([]byte, error) { - signature, err := crypto.Sign(input, m.PrivKey) - if err != nil { - return nil, err - } - return signature, nil -} + pubKeys, err := utils.GenerateSEDAKeys(tmpDir) + require.NoError(t, err) + signer, err := utils.LoadSEDASigner(privValKeyPath) + require.NoError(t, err) + + secp256k1PubKey := pubKeys[utils.SEDAKeyIndexSecp256k1].PubKey -func TestExtendVerifyVoteHandlers(t *testing.T) { // Configure address prefixes. cfg := sdk.GetConfig() cfg.SetBech32PrefixForAccount(params.Bech32PrefixAccAddr, params.Bech32PrefixAccPub) @@ -69,13 +71,13 @@ func TestExtendVerifyVoteHandlers(t *testing.T) { BlockHeight: 100, // created from the previous block } - privKey, err := crypto.HexToECDSA("79afbf7147841fca72b45a1978dd7669470ba67abbe5c220062924380c9c364b") - require.NoError(t, err) - pubKey := elliptic.Marshal(privKey.PublicKey, privKey.PublicKey.X, privKey.PublicKey.Y) - - signer := mockSigner{privKey} - mockBatchingKeeper.EXPECT().GetBatchForHeight(gomock.Any(), int64(100)).Return(mockBatch, nil).AnyTimes() + mockStakingKeeper.EXPECT().GetValidatorByConsAddr(gomock.Any(), gomock.Any()).Return( + stakingtypes.Validator{ + OperatorAddress: "sedavaloper1ucv5709wlf9jn84ynyjzyzeavwvurmdydtn3px", + }, nil, + ) + mockPubKeyKeeper.EXPECT().GetValidatorKeys(gomock.Any(), gomock.Any()).Return(pubkeytypes.ValidatorPubKeys{}, nil) // Construct the handler and execute it. handler := NewHandlers( @@ -86,7 +88,7 @@ func TestExtendVerifyVoteHandlers(t *testing.T) { authcodec.NewBech32Codec(sdk.GetConfig().GetBech32ValidatorAddrPrefix()), logger, ) - handler.SetSEDASigner(&signer) + handler.SetSEDASigner(signer) extendVoteHandler := handler.ExtendVoteHandler() verifyVoteHandler := handler.VerifyVoteExtensionHandler() @@ -98,7 +100,7 @@ func TestExtendVerifyVoteHandlers(t *testing.T) { // Recover and verify public key sigPubKey, err := crypto.Ecrecover(mockBatch.BatchId, evRes.VoteExtension) require.NoError(t, err) - require.Equal(t, pubKey, sigPubKey) + require.Equal(t, secp256k1PubKey, sigPubKey) testVal := sdk.ConsAddress([]byte("testval")) mockStakingKeeper.EXPECT().GetValidatorByConsAddr(gomock.Any(), testVal).Return( @@ -110,7 +112,7 @@ func TestExtendVerifyVoteHandlers(t *testing.T) { gomock.Any(), []byte{230, 25, 79, 60, 174, 250, 75, 41, 158, 164, 153, 36, 34, 11, 61, 99, 153, 193, 237, 164}, utils.SEDAKeyIndexSecp256k1, - ).Return(pubKey, nil) + ).Return(secp256k1PubKey, nil) vvRes, err := verifyVoteHandler(ctx, &cometabci.RequestVerifyVoteExtension{ Height: 101, diff --git a/app/app.go b/app/app.go index 44866a70..99f550d6 100644 --- a/app/app.go +++ b/app/app.go @@ -1002,12 +1002,10 @@ func NewApp( app.SetPreBlocker(app.PreBlocker) // Register SEDA signer and ExtendVote handler. - fmt.Println(cast.ToString(appOpts.Get("priv_validator_key_file"))) pvKeyFile := filepath.Join(homePath, cast.ToString(appOpts.Get("priv_validator_key_file"))) - // loadPath := filepath.Join(filepath.Dir(pvKeyFile), utils.SEDAKeyFileName) signer, err := utils.LoadSEDASigner(pvKeyFile) if err != nil { - // app.Logger().Error("error loading SEDA signer - ExtendVote handler will not run", "path", loadPath) // TODO + app.Logger().Error("error loading SEDA signer - ExtendVote handler will not run", "err", err) } else { app.Logger().Info("SEDA signer successfully loaded") abciHandler.SetSEDASigner(signer) diff --git a/app/utils/seda_signer.go b/app/utils/seda_signer.go index ee8d4280..d187afec 100644 --- a/app/utils/seda_signer.go +++ b/app/utils/seda_signer.go @@ -35,8 +35,6 @@ type sedaKeys struct { // LoadSEDASigner loads the SEDA keys from a given file and returns // a SEDASigner interface. func LoadSEDASigner(pvKeyFilePath string) (SEDASigner, error) { - // TODO What if there is a rotation? - // TODO Can we safely assume that the file will be loaded? _, err := os.ReadFile(pvKeyFilePath) if err != nil { return nil, fmt.Errorf("failed to read private validator key from %v: %v", pvKeyFilePath, err) @@ -120,7 +118,6 @@ func (s *sedaKeys) ReloadIfMismatch(pubKeys []pubkeytypes.IndexedPubKey) error { // Reload reloads the signer from the key file. func (s *sedaKeys) reload() error { - // TODO merge with LoadSEDASigner?? keysJSONBytes, err := os.ReadFile(s.keyPath) if err != nil { return fmt.Errorf("failed to read SEDA keys from %v: %v", s.keyPath, err) diff --git a/proto/sedachain/batching/v1/batching.proto b/proto/sedachain/batching/v1/batching.proto index 7c46fec6..c1faf6d3 100644 --- a/proto/sedachain/batching/v1/batching.proto +++ b/proto/sedachain/batching/v1/batching.proto @@ -32,7 +32,7 @@ message Batch { // TreeEntries are the given batch's data result tree entries and // validator tree entries. message TreeEntries { - // batch_number is the identifier of the batch the tree entries from. + // batch_number is the identifier of the batch. uint64 batch_number = 1; // data_result_entries are the entries (unhashed leaf contents) of // the data result tree. @@ -45,9 +45,11 @@ message TreeEntries { // BatchSignatures contains basic validator data and its batch signatures // under various cryptographic schemes. message BatchSignatures { - string validator_addr = 1 + // batch_number is the identifier of the batch. + uint64 batch_number = 1; + string validator_addr = 2 [ (cosmos_proto.scalar) = "cosmos.ValidatorAddressString" ]; - bytes signatures = 2; + bytes signatures = 3; } // Params is a list of parameters which can be changed through governance. diff --git a/proto/sedachain/batching/v1/genesis.proto b/proto/sedachain/batching/v1/genesis.proto index 200136c1..01bf64ee 100644 --- a/proto/sedachain/batching/v1/genesis.proto +++ b/proto/sedachain/batching/v1/genesis.proto @@ -12,11 +12,13 @@ message GenesisState { // created batch. uint64 current_batch_number = 1; repeated Batch batches = 2 [ (gogoproto.nullable) = false ]; - repeated TreeEntries tree_entries = 3 [ (gogoproto.nullable) = false ]; + repeated TreeEntry tree_entries = 3 [ (gogoproto.nullable) = false ]; repeated DataResult data_results = 4 [ (gogoproto.nullable) = false ]; repeated BatchAssignment batch_assignments = 5 [ (gogoproto.nullable) = false ]; - Params params = 6 [ (gogoproto.nullable) = false ]; + repeated BatchSignatures batch_signatures = 6 + [ (gogoproto.nullable) = false ]; + Params params = 7 [ (gogoproto.nullable) = false ]; } // BatchAssignment represents a batch assignment for genesis export @@ -25,3 +27,13 @@ message BatchAssignment { uint64 batch_number = 1; string data_request_id = 2; } + +// TreeEntry represents a tree entry for genesis export and import. +message TreeEntry { + // K1 is the first part of the key. It represents the batch number. + uint64 k1 = 1; + // K2 is the second part of the key. It is empty for a data result + // tree entry and validator address for a validator tree entry. + bytes k2 = 2; + bytes entry = 3; +} diff --git a/scripts/local_multi_setup.sh b/scripts/local_multi_setup.sh index 2bec50d4..b47b0c3d 100755 --- a/scripts/local_multi_setup.sh +++ b/scripts/local_multi_setup.sh @@ -112,9 +112,6 @@ NODE1_ID=$($BIN tendermint show-node-id --home=$HOME/.sedad/validator1 | tail -1 sed -i '' -E "s|persistent_peers = \"\"|persistent_peers = \"${NODE1_ID}@localhost:26656\"|g" $HOME/.sedad/validator2/config/config.toml sed -i '' -E "s|persistent_peers = \"\"|persistent_peers = \"${NODE1_ID}@localhost:26656\"|g" $HOME/.sedad/validator3/config/config.toml sed -i '' -E "s|persistent_peers = \"\"|persistent_peers = \"${NODE1_ID}@localhost:26656\"|g" $HOME/.sedad/validator4/config/config.toml -# sed -i '' -E "s|persistent_peers = \"\"|persistent_peers = \"$($BIN tendermint show-node-id --home=$HOME/.sedad/validator1)@localhost:26656\"|g" $HOME/.sedad/validator2/config/config.toml -# sed -i '' -E "s|persistent_peers = \"\"|persistent_peers = \"$($BIN tendermint show-node-id --home=$HOME/.sedad/validator1)@localhost:26656\"|g" $HOME/.sedad/validator3/config/config.toml -# sed -i '' -E "s|persistent_peers = \"\"|persistent_peers = \"$($BIN tendermint show-node-id --home=$HOME/.sedad/validator1)@localhost:26656\"|g" $HOME/.sedad/validator4/config/config.toml # start all four validators tmux new-session -s validator1 -d diff --git a/x/batching/keeper/batch.go b/x/batching/keeper/batch.go index 966c835d..aaf8c10d 100644 --- a/x/batching/keeper/batch.go +++ b/x/batching/keeper/batch.go @@ -38,7 +38,7 @@ func (k Keeper) setBatch(ctx context.Context, batch types.Batch) error { // and validator tree entries. It returns an error if a batch already // exists at the given batch's block height or if the given batch's // batch number does not match the next batch number. -func (k Keeper) SetNewBatch(ctx context.Context, batch types.Batch, dataEntries, valEntries [][]byte) error { +func (k Keeper) SetNewBatch(ctx context.Context, batch types.Batch, dataEntries [][]byte, valEntries []ValidatorTreeEntry) error { found, err := k.batches.Has(ctx, batch.BlockHeight) if err != nil { return err @@ -56,7 +56,7 @@ func (k Keeper) SetNewBatch(ctx context.Context, batch types.Batch, dataEntries, } batch.BatchNumber = newBatchNum - err = k.setTreeEntries(ctx, newBatchNum, dataEntries, valEntries) + err = k.setTreeEntriesForBatch(ctx, newBatchNum, dataEntries, valEntries) if err != nil { return err } @@ -127,33 +127,76 @@ func (k Keeper) GetAllBatches(ctx sdk.Context) ([]types.Batch, error) { return batches, nil } -// setTreeEntries stores the data result entries and validator entries -// using the given batch number as the key. -func (k Keeper) setTreeEntries(ctx context.Context, batchNum uint64, dataEntries, valEntries [][]byte) error { - return k.treeEntries.Set( - ctx, - batchNum, - types.TreeEntries{ - BatchNumber: batchNum, - DataResultEntries: dataEntries, - ValidatorEntries: valEntries, - }, - ) +type ValidatorTreeEntry struct { + ValAddress sdk.ValAddress + Entry []byte } -// GetTreeEntries returns the tree entries corresponding to the given -// batch number. -func (k Keeper) GetTreeEntries(ctx context.Context, batchNum uint64) (types.TreeEntries, error) { - entries, err := k.treeEntries.Get(ctx, batchNum) +// setTreeEntry stores a tree entry. +func (k Keeper) setTreeEntry(ctx context.Context, entry types.TreeEntry) error { + return k.treeEntries.Set(ctx, collections.Join(entry.K1, entry.K2), entry.Entry) +} + +// setTreeEntriesForBatch stores the data result entries and validator +// entries for a given batch. +func (k Keeper) setTreeEntriesForBatch(ctx context.Context, batchNum uint64, dataEntries [][]byte, valEntries []ValidatorTreeEntry) error { + for _, entry := range dataEntries { + err := k.treeEntries.Set(ctx, collections.Join(batchNum, []byte{}), entry) + if err != nil { + return err + } + } + for _, entry := range valEntries { + err := k.treeEntries.Set(ctx, collections.Join(batchNum, entry.ValAddress.Bytes()), entry.Entry) + if err != nil { + return err + } + } + return nil +} + +// GetValidatorTreeEntry returns the tree entry of a given validator +// for a specified batch +func (k Keeper) GetValidatorTreeEntry(ctx context.Context, batchNum uint64, valAddress sdk.ValAddress) ([]byte, error) { + return k.treeEntries.Get(ctx, collections.Join(batchNum, valAddress.Bytes())) +} + +// GetTreeEntriesForBatch returns the tree entries corresponding to +// the givenbatch number. +func (k Keeper) GetTreeEntriesForBatch(ctx context.Context, batchNum uint64) (types.TreeEntries, error) { + rng := collections.NewPrefixedPairRange[uint64, []byte](batchNum) + itr, err := k.treeEntries.Iterate(ctx, rng) if err != nil { return types.TreeEntries{}, err } - return entries, nil + defer itr.Close() + + kvs, err := itr.KeyValues() + if err != nil { + return types.TreeEntries{}, err + } + if len(kvs) == 0 { + return types.TreeEntries{}, collections.ErrNotFound + } + + var dataEntries, valEntries [][]byte + for _, kv := range kvs { + if len(kv.Key.K2()) == 0 { + dataEntries = append(dataEntries, kv.Value) + } else { + valEntries = append(valEntries, kv.Value) + } + } + return types.TreeEntries{ + BatchNumber: batchNum, + DataResultEntries: dataEntries, + ValidatorEntries: valEntries, + }, nil } // IterateBatches iterates over the tree entries and performs a given // callback function. -func (k Keeper) IterateTreeEntries(ctx sdk.Context, callback func(types.TreeEntries) (stop bool)) error { +func (k Keeper) IterateTreeEntries(ctx sdk.Context, callback func(types.TreeEntry) (stop bool)) error { iter, err := k.treeEntries.Iterate(ctx, nil) if err != nil { return err @@ -166,7 +209,11 @@ func (k Keeper) IterateTreeEntries(ctx sdk.Context, callback func(types.TreeEntr return err } - if callback(kv.Value) { + if callback(types.TreeEntry{ + K1: kv.Key.K1(), + K2: kv.Key.K2(), + Entry: kv.Value, + }) { break } } @@ -174,9 +221,9 @@ func (k Keeper) IterateTreeEntries(ctx sdk.Context, callback func(types.TreeEntr } // GetAllTreeEntries retrieves all tree entries from the store. -func (k Keeper) GetAllTreeEntries(ctx sdk.Context) ([]types.TreeEntries, error) { - var entries []types.TreeEntries - err := k.IterateTreeEntries(ctx, func(entry types.TreeEntries) bool { +func (k Keeper) GetAllTreeEntries(ctx sdk.Context) ([]types.TreeEntry, error) { + var entries []types.TreeEntry + err := k.IterateTreeEntries(ctx, func(entry types.TreeEntry) bool { entries = append(entries, entry) return false }) @@ -187,12 +234,12 @@ func (k Keeper) GetAllTreeEntries(ctx sdk.Context) ([]types.TreeEntries, error) } // SetBatchSignatures stores a validator's signatures of a batch. -func (k Keeper) SetBatchSignatures(ctx context.Context, batchNum uint64, sigs types.BatchSignatures) error { +func (k Keeper) SetBatchSignatures(ctx context.Context, sigs types.BatchSignatures) error { valAddr, err := k.validatorAddressCodec.StringToBytes(sigs.ValidatorAddr) if err != nil { return err } - return k.batchSignatures.Set(ctx, collections.Join(batchNum, valAddr), sigs) + return k.batchSignatures.Set(ctx, collections.Join(sigs.BatchNumber, valAddr), sigs) } // GetBatchSignatures retrieves the batch signatures by a given @@ -232,3 +279,26 @@ func (k Keeper) GetBatchSigsForBatch(ctx context.Context, batchNum uint64) ([]ty } return sigs, err } + +// GetAllBatchSignatures returns all batch signatures in the store. +func (k Keeper) GetAllBatchSignatures(ctx context.Context) ([]types.BatchSignatures, error) { + itr, err := k.batchSignatures.Iterate(ctx, nil) + if err != nil { + return nil, err + } + defer itr.Close() + + kvs, err := itr.KeyValues() + if err != nil { + return nil, err + } + if len(kvs) == 0 { + return nil, collections.ErrNotFound + } + + sigs := make([]types.BatchSignatures, len(kvs)) + for i, kv := range kvs { + sigs[i] = kv.Value + } + return sigs, err +} diff --git a/x/batching/keeper/endblock.go b/x/batching/keeper/endblock.go index fea23e77..9109a81e 100644 --- a/x/batching/keeper/endblock.go +++ b/x/batching/keeper/endblock.go @@ -51,7 +51,7 @@ func (k Keeper) EndBlock(ctx sdk.Context) (err error) { // results and a validator tree from the current active validator set. // It returns a resulting batch, data result tree entries, and validator // tree entries in that order. -func (k Keeper) ConstructBatch(ctx sdk.Context) (types.Batch, [][]byte, [][]byte, error) { +func (k Keeper) ConstructBatch(ctx sdk.Context) (types.Batch, [][]byte, []ValidatorTreeEntry, error) { var newBatchNum uint64 var latestDataRootHex, latestValRootHex string latestBatch, err := k.GetLatestBatch(ctx) @@ -157,13 +157,13 @@ func (k Keeper) ConstructDataResultTree(ctx sdk.Context, newBatchNum uint64) ([] // validators in the active set and their registered public keys. // It returns the tree's entries without the domain separators and // the tree root. -func (k Keeper) ConstructValidatorTree(ctx sdk.Context) ([][]byte, []byte, error) { +func (k Keeper) ConstructValidatorTree(ctx sdk.Context) ([]ValidatorTreeEntry, []byte, error) { totalPower, err := k.stakingKeeper.GetLastTotalPower(ctx) if err != nil { return nil, nil, err } - var entries [][]byte + var entries []ValidatorTreeEntry var treeEntries [][]byte err = k.stakingKeeper.IterateLastValidatorPowers(ctx, func(valAddr sdk.ValAddress, power int64) (stop bool) { // Retrieve corresponding public key and convert it to @@ -193,7 +193,10 @@ func (k Keeper) ConstructValidatorTree(ctx sdk.Context) ([][]byte, []byte, error //nolint:gosec // G115: Max of powerPercent should be 1e8 < 2^64. binary.BigEndian.PutUint32(treeEntry[len(separator)+len(ethAddr):], uint32(powerPercent)) - entries = append(entries, treeEntry[len(separator):]) + entries = append(entries, ValidatorTreeEntry{ + ValAddress: valAddr, + Entry: treeEntry[len(separator):], + }) treeEntries = append(treeEntries, treeEntry) return false }) diff --git a/x/batching/keeper/abci_test.go b/x/batching/keeper/endblock_test.go similarity index 99% rename from x/batching/keeper/abci_test.go rename to x/batching/keeper/endblock_test.go index 13ae05eb..d1f879a5 100644 --- a/x/batching/keeper/abci_test.go +++ b/x/batching/keeper/endblock_test.go @@ -82,11 +82,11 @@ func Test_ConstructValidatorTree(t *testing.T) { parsedPowers := make([]uint32, len(entries)) entriesWithSep := make([][]byte, len(entries)) for i, entry := range entries { - parsedAddrs[i] = entry[:20] - parsedPowers[i] = binary.BigEndian.Uint32(entry[20:]) + parsedAddrs[i] = entry.Entry[:20] + parsedPowers[i] = binary.BigEndian.Uint32(entry.Entry[20:]) expectedAddrs[i], err = utils.PubKeyToEthAddress(pks[i]) require.NoError(t, err) - entriesWithSep[i] = append([]byte{utils.SEDASeparatorSecp256k1}, entry...) + entriesWithSep[i] = append([]byte{utils.SEDASeparatorSecp256k1}, entry.Entry...) } require.ElementsMatch(t, expectedAddrs, parsedAddrs) require.ElementsMatch(t, powerPercents, parsedPowers) @@ -529,8 +529,8 @@ func Test_ConstructValidatorTreeWithTestData(t *testing.T) { parsedAddrs := make([][]byte, len(entries)) parsedPowers := make([]uint32, len(entries)) for i, entry := range entries { - parsedAddrs[i] = entry[:20] - parsedPowers[i] = binary.BigEndian.Uint32(entry[20:]) + parsedAddrs[i] = entry.Entry[:20] + parsedPowers[i] = binary.BigEndian.Uint32(entry.Entry[20:]) expectedAddr, err := hex.DecodeString(data.Validators[i].Identity[2:]) require.NoError(t, err) expectedAddrs[i] = expectedAddr diff --git a/x/batching/keeper/genesis.go b/x/batching/keeper/genesis.go index c4178cbf..f54ebaab 100644 --- a/x/batching/keeper/genesis.go +++ b/x/batching/keeper/genesis.go @@ -17,7 +17,7 @@ func (k Keeper) InitGenesis(ctx sdk.Context, data types.GenesisState) { } } for _, entry := range data.TreeEntries { - if err := k.setTreeEntries(ctx, entry.BatchNumber, entry.DataResultEntries, entry.ValidatorEntries); err != nil { + if err := k.setTreeEntry(ctx, entry); err != nil { panic(err) } } @@ -48,9 +48,13 @@ func (k Keeper) ExportGenesis(ctx sdk.Context) types.GenesisState { if err != nil { panic(err) } + signatures, err := k.GetAllBatchSignatures(ctx) + if err != nil { + panic(err) + } params, err := k.GetParams(ctx) if err != nil { panic(err) } - return types.NewGenesisState(curBatchNum, batches, entries, dataResults, batchAssignments, params) + return types.NewGenesisState(curBatchNum, batches, entries, dataResults, batchAssignments, signatures, params) } diff --git a/x/batching/keeper/integration_test.go b/x/batching/keeper/integration_test.go index c4e1a0b5..29f88ad9 100644 --- a/x/batching/keeper/integration_test.go +++ b/x/batching/keeper/integration_test.go @@ -39,7 +39,7 @@ import ( "github.com/sedaprotocol/seda-chain/app/params" "github.com/sedaprotocol/seda-chain/integration" "github.com/sedaprotocol/seda-chain/x/batching" - "github.com/sedaprotocol/seda-chain/x/batching/keeper" + batchingkeeper "github.com/sedaprotocol/seda-chain/x/batching/keeper" "github.com/sedaprotocol/seda-chain/x/batching/types" "github.com/sedaprotocol/seda-chain/x/pubkey" pubkeykeeper "github.com/sedaprotocol/seda-chain/x/pubkey/keeper" @@ -84,7 +84,7 @@ type fixture struct { wasmStorageKeeper wasmstoragekeeper.Keeper tallyKeeper tallykeeper.Keeper pubKeyKeeper pubkeykeeper.Keeper - batchingKeeper keeper.Keeper + batchingKeeper batchingkeeper.Keeper mockViewKeeper *testutil.MockViewKeeper logBuf *bytes.Buffer } @@ -191,7 +191,7 @@ func initFixture(tb testing.TB) *fixture { stakingKeeper, addresscodec.NewBech32Codec(params.Bech32PrefixValAddr), ) - batchingKeeper := keeper.NewKeeper( + batchingKeeper := batchingkeeper.NewKeeper( cdc, runtime.NewKVStoreService(keys[types.StoreKey]), authtypes.NewModuleAddress(govtypes.ModuleName).String(), diff --git a/x/batching/keeper/keeper.go b/x/batching/keeper/keeper.go index 0bb108f4..a67b17fc 100644 --- a/x/batching/keeper/keeper.go +++ b/x/batching/keeper/keeper.go @@ -34,7 +34,7 @@ type Keeper struct { batchAssignments collections.Map[string, uint64] currentBatchNumber collections.Sequence batches *collections.IndexedMap[int64, types.Batch, BatchIndexes] - treeEntries collections.Map[uint64, types.TreeEntries] + treeEntries collections.Map[collections.Pair[uint64, []byte], []byte] batchSignatures collections.Map[collections.Pair[uint64, []byte], types.BatchSignatures] params collections.Item[types.Params] } @@ -64,7 +64,7 @@ func NewKeeper( batchAssignments: collections.NewMap(sb, types.BatchAssignmentsPrefix, "batch_assignments", collections.StringKey, collections.Uint64Value), currentBatchNumber: collections.NewSequence(sb, types.CurrentBatchNumberKey, "current_batch_number"), batches: collections.NewIndexedMap(sb, types.BatchesKeyPrefix, "batches", collections.Int64Key, codec.CollValue[types.Batch](cdc), NewBatchIndexes(sb)), - treeEntries: collections.NewMap(sb, types.TreeEntriesKeyPrefix, "tree_entries", collections.Uint64Key, codec.CollValue[types.TreeEntries](cdc)), + treeEntries: collections.NewMap(sb, types.TreeEntriesKeyPrefix, "tree_entries", collections.PairKeyCodec(collections.Uint64Key, collections.BytesKey), collections.BytesValue), batchSignatures: collections.NewMap(sb, types.BatchSignaturesKeyPrefix, "batch_signatures", collections.PairKeyCodec(collections.Uint64Key, collections.BytesKey), codec.CollValue[types.BatchSignatures](cdc)), params: collections.NewItem(sb, types.ParamsKey, "params", codec.CollValue[types.Params](cdc)), } diff --git a/x/batching/keeper/querier.go b/x/batching/keeper/querier.go index 91f903f8..557cd589 100644 --- a/x/batching/keeper/querier.go +++ b/x/batching/keeper/querier.go @@ -74,7 +74,7 @@ func (q Querier) Batches(c context.Context, req *types.QueryBatchesRequest) (*ty func (q Querier) TreeEntries(c context.Context, req *types.QueryTreeEntriesRequest) (*types.QueryTreeEntriesResponse, error) { ctx := sdk.UnwrapSDKContext(c) - entries, err := q.Keeper.GetTreeEntries(ctx, req.BatchNumber) + entries, err := q.Keeper.GetTreeEntriesForBatch(ctx, req.BatchNumber) if err != nil { return nil, err } diff --git a/x/batching/types/batching.pb.go b/x/batching/types/batching.pb.go index fd92adb2..18650f71 100644 --- a/x/batching/types/batching.pb.go +++ b/x/batching/types/batching.pb.go @@ -132,7 +132,7 @@ func (m *Batch) GetProvingMedatada() []byte { // TreeEntries are the given batch's data result tree entries and // validator tree entries. type TreeEntries struct { - // batch_number is the identifier of the batch the tree entries from. + // batch_number is the identifier of the batch. BatchNumber uint64 `protobuf:"varint,1,opt,name=batch_number,json=batchNumber,proto3" json:"batch_number,omitempty"` // data_result_entries are the entries (unhashed leaf contents) of // the data result tree. @@ -199,8 +199,10 @@ func (m *TreeEntries) GetValidatorEntries() [][]byte { // BatchSignatures contains basic validator data and its batch signatures // under various cryptographic schemes. type BatchSignatures struct { - ValidatorAddr string `protobuf:"bytes,1,opt,name=validator_addr,json=validatorAddr,proto3" json:"validator_addr,omitempty"` - Signatures []byte `protobuf:"bytes,2,opt,name=signatures,proto3" json:"signatures,omitempty"` + // batch_number is the identifier of the batch. + BatchNumber uint64 `protobuf:"varint,1,opt,name=batch_number,json=batchNumber,proto3" json:"batch_number,omitempty"` + ValidatorAddr string `protobuf:"bytes,2,opt,name=validator_addr,json=validatorAddr,proto3" json:"validator_addr,omitempty"` + Signatures []byte `protobuf:"bytes,3,opt,name=signatures,proto3" json:"signatures,omitempty"` } func (m *BatchSignatures) Reset() { *m = BatchSignatures{} } @@ -236,6 +238,13 @@ func (m *BatchSignatures) XXX_DiscardUnknown() { var xxx_messageInfo_BatchSignatures proto.InternalMessageInfo +func (m *BatchSignatures) GetBatchNumber() uint64 { + if m != nil { + return m.BatchNumber + } + return 0 +} + func (m *BatchSignatures) GetValidatorAddr() string { if m != nil { return m.ValidatorAddr @@ -439,53 +448,53 @@ func init() { } var fileDescriptor_5b2a028024867de2 = []byte{ - // 721 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x8c, 0x94, 0xcf, 0x6a, 0x1b, 0x49, - 0x10, 0xc6, 0x3d, 0x92, 0xac, 0x3f, 0x2d, 0xc9, 0x96, 0xdb, 0xeb, 0xdd, 0xb1, 0x61, 0x67, 0x64, - 0xb1, 0x66, 0xb5, 0x6b, 0x2c, 0x61, 0x7c, 0x58, 0xd8, 0xdd, 0x4b, 0x26, 0x09, 0xd8, 0x04, 0x07, - 0xd3, 0x76, 0x72, 0xc8, 0x65, 0x68, 0x4d, 0x37, 0xa3, 0xc6, 0xd2, 0xb4, 0xe8, 0x6e, 0x09, 0x9b, - 0x3c, 0x43, 0x20, 0x2f, 0x10, 0xc8, 0x43, 0xe4, 0x92, 0x37, 0xc8, 0xd1, 0xe4, 0x94, 0xd3, 0x10, - 0xec, 0x4b, 0x98, 0xa7, 0x08, 0xd3, 0x3d, 0x1a, 0xc9, 0xce, 0x25, 0x27, 0x55, 0xfd, 0xea, 0xeb, - 0xd2, 0x54, 0xd7, 0x47, 0x83, 0x3f, 0x24, 0x25, 0x38, 0x18, 0x62, 0x16, 0xf5, 0x07, 0x58, 0x05, - 0x43, 0x16, 0x85, 0xfd, 0xd9, 0x61, 0x1e, 0xf7, 0x26, 0x82, 0x2b, 0x0e, 0xb7, 0x72, 0x55, 0x2f, - 0xaf, 0xcc, 0x0e, 0x77, 0x7e, 0x09, 0x79, 0xc8, 0xb5, 0xa2, 0x9f, 0x46, 0x46, 0xbc, 0xb3, 0x1d, - 0x70, 0x39, 0xe6, 0xd2, 0x37, 0x05, 0x93, 0x98, 0x52, 0xe7, 0x5d, 0x01, 0xac, 0x7a, 0x69, 0x03, - 0xb8, 0x0b, 0x1a, 0xba, 0x93, 0x1f, 0x4d, 0xc7, 0x03, 0x2a, 0x6c, 0xab, 0x6d, 0x75, 0x4b, 0xa8, - 0xae, 0xd9, 0x73, 0x8d, 0xb4, 0x64, 0xc4, 0x83, 0x4b, 0x7f, 0x48, 0x59, 0x38, 0x54, 0x76, 0xa1, - 0x6d, 0x75, 0x8b, 0xa8, 0xae, 0xd9, 0xb1, 0x46, 0xf0, 0x1f, 0x60, 0x07, 0x53, 0x21, 0x68, 0xa4, - 0x7c, 0x82, 0x15, 0xf6, 0x05, 0x95, 0xd3, 0x91, 0xf2, 0x05, 0xe7, 0xca, 0x2e, 0xb6, 0xad, 0x6e, - 0x0d, 0x6d, 0x65, 0xf5, 0x27, 0x58, 0x61, 0xa4, 0xab, 0x88, 0x73, 0x05, 0xbb, 0xa0, 0xf5, 0xc3, - 0x81, 0x92, 0x3e, 0xb0, 0x46, 0xee, 0x2b, 0xf7, 0xc0, 0xda, 0x0c, 0x8f, 0x18, 0xc1, 0x8a, 0x0b, - 0xa3, 0x5b, 0xd5, 0xba, 0x66, 0x4e, 0xb5, 0x6c, 0x1b, 0x54, 0xcd, 0x3c, 0x8c, 0xd8, 0xe5, 0xb6, - 0xd5, 0x6d, 0xa0, 0x8a, 0xce, 0x4f, 0x08, 0xfc, 0x0b, 0xb4, 0x26, 0x82, 0xcf, 0x58, 0x14, 0xfa, - 0x63, 0x9a, 0x76, 0x27, 0xd8, 0xae, 0x68, 0xc9, 0x7a, 0xc6, 0x4f, 0x33, 0xdc, 0x79, 0x63, 0x81, - 0xfa, 0x85, 0xa0, 0xf4, 0x69, 0xa4, 0x04, 0xa3, 0xf2, 0x67, 0x6e, 0xa9, 0x07, 0x36, 0x97, 0x27, - 0xa1, 0xe6, 0xa4, 0x5d, 0x68, 0x17, 0xbb, 0x0d, 0xb4, 0xb1, 0x18, 0x66, 0xde, 0x72, 0x1f, 0x6c, - 0x2c, 0xe6, 0x99, 0xab, 0x8b, 0x5a, 0xdd, 0xca, 0x0b, 0x99, 0xb8, 0xf3, 0x1a, 0xac, 0xeb, 0x75, - 0x9d, 0xb3, 0x30, 0xc2, 0x6a, 0x2a, 0xa8, 0x84, 0xc7, 0xcb, 0xf7, 0x81, 0x09, 0x31, 0x1f, 0x55, - 0xf3, 0x76, 0x3f, 0x7f, 0x38, 0xf8, 0x3d, 0x5b, 0xf6, 0xcb, 0xb9, 0xe0, 0x11, 0x21, 0x82, 0x4a, - 0x79, 0xae, 0x04, 0x8b, 0xc2, 0xa5, 0x2b, 0x4b, 0x39, 0x74, 0x00, 0x90, 0x79, 0x5f, 0xbd, 0xdd, - 0x06, 0x5a, 0x22, 0x9d, 0x67, 0xa0, 0x7c, 0x86, 0x05, 0x1e, 0x4b, 0xf8, 0x1f, 0xd8, 0x59, 0xfc, - 0xa7, 0xa4, 0xca, 0x57, 0x82, 0x8d, 0xfd, 0x09, 0x15, 0x01, 0x8d, 0x94, 0xfe, 0xff, 0x26, 0xfa, - 0x2d, 0x57, 0x9c, 0x53, 0x75, 0x21, 0xd8, 0xf8, 0xcc, 0x94, 0xff, 0x2d, 0x7d, 0x7b, 0xef, 0x5a, - 0x9d, 0x8f, 0x45, 0x00, 0x16, 0x1e, 0x80, 0xbf, 0x82, 0x02, 0x23, 0xd9, 0x97, 0x97, 0x93, 0xd8, - 0x2d, 0x30, 0x82, 0x0a, 0x8c, 0x40, 0x07, 0xac, 0x12, 0x91, 0xee, 0xb0, 0xa0, 0x4b, 0xb5, 0x24, - 0x76, 0x0d, 0x40, 0x25, 0x22, 0x4e, 0x08, 0xdc, 0x03, 0x95, 0x19, 0x15, 0x92, 0xf1, 0xc8, 0xf8, - 0xcb, 0xab, 0x27, 0xb1, 0x3b, 0x47, 0x68, 0x1e, 0xc0, 0xa3, 0x07, 0xd6, 0x4d, 0xad, 0x55, 0xf2, - 0x5a, 0x49, 0xec, 0xde, 0xe3, 0xf7, 0xcd, 0xfc, 0x37, 0xa8, 0xd1, 0x2b, 0xa6, 0xfc, 0x80, 0x13, - 0xaa, 0x4d, 0xd6, 0xf4, 0x9a, 0x49, 0xec, 0x2e, 0x20, 0xaa, 0xa6, 0xe1, 0x63, 0x4e, 0x28, 0xfc, - 0x13, 0x54, 0x43, 0x2c, 0xfd, 0xa9, 0xa4, 0xc6, 0x6e, 0x25, 0xaf, 0x91, 0xc4, 0x6e, 0xce, 0x50, - 0x25, 0xc4, 0xf2, 0x85, 0xa4, 0x04, 0x76, 0x40, 0xd9, 0x38, 0xc3, 0x58, 0xce, 0x03, 0x49, 0xec, - 0x66, 0x04, 0x65, 0xbf, 0xf0, 0x7f, 0xb0, 0x3e, 0xc1, 0xd7, 0x03, 0x1c, 0x5c, 0xea, 0x85, 0x52, - 0x29, 0xed, 0xaa, 0x1e, 0x6e, 0x33, 0x89, 0xdd, 0x87, 0x25, 0xb4, 0x96, 0x81, 0x6c, 0xb7, 0xe9, - 0xac, 0xe9, 0xeb, 0xe0, 0x4f, 0xf0, 0xf5, 0x88, 0x63, 0x62, 0xd7, 0xf4, 0x51, 0x3d, 0xeb, 0x32, - 0x47, 0xf5, 0x34, 0x3b, 0x33, 0x09, 0xdc, 0x07, 0xb5, 0x80, 0x47, 0x92, 0x46, 0x72, 0x2a, 0x6d, - 0xd0, 0xb6, 0xba, 0x55, 0x33, 0x6b, 0x0e, 0xd1, 0x22, 0xf4, 0x4e, 0x3f, 0xdd, 0x3a, 0xd6, 0xcd, - 0xad, 0x63, 0x7d, 0xbd, 0x75, 0xac, 0xb7, 0x77, 0xce, 0xca, 0xcd, 0x9d, 0xb3, 0xf2, 0xe5, 0xce, - 0x59, 0x79, 0x75, 0x14, 0x32, 0x35, 0x9c, 0x0e, 0x7a, 0x01, 0x1f, 0xf7, 0xd3, 0xf6, 0xfa, 0x95, - 0x09, 0xf8, 0x48, 0x27, 0x07, 0xe6, 0x59, 0xbb, 0x5a, 0x3c, 0x6c, 0xea, 0x7a, 0x42, 0xe5, 0xa0, - 0xac, 0x55, 0x47, 0xdf, 0x03, 0x00, 0x00, 0xff, 0xff, 0x0d, 0xdc, 0x92, 0xdc, 0xfb, 0x04, 0x00, - 0x00, + // 723 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x8c, 0x54, 0xcd, 0x6e, 0xdb, 0x38, + 0x10, 0x8e, 0x6c, 0xc7, 0x3f, 0xb4, 0x9d, 0x38, 0xcc, 0x66, 0x57, 0x09, 0xb0, 0x92, 0x63, 0x6c, + 0xb0, 0xde, 0x0d, 0x62, 0x23, 0xc8, 0x61, 0x81, 0xdd, 0xbd, 0xac, 0xb6, 0x05, 0x12, 0x14, 0x29, + 0x02, 0x26, 0xed, 0xa1, 0x17, 0x81, 0x16, 0x09, 0x99, 0x88, 0x2d, 0x1a, 0x24, 0x6d, 0x24, 0x0f, + 0x51, 0xa0, 0x2f, 0xd0, 0xa2, 0x0f, 0xd1, 0x4b, 0xdf, 0xa0, 0xc7, 0xa0, 0xa7, 0x9e, 0x84, 0x22, + 0xb9, 0x14, 0x7a, 0x8a, 0x42, 0xa4, 0x2c, 0x3b, 0xe9, 0x25, 0x27, 0xcd, 0x7c, 0xf3, 0xcd, 0x88, + 0x1f, 0xe7, 0x03, 0xc1, 0x6f, 0x92, 0x12, 0x1c, 0x0c, 0x31, 0x8b, 0xfa, 0x03, 0xac, 0x82, 0x21, + 0x8b, 0xc2, 0xfe, 0xec, 0x30, 0x8f, 0x7b, 0x13, 0xc1, 0x15, 0x87, 0x5b, 0x39, 0xab, 0x97, 0x57, + 0x66, 0x87, 0x3b, 0x3f, 0x85, 0x3c, 0xe4, 0x9a, 0xd1, 0x4f, 0x23, 0x43, 0xde, 0xd9, 0x0e, 0xb8, + 0x1c, 0x73, 0xe9, 0x9b, 0x82, 0x49, 0x4c, 0xa9, 0xf3, 0xb6, 0x00, 0x56, 0xbd, 0x74, 0x00, 0xdc, + 0x05, 0x0d, 0x3d, 0xc9, 0x8f, 0xa6, 0xe3, 0x01, 0x15, 0xb6, 0xd5, 0xb6, 0xba, 0x25, 0x54, 0xd7, + 0xd8, 0x73, 0x0d, 0x69, 0xca, 0x88, 0x07, 0x97, 0xfe, 0x90, 0xb2, 0x70, 0xa8, 0xec, 0x42, 0xdb, + 0xea, 0x16, 0x51, 0x5d, 0x63, 0xc7, 0x1a, 0x82, 0x7f, 0x01, 0x3b, 0x98, 0x0a, 0x41, 0x23, 0xe5, + 0x13, 0xac, 0xb0, 0x2f, 0xa8, 0x9c, 0x8e, 0x94, 0x2f, 0x38, 0x57, 0x76, 0xb1, 0x6d, 0x75, 0x6b, + 0x68, 0x2b, 0xab, 0x3f, 0xc1, 0x0a, 0x23, 0x5d, 0x45, 0x9c, 0x2b, 0xd8, 0x05, 0xad, 0x1f, 0x1a, + 0x4a, 0xba, 0x61, 0x8d, 0xdc, 0x67, 0xee, 0x81, 0xb5, 0x19, 0x1e, 0x31, 0x82, 0x15, 0x17, 0x86, + 0xb7, 0xaa, 0x79, 0xcd, 0x1c, 0xd5, 0xb4, 0x6d, 0x50, 0x35, 0x7a, 0x18, 0xb1, 0xcb, 0x6d, 0xab, + 0xdb, 0x40, 0x15, 0x9d, 0x9f, 0x10, 0xf8, 0x07, 0x68, 0x4d, 0x04, 0x9f, 0xb1, 0x28, 0xf4, 0xc7, + 0x34, 0x9d, 0x4e, 0xb0, 0x5d, 0xd1, 0x94, 0xf5, 0x0c, 0x3f, 0xcd, 0xe0, 0xce, 0x6b, 0x0b, 0xd4, + 0x2f, 0x04, 0xa5, 0x4f, 0x23, 0x25, 0x18, 0x95, 0x8f, 0xb9, 0xa5, 0x1e, 0xd8, 0x5c, 0x56, 0x42, + 0x4d, 0xa7, 0x5d, 0x68, 0x17, 0xbb, 0x0d, 0xb4, 0xb1, 0x10, 0x33, 0x1f, 0xb9, 0x0f, 0x36, 0x16, + 0x7a, 0xe6, 0xec, 0xa2, 0x66, 0xb7, 0xf2, 0x42, 0x46, 0xee, 0xbc, 0xb3, 0xc0, 0xba, 0xde, 0xd7, + 0x39, 0x0b, 0x23, 0xac, 0xa6, 0xe2, 0x71, 0x67, 0x3a, 0x5e, 0xbe, 0x33, 0x4c, 0x88, 0xd0, 0xbb, + 0xab, 0x79, 0xbb, 0x9f, 0x3f, 0x1c, 0xfc, 0x9a, 0x19, 0xe2, 0xe5, 0x9c, 0xf0, 0x1f, 0x21, 0x82, + 0x4a, 0x79, 0xae, 0x04, 0x8b, 0xc2, 0xa5, 0x6b, 0x4d, 0x71, 0xe8, 0x00, 0x20, 0xf3, 0x5f, 0xeb, + 0x95, 0x36, 0xd0, 0x12, 0xd2, 0x79, 0x06, 0xca, 0x67, 0x58, 0xe0, 0xb1, 0x84, 0xff, 0x80, 0x9d, + 0xc5, 0x3f, 0x25, 0x55, 0xbe, 0x12, 0x6c, 0xec, 0x4f, 0xa8, 0x08, 0x68, 0xa4, 0xf4, 0x21, 0x9b, + 0xe8, 0x97, 0x9c, 0x71, 0x4e, 0xd5, 0x85, 0x60, 0xe3, 0x33, 0x53, 0xfe, 0xbb, 0xf4, 0xed, 0xbd, + 0x6b, 0x75, 0x3e, 0x16, 0x01, 0x58, 0xf8, 0x04, 0xfe, 0x0c, 0x0a, 0x8c, 0xe8, 0xce, 0x9a, 0x57, + 0x4e, 0x62, 0xb7, 0xc0, 0x08, 0x2a, 0x30, 0x02, 0x1d, 0xb0, 0x4a, 0x44, 0xba, 0x67, 0x23, 0xaa, + 0x96, 0xc4, 0xae, 0x01, 0x50, 0x89, 0x88, 0x13, 0x02, 0xf7, 0x40, 0x65, 0x46, 0x85, 0x64, 0x3c, + 0x32, 0x1e, 0xf4, 0xea, 0x49, 0xec, 0xce, 0x21, 0x34, 0x0f, 0xe0, 0xd1, 0x03, 0x7b, 0xa7, 0xf6, + 0x2b, 0x79, 0xad, 0x24, 0x76, 0xef, 0xe1, 0xf7, 0x0d, 0xff, 0x27, 0xa8, 0xd1, 0x2b, 0xa6, 0xfc, + 0x80, 0x13, 0xaa, 0x8d, 0xd8, 0xf4, 0x9a, 0x49, 0xec, 0x2e, 0x40, 0x54, 0x4d, 0xc3, 0xff, 0x39, + 0xa1, 0xf0, 0x77, 0x50, 0x0d, 0xb1, 0xf4, 0xa7, 0x92, 0x1a, 0x4b, 0x96, 0xbc, 0x46, 0x12, 0xbb, + 0x39, 0x86, 0x2a, 0x21, 0x96, 0x2f, 0x24, 0x25, 0xb0, 0x03, 0xca, 0xc6, 0x3d, 0xc6, 0x96, 0x1e, + 0x48, 0x62, 0x37, 0x43, 0x50, 0xf6, 0x85, 0xff, 0x82, 0xf5, 0x09, 0xbe, 0x1e, 0xe0, 0xe0, 0x52, + 0x2f, 0x94, 0x4a, 0x69, 0x57, 0xb5, 0xb8, 0xcd, 0x24, 0x76, 0x1f, 0x96, 0xd0, 0x5a, 0x06, 0x64, + 0xbb, 0x4d, 0xb5, 0xa6, 0x2f, 0x88, 0x3f, 0xc1, 0xd7, 0x23, 0x8e, 0x89, 0x5d, 0xd3, 0xad, 0x5a, + 0xeb, 0x32, 0x8e, 0xea, 0x69, 0x76, 0x66, 0x12, 0xb8, 0x0f, 0x6a, 0x01, 0x8f, 0x24, 0x8d, 0xe4, + 0x54, 0xda, 0xa0, 0x6d, 0x75, 0xab, 0x46, 0x6b, 0x0e, 0xa2, 0x45, 0xe8, 0x9d, 0x7e, 0xba, 0x75, + 0xac, 0x9b, 0x5b, 0xc7, 0xfa, 0x7a, 0xeb, 0x58, 0x6f, 0xee, 0x9c, 0x95, 0x9b, 0x3b, 0x67, 0xe5, + 0xcb, 0x9d, 0xb3, 0xf2, 0xea, 0x28, 0x64, 0x6a, 0x38, 0x1d, 0xf4, 0x02, 0x3e, 0xee, 0xa7, 0xe3, + 0xf5, 0x4b, 0x14, 0xf0, 0x91, 0x4e, 0x0e, 0xcc, 0xd3, 0x77, 0xb5, 0x78, 0xfc, 0xd4, 0xf5, 0x84, + 0xca, 0x41, 0x59, 0xb3, 0x8e, 0xbe, 0x07, 0x00, 0x00, 0xff, 0xff, 0x3b, 0x8c, 0x69, 0x47, 0x1f, + 0x05, 0x00, 0x00, } func (this *Params) Equal(that interface{}) bool { @@ -651,14 +660,19 @@ func (m *BatchSignatures) MarshalToSizedBuffer(dAtA []byte) (int, error) { copy(dAtA[i:], m.Signatures) i = encodeVarintBatching(dAtA, i, uint64(len(m.Signatures))) i-- - dAtA[i] = 0x12 + dAtA[i] = 0x1a } if len(m.ValidatorAddr) > 0 { i -= len(m.ValidatorAddr) copy(dAtA[i:], m.ValidatorAddr) i = encodeVarintBatching(dAtA, i, uint64(len(m.ValidatorAddr))) i-- - dAtA[i] = 0xa + dAtA[i] = 0x12 + } + if m.BatchNumber != 0 { + i = encodeVarintBatching(dAtA, i, uint64(m.BatchNumber)) + i-- + dAtA[i] = 0x8 } return len(dAtA) - i, nil } @@ -857,6 +871,9 @@ func (m *BatchSignatures) Size() (n int) { } var l int _ = l + if m.BatchNumber != 0 { + n += 1 + sovBatching(uint64(m.BatchNumber)) + } l = len(m.ValidatorAddr) if l > 0 { n += 1 + l + sovBatching(uint64(l)) @@ -1346,6 +1363,25 @@ func (m *BatchSignatures) Unmarshal(dAtA []byte) error { } switch fieldNum { case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field BatchNumber", wireType) + } + m.BatchNumber = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowBatching + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.BatchNumber |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 2: if wireType != 2 { return fmt.Errorf("proto: wrong wireType = %d for field ValidatorAddr", wireType) } @@ -1377,7 +1413,7 @@ func (m *BatchSignatures) Unmarshal(dAtA []byte) error { } m.ValidatorAddr = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex - case 2: + case 3: if wireType != 2 { return fmt.Errorf("proto: wrong wireType = %d for field Signatures", wireType) } diff --git a/x/batching/types/genesis.go b/x/batching/types/genesis.go index 8b1bdcab..082c5330 100644 --- a/x/batching/types/genesis.go +++ b/x/batching/types/genesis.go @@ -6,9 +6,10 @@ import "cosmossdk.io/collections" func NewGenesisState( curBatchNum uint64, batches []Batch, - entries []TreeEntries, + entries []TreeEntry, dataResults []DataResult, batchAssignments []BatchAssignment, + signatures []BatchSignatures, params Params, ) GenesisState { return GenesisState{ @@ -17,13 +18,14 @@ func NewGenesisState( TreeEntries: entries, DataResults: dataResults, BatchAssignments: batchAssignments, + BatchSignatures: signatures, Params: params, } } // DefaultGenesisState creates a default GenesisState object. func DefaultGenesisState() *GenesisState { - state := NewGenesisState(collections.DefaultSequenceStart, nil, nil, nil, nil, DefaultParams()) + state := NewGenesisState(collections.DefaultSequenceStart, nil, nil, nil, nil, nil, DefaultParams()) return &state } diff --git a/x/batching/types/genesis.pb.go b/x/batching/types/genesis.pb.go index 3e53542b..6113d8e5 100644 --- a/x/batching/types/genesis.pb.go +++ b/x/batching/types/genesis.pb.go @@ -29,10 +29,11 @@ type GenesisState struct { // created batch. CurrentBatchNumber uint64 `protobuf:"varint,1,opt,name=current_batch_number,json=currentBatchNumber,proto3" json:"current_batch_number,omitempty"` Batches []Batch `protobuf:"bytes,2,rep,name=batches,proto3" json:"batches"` - TreeEntries []TreeEntries `protobuf:"bytes,3,rep,name=tree_entries,json=treeEntries,proto3" json:"tree_entries"` + TreeEntries []TreeEntry `protobuf:"bytes,3,rep,name=tree_entries,json=treeEntries,proto3" json:"tree_entries"` DataResults []DataResult `protobuf:"bytes,4,rep,name=data_results,json=dataResults,proto3" json:"data_results"` BatchAssignments []BatchAssignment `protobuf:"bytes,5,rep,name=batch_assignments,json=batchAssignments,proto3" json:"batch_assignments"` - Params Params `protobuf:"bytes,6,opt,name=params,proto3" json:"params"` + BatchSignatures []BatchSignatures `protobuf:"bytes,6,rep,name=batch_signatures,json=batchSignatures,proto3" json:"batch_signatures"` + Params Params `protobuf:"bytes,7,opt,name=params,proto3" json:"params"` } func (m *GenesisState) Reset() { *m = GenesisState{} } @@ -82,7 +83,7 @@ func (m *GenesisState) GetBatches() []Batch { return nil } -func (m *GenesisState) GetTreeEntries() []TreeEntries { +func (m *GenesisState) GetTreeEntries() []TreeEntry { if m != nil { return m.TreeEntries } @@ -103,6 +104,13 @@ func (m *GenesisState) GetBatchAssignments() []BatchAssignment { return nil } +func (m *GenesisState) GetBatchSignatures() []BatchSignatures { + if m != nil { + return m.BatchSignatures + } + return nil +} + func (m *GenesisState) GetParams() Params { if m != nil { return m.Params @@ -164,9 +172,74 @@ func (m *BatchAssignment) GetDataRequestId() string { return "" } +// TreeEntry represents a tree entry for genesis export and import. +type TreeEntry struct { + // K1 is the first part of the key. It represents the batch number. + K1 uint64 `protobuf:"varint,1,opt,name=k1,proto3" json:"k1,omitempty"` + // K2 is the second part of the key. It is empty for a data result + // tree entry and validator address for a validator tree entry. + K2 []byte `protobuf:"bytes,2,opt,name=k2,proto3" json:"k2,omitempty"` + Entry []byte `protobuf:"bytes,3,opt,name=entry,proto3" json:"entry,omitempty"` +} + +func (m *TreeEntry) Reset() { *m = TreeEntry{} } +func (m *TreeEntry) String() string { return proto.CompactTextString(m) } +func (*TreeEntry) ProtoMessage() {} +func (*TreeEntry) Descriptor() ([]byte, []int) { + return fileDescriptor_eccca5d98d3cb479, []int{2} +} +func (m *TreeEntry) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *TreeEntry) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_TreeEntry.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *TreeEntry) XXX_Merge(src proto.Message) { + xxx_messageInfo_TreeEntry.Merge(m, src) +} +func (m *TreeEntry) XXX_Size() int { + return m.Size() +} +func (m *TreeEntry) XXX_DiscardUnknown() { + xxx_messageInfo_TreeEntry.DiscardUnknown(m) +} + +var xxx_messageInfo_TreeEntry proto.InternalMessageInfo + +func (m *TreeEntry) GetK1() uint64 { + if m != nil { + return m.K1 + } + return 0 +} + +func (m *TreeEntry) GetK2() []byte { + if m != nil { + return m.K2 + } + return nil +} + +func (m *TreeEntry) GetEntry() []byte { + if m != nil { + return m.Entry + } + return nil +} + func init() { proto.RegisterType((*GenesisState)(nil), "sedachain.batching.v1.GenesisState") proto.RegisterType((*BatchAssignment)(nil), "sedachain.batching.v1.BatchAssignment") + proto.RegisterType((*TreeEntry)(nil), "sedachain.batching.v1.TreeEntry") } func init() { @@ -174,33 +247,37 @@ func init() { } var fileDescriptor_eccca5d98d3cb479 = []byte{ - // 410 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x7c, 0x92, 0x4f, 0x6e, 0xd3, 0x40, - 0x14, 0x87, 0xed, 0x24, 0x04, 0x31, 0x36, 0x0a, 0x8c, 0x82, 0x64, 0x45, 0x60, 0x9c, 0x80, 0x22, - 0x6f, 0xb0, 0x49, 0xb2, 0x84, 0x0d, 0x11, 0x08, 0x01, 0x02, 0x21, 0xc3, 0xa6, 0x55, 0x25, 0x6b, - 0x6c, 0x3f, 0x39, 0x96, 0xe2, 0x71, 0x3a, 0x33, 0x8e, 0xda, 0x5b, 0xf4, 0x26, 0xbd, 0x46, 0x96, - 0x59, 0x76, 0x55, 0x55, 0xc9, 0x45, 0xaa, 0x8c, 0x1d, 0xa7, 0xad, 0x92, 0xee, 0xec, 0xf7, 0xbe, - 0xdf, 0x37, 0xff, 0x1e, 0x7a, 0xc7, 0x21, 0x22, 0xe1, 0x84, 0x24, 0xd4, 0x0d, 0x88, 0x08, 0x27, - 0x09, 0x8d, 0xdd, 0xf9, 0xc0, 0x8d, 0x81, 0x02, 0x4f, 0xb8, 0x33, 0x63, 0x99, 0xc8, 0xf0, 0xab, - 0x0a, 0x72, 0xb6, 0x90, 0x33, 0x1f, 0x74, 0xda, 0x71, 0x16, 0x67, 0x92, 0x70, 0x37, 0x5f, 0x05, - 0xdc, 0x79, 0xbf, 0xdf, 0x58, 0x05, 0x25, 0xd5, 0xbb, 0xac, 0x23, 0xfd, 0x7b, 0xb1, 0xc8, 0x3f, - 0x41, 0x04, 0xe0, 0x8f, 0xa8, 0x1d, 0xe6, 0x8c, 0x01, 0x15, 0xbe, 0x44, 0x7d, 0x9a, 0xa7, 0x01, - 0x30, 0x43, 0xb5, 0x54, 0xbb, 0xe1, 0xe1, 0xb2, 0x37, 0xde, 0xb4, 0xfe, 0xc8, 0x0e, 0xfe, 0x8c, - 0x9e, 0x4a, 0x12, 0xb8, 0x51, 0xb3, 0xea, 0xb6, 0x36, 0x7c, 0xed, 0xec, 0xdd, 0xa7, 0x23, 0x43, - 0xe3, 0xc6, 0xe2, 0xfa, 0xad, 0xe2, 0x6d, 0x23, 0xf8, 0x17, 0xd2, 0x05, 0x03, 0xf0, 0x81, 0x0a, - 0x96, 0x00, 0x37, 0xea, 0x52, 0xd1, 0x3b, 0xa0, 0xf8, 0xcf, 0x00, 0xbe, 0x15, 0x64, 0x29, 0xd2, - 0xc4, 0xae, 0x84, 0x7f, 0x22, 0x3d, 0x22, 0x82, 0xf8, 0x0c, 0x78, 0x3e, 0x15, 0xdc, 0x68, 0x48, - 0x59, 0xf7, 0x80, 0xec, 0x2b, 0x11, 0xc4, 0x93, 0xe4, 0xd6, 0x15, 0x55, 0x15, 0x8e, 0x8f, 0xd0, - 0xcb, 0xe2, 0x02, 0x08, 0xe7, 0x49, 0x4c, 0x53, 0xa0, 0x82, 0x1b, 0x4f, 0xa4, 0xb0, 0xff, 0xd8, - 0x01, 0xbf, 0x54, 0x78, 0x69, 0x7d, 0x11, 0xdc, 0x2f, 0x73, 0xfc, 0x09, 0x35, 0x67, 0x84, 0x91, - 0x94, 0x1b, 0x4d, 0x4b, 0xb5, 0xb5, 0xe1, 0x9b, 0x03, 0xbe, 0xbf, 0x12, 0x2a, 0x35, 0x65, 0xa4, - 0x77, 0x82, 0x5a, 0x0f, 0xd6, 0xc1, 0x5d, 0xa4, 0xef, 0x79, 0x2b, 0x2d, 0xb8, 0xf3, 0x48, 0x7d, - 0xd4, 0x2a, 0x6f, 0xe6, 0x34, 0x07, 0x2e, 0xfc, 0x24, 0x32, 0x6a, 0x96, 0x6a, 0x3f, 0xf3, 0x9e, - 0x17, 0x67, 0x96, 0xd5, 0x1f, 0xd1, 0xf8, 0xf7, 0x62, 0x65, 0xaa, 0xcb, 0x95, 0xa9, 0xde, 0xac, - 0x4c, 0xf5, 0x62, 0x6d, 0x2a, 0xcb, 0xb5, 0xa9, 0x5c, 0xad, 0x4d, 0xe5, 0x78, 0x14, 0x27, 0x62, - 0x92, 0x07, 0x4e, 0x98, 0xa5, 0xee, 0x66, 0xbb, 0x72, 0x7e, 0xc2, 0x6c, 0x2a, 0x7f, 0x3e, 0x14, - 0x83, 0x76, 0xb6, 0x1b, 0x35, 0x71, 0x3e, 0x03, 0x1e, 0x34, 0x25, 0x35, 0xba, 0x0d, 0x00, 0x00, - 0xff, 0xff, 0xed, 0x8e, 0x51, 0x01, 0xdf, 0x02, 0x00, 0x00, + // 469 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x84, 0x93, 0x41, 0x8b, 0xd3, 0x40, + 0x14, 0xc7, 0x9b, 0xb4, 0xdb, 0x65, 0xa7, 0xd1, 0xea, 0x50, 0x21, 0x2c, 0x1a, 0xb3, 0x55, 0x96, + 0x5e, 0x4c, 0x6c, 0xf7, 0xa8, 0x97, 0x2d, 0x8a, 0xac, 0xa0, 0x48, 0x56, 0x10, 0x45, 0x08, 0x93, + 0xe4, 0x91, 0x86, 0x6e, 0x27, 0x75, 0x66, 0xb2, 0xd8, 0x2f, 0xe0, 0xd9, 0x8f, 0xb5, 0xc7, 0x3d, + 0x7a, 0x12, 0x69, 0xbf, 0x88, 0xe4, 0x65, 0x1a, 0xad, 0xb4, 0xee, 0xad, 0xf3, 0x9f, 0xdf, 0xfb, + 0x35, 0x6f, 0xe6, 0x0d, 0x79, 0x24, 0x21, 0x61, 0xf1, 0x84, 0x65, 0xdc, 0x8f, 0x98, 0x8a, 0x27, + 0x19, 0x4f, 0xfd, 0xcb, 0xa1, 0x9f, 0x02, 0x07, 0x99, 0x49, 0x6f, 0x2e, 0x72, 0x95, 0xd3, 0x7b, + 0x35, 0xe4, 0xad, 0x21, 0xef, 0x72, 0x78, 0xd8, 0x4b, 0xf3, 0x34, 0x47, 0xc2, 0x2f, 0x7f, 0x55, + 0xf0, 0xe1, 0xe3, 0xed, 0xc6, 0xba, 0x10, 0xa9, 0xfe, 0xb7, 0x16, 0xb1, 0x5e, 0x55, 0x7f, 0x72, + 0xae, 0x98, 0x02, 0xfa, 0x94, 0xf4, 0xe2, 0x42, 0x08, 0xe0, 0x2a, 0x44, 0x34, 0xe4, 0xc5, 0x2c, + 0x02, 0x61, 0x1b, 0xae, 0x31, 0x68, 0x05, 0x54, 0xef, 0x8d, 0xcb, 0xad, 0xb7, 0xb8, 0x43, 0x9f, + 0x93, 0x7d, 0x24, 0x41, 0xda, 0xa6, 0xdb, 0x1c, 0x74, 0x46, 0xf7, 0xbd, 0xad, 0xdf, 0xe9, 0x61, + 0xd1, 0xb8, 0x75, 0xf5, 0xf3, 0x61, 0x23, 0x58, 0x97, 0xd0, 0x33, 0x62, 0x29, 0x01, 0x10, 0x02, + 0x57, 0x22, 0x03, 0x69, 0x37, 0x51, 0xe1, 0xee, 0x50, 0xbc, 0x17, 0x00, 0x2f, 0xb9, 0x12, 0x0b, + 0xad, 0xe9, 0x28, 0x1d, 0x64, 0x20, 0xe9, 0x6b, 0x62, 0x25, 0x4c, 0xb1, 0x50, 0x80, 0x2c, 0x2e, + 0x94, 0xb4, 0x5b, 0xa8, 0x3a, 0xda, 0xa1, 0x7a, 0xc1, 0x14, 0x0b, 0x90, 0x5c, 0xbb, 0x92, 0x3a, + 0x91, 0xf4, 0x23, 0xb9, 0x5b, 0xb5, 0xcf, 0xa4, 0xcc, 0x52, 0x3e, 0x03, 0xae, 0xa4, 0xbd, 0x87, + 0xc2, 0xe3, 0xff, 0xb5, 0x77, 0x5a, 0xe3, 0xda, 0x7a, 0x27, 0xda, 0x8c, 0x25, 0xfd, 0x40, 0xaa, + 0x2c, 0x2c, 0x23, 0xa6, 0x0a, 0x01, 0xd2, 0x6e, 0xdf, 0x6c, 0x3e, 0xaf, 0x69, 0x6d, 0xee, 0x46, + 0x9b, 0x31, 0x7d, 0x46, 0xda, 0x73, 0x26, 0xd8, 0x4c, 0xda, 0xfb, 0xae, 0x31, 0xe8, 0x8c, 0x1e, + 0xec, 0xd0, 0xbd, 0x43, 0x48, 0x5b, 0x74, 0x49, 0xff, 0x33, 0xe9, 0xfe, 0xd3, 0x00, 0x3d, 0x22, + 0xd6, 0x96, 0x11, 0xe8, 0x44, 0x7f, 0xdd, 0xfd, 0x31, 0xe9, 0xea, 0x23, 0xff, 0x52, 0x80, 0x54, + 0x61, 0x96, 0xd8, 0xa6, 0x6b, 0x0c, 0x0e, 0x82, 0x5b, 0xd5, 0x61, 0x62, 0x7a, 0x96, 0xf4, 0x4f, + 0xc9, 0x41, 0x7d, 0x75, 0xf4, 0x36, 0x31, 0xa7, 0x43, 0x6d, 0x33, 0xa7, 0x43, 0x5c, 0x8f, 0xb0, + 0xce, 0x0a, 0xcc, 0xe9, 0x88, 0xf6, 0xc8, 0x5e, 0x39, 0x0d, 0x0b, 0xbb, 0x89, 0x51, 0xb5, 0x18, + 0xbf, 0xb9, 0x5a, 0x3a, 0xc6, 0xf5, 0xd2, 0x31, 0x7e, 0x2d, 0x1d, 0xe3, 0xfb, 0xca, 0x69, 0x5c, + 0xaf, 0x9c, 0xc6, 0x8f, 0x95, 0xd3, 0xf8, 0x74, 0x92, 0x66, 0x6a, 0x52, 0x44, 0x5e, 0x9c, 0xcf, + 0xfc, 0xb2, 0x63, 0x9c, 0xec, 0x38, 0xbf, 0xc0, 0xc5, 0x93, 0xea, 0x09, 0x7c, 0xfd, 0xf3, 0x08, + 0xd4, 0x62, 0x0e, 0x32, 0x6a, 0x23, 0x75, 0xf2, 0x3b, 0x00, 0x00, 0xff, 0xff, 0x1b, 0x0c, 0x0e, + 0xd5, 0x79, 0x03, 0x00, 0x00, } func (m *GenesisState) Marshal() (dAtA []byte, err error) { @@ -232,7 +309,21 @@ func (m *GenesisState) MarshalToSizedBuffer(dAtA []byte) (int, error) { i = encodeVarintGenesis(dAtA, i, uint64(size)) } i-- - dAtA[i] = 0x32 + dAtA[i] = 0x3a + if len(m.BatchSignatures) > 0 { + for iNdEx := len(m.BatchSignatures) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.BatchSignatures[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintGenesis(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x32 + } + } if len(m.BatchAssignments) > 0 { for iNdEx := len(m.BatchAssignments) - 1; iNdEx >= 0; iNdEx-- { { @@ -332,6 +423,48 @@ func (m *BatchAssignment) MarshalToSizedBuffer(dAtA []byte) (int, error) { return len(dAtA) - i, nil } +func (m *TreeEntry) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *TreeEntry) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *TreeEntry) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if len(m.Entry) > 0 { + i -= len(m.Entry) + copy(dAtA[i:], m.Entry) + i = encodeVarintGenesis(dAtA, i, uint64(len(m.Entry))) + i-- + dAtA[i] = 0x1a + } + if len(m.K2) > 0 { + i -= len(m.K2) + copy(dAtA[i:], m.K2) + i = encodeVarintGenesis(dAtA, i, uint64(len(m.K2))) + i-- + dAtA[i] = 0x12 + } + if m.K1 != 0 { + i = encodeVarintGenesis(dAtA, i, uint64(m.K1)) + i-- + dAtA[i] = 0x8 + } + return len(dAtA) - i, nil +} + func encodeVarintGenesis(dAtA []byte, offset int, v uint64) int { offset -= sovGenesis(v) base := offset @@ -376,6 +509,12 @@ func (m *GenesisState) Size() (n int) { n += 1 + l + sovGenesis(uint64(l)) } } + if len(m.BatchSignatures) > 0 { + for _, e := range m.BatchSignatures { + l = e.Size() + n += 1 + l + sovGenesis(uint64(l)) + } + } l = m.Params.Size() n += 1 + l + sovGenesis(uint64(l)) return n @@ -397,6 +536,26 @@ func (m *BatchAssignment) Size() (n int) { return n } +func (m *TreeEntry) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.K1 != 0 { + n += 1 + sovGenesis(uint64(m.K1)) + } + l = len(m.K2) + if l > 0 { + n += 1 + l + sovGenesis(uint64(l)) + } + l = len(m.Entry) + if l > 0 { + n += 1 + l + sovGenesis(uint64(l)) + } + return n +} + func sovGenesis(x uint64) (n int) { return (math_bits.Len64(x|1) + 6) / 7 } @@ -514,7 +673,7 @@ func (m *GenesisState) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.TreeEntries = append(m.TreeEntries, TreeEntries{}) + m.TreeEntries = append(m.TreeEntries, TreeEntry{}) if err := m.TreeEntries[len(m.TreeEntries)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -588,6 +747,40 @@ func (m *GenesisState) Unmarshal(dAtA []byte) error { } iNdEx = postIndex case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BatchSignatures", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowGenesis + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthGenesis + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthGenesis + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.BatchSignatures = append(m.BatchSignatures, BatchSignatures{}) + if err := m.BatchSignatures[len(m.BatchSignatures)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 7: if wireType != 2 { return fmt.Errorf("proto: wrong wireType = %d for field Params", wireType) } @@ -742,6 +935,143 @@ func (m *BatchAssignment) Unmarshal(dAtA []byte) error { } return nil } +func (m *TreeEntry) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowGenesis + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: TreeEntry: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: TreeEntry: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field K1", wireType) + } + m.K1 = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowGenesis + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.K1 |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field K2", wireType) + } + var byteLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowGenesis + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + byteLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if byteLen < 0 { + return ErrInvalidLengthGenesis + } + postIndex := iNdEx + byteLen + if postIndex < 0 { + return ErrInvalidLengthGenesis + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.K2 = append(m.K2[:0], dAtA[iNdEx:postIndex]...) + if m.K2 == nil { + m.K2 = []byte{} + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Entry", wireType) + } + var byteLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowGenesis + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + byteLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if byteLen < 0 { + return ErrInvalidLengthGenesis + } + postIndex := iNdEx + byteLen + if postIndex < 0 { + return ErrInvalidLengthGenesis + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Entry = append(m.Entry[:0], dAtA[iNdEx:postIndex]...) + if m.Entry == nil { + m.Entry = []byte{} + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipGenesis(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthGenesis + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} func skipGenesis(dAtA []byte) (n int, err error) { l := len(dAtA) iNdEx := 0 diff --git a/x/pubkey/types/pubkey.pb.go b/x/pubkey/types/pubkey.pb.go index 6fab7119..312a4726 100644 --- a/x/pubkey/types/pubkey.pb.go +++ b/x/pubkey/types/pubkey.pb.go @@ -6,7 +6,6 @@ package types import ( fmt "fmt" _ "github.com/cosmos/cosmos-proto" - _ "github.com/cosmos/cosmos-sdk/codec/types" proto "github.com/cosmos/gogoproto/proto" io "io" math "math" @@ -84,22 +83,21 @@ func init() { func init() { proto.RegisterFile("sedachain/pubkey/v1/pubkey.proto", fileDescriptor_a51ebcd05a6c14e0) } var fileDescriptor_a51ebcd05a6c14e0 = []byte{ - // 234 bytes of a gzipped FileDescriptorProto + // 215 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe2, 0x52, 0x28, 0x4e, 0x4d, 0x49, 0x4c, 0xce, 0x48, 0xcc, 0xcc, 0xd3, 0x2f, 0x28, 0x4d, 0xca, 0x4e, 0xad, 0xd4, 0x2f, 0x33, 0x84, 0xb2, 0xf4, 0x0a, 0x8a, 0xf2, 0x4b, 0xf2, 0x85, 0x84, 0xe1, 0x2a, 0xf4, 0xa0, 0xe2, 0x65, 0x86, - 0x52, 0x92, 0xe9, 0xf9, 0xf9, 0xe9, 0x39, 0xa9, 0xfa, 0x60, 0x25, 0x49, 0xa5, 0x69, 0xfa, 0x89, - 0x79, 0x50, 0xf5, 0x52, 0x92, 0xc9, 0xf9, 0xc5, 0xb9, 0xf9, 0xc5, 0xf1, 0x60, 0x9e, 0x3e, 0x84, - 0x03, 0x91, 0x52, 0x8a, 0xe0, 0xe2, 0xf5, 0xcc, 0x4b, 0x49, 0xad, 0x48, 0x4d, 0x09, 0x28, 0x4d, - 0xf2, 0x4e, 0xad, 0x14, 0x12, 0xe1, 0x62, 0xcd, 0x04, 0x09, 0x48, 0x30, 0x2a, 0x30, 0x6a, 0xf0, - 0x06, 0x41, 0x38, 0x42, 0x86, 0x5c, 0xec, 0x05, 0xa5, 0x49, 0xf1, 0xd9, 0xa9, 0x95, 0x12, 0x4c, - 0x0a, 0x8c, 0x1a, 0x3c, 0x4e, 0x12, 0xa7, 0xb6, 0xe8, 0x8a, 0x40, 0x4d, 0x4a, 0x2e, 0xaa, 0x2c, - 0x28, 0xc9, 0xd7, 0x83, 0x18, 0x10, 0xc4, 0x56, 0x00, 0xa6, 0x9d, 0xbc, 0x4f, 0x3c, 0x92, 0x63, - 0xbc, 0xf0, 0x48, 0x8e, 0xf1, 0xc1, 0x23, 0x39, 0xc6, 0x09, 0x8f, 0xe5, 0x18, 0x2e, 0x3c, 0x96, - 0x63, 0xb8, 0xf1, 0x58, 0x8e, 0x21, 0xca, 0x30, 0x3d, 0xb3, 0x24, 0xa3, 0x34, 0x49, 0x2f, 0x39, - 0x3f, 0x57, 0x1f, 0xe4, 0x13, 0xb0, 0x4b, 0x92, 0xf3, 0x73, 0xc0, 0x1c, 0x5d, 0x88, 0xcf, 0x2b, - 0x60, 0x7e, 0x2f, 0xa9, 0x2c, 0x48, 0x2d, 0x4e, 0x62, 0x03, 0xab, 0x31, 0x06, 0x04, 0x00, 0x00, - 0xff, 0xff, 0xc5, 0x0a, 0xa3, 0xfa, 0x1c, 0x01, 0x00, 0x00, + 0x52, 0x92, 0xc9, 0xf9, 0xc5, 0xb9, 0xf9, 0xc5, 0xf1, 0x60, 0x25, 0xfa, 0x10, 0x0e, 0x44, 0xbd, + 0x52, 0x04, 0x17, 0xaf, 0x67, 0x5e, 0x4a, 0x6a, 0x45, 0x6a, 0x4a, 0x40, 0x69, 0x92, 0x77, 0x6a, + 0xa5, 0x90, 0x08, 0x17, 0x6b, 0x26, 0x48, 0x40, 0x82, 0x51, 0x81, 0x51, 0x83, 0x37, 0x08, 0xc2, + 0x11, 0x32, 0xe4, 0x62, 0x2f, 0x28, 0x4d, 0x8a, 0xcf, 0x4e, 0xad, 0x94, 0x60, 0x52, 0x60, 0xd4, + 0xe0, 0x71, 0x92, 0x38, 0xb5, 0x45, 0x57, 0x04, 0x6a, 0x52, 0x72, 0x51, 0x65, 0x41, 0x49, 0xbe, + 0x1e, 0xc4, 0x80, 0x20, 0xb6, 0x02, 0x30, 0xed, 0xe4, 0x7d, 0xe2, 0x91, 0x1c, 0xe3, 0x85, 0x47, + 0x72, 0x8c, 0x0f, 0x1e, 0xc9, 0x31, 0x4e, 0x78, 0x2c, 0xc7, 0x70, 0xe1, 0xb1, 0x1c, 0xc3, 0x8d, + 0xc7, 0x72, 0x0c, 0x51, 0x86, 0xe9, 0x99, 0x25, 0x19, 0xa5, 0x49, 0x7a, 0xc9, 0xf9, 0xb9, 0xfa, + 0x20, 0xe7, 0x82, 0x5d, 0x92, 0x9c, 0x9f, 0x03, 0xe6, 0xe8, 0x42, 0xbc, 0x57, 0x01, 0xf3, 0x60, + 0x49, 0x65, 0x41, 0x6a, 0x71, 0x12, 0x1b, 0x58, 0x8d, 0x31, 0x20, 0x00, 0x00, 0xff, 0xff, 0xf2, + 0x9a, 0x73, 0x37, 0x01, 0x01, 0x00, 0x00, } func (m *IndexedPubKey) Marshal() (dAtA []byte, err error) {