From 3ae00caed6c11892f040d806e9aa5c2bffeffaa2 Mon Sep 17 00:00:00 2001 From: p4u Date: Wed, 27 Sep 2023 09:16:19 +0200 Subject: [PATCH 01/16] vochain: order transactions when creating new blocks Signed-off-by: p4u --- apiclient/account.go | 10 +- cmd/end2endtest/account.go | 65 +++++++--- dockerfiles/testsuite/env.gateway0 | 2 +- dockerfiles/testsuite/genesis.json | 14 +- vochain/app.go | 142 ++++++++++++++++++--- vochain/proposal_test.go | 103 +++++++++++++++ vochain/transaction/account_tx.go | 14 -- vochain/transaction/election_tx.go | 8 -- vochain/transaction/nonce.go | 111 ++++++++++++++++ vochain/transaction/tokens_tx.go | 10 -- vochain/transaction/transaction.go | 7 +- vochain/transaction/vochaintx/vochaintx.go | 9 +- 12 files changed, 412 insertions(+), 83 deletions(-) create mode 100644 vochain/proposal_test.go create mode 100644 vochain/transaction/nonce.go diff --git a/apiclient/account.go b/apiclient/account.go index 08784d251..abba79d07 100644 --- a/apiclient/account.go +++ b/apiclient/account.go @@ -69,18 +69,26 @@ func (c *HTTPclient) Account(address string) (*api.Account, error) { } // Transfer sends tokens from the account associated with the client to the given address. +// The nonce is automatically calculated from the account information. // Returns the transaction hash. func (c *HTTPclient) Transfer(to common.Address, amount uint64) (types.HexBytes, error) { acc, err := c.Account("") if err != nil { return nil, err } + return c.TransferWithNonce(to, amount, acc.Nonce) +} + +// TransferWithNonce sends tokens from the account associated with the client to the given address. +// Returns the transaction hash. +func (c *HTTPclient) TransferWithNonce(to common.Address, amount uint64, nonce uint32) (types.HexBytes, error) { + var err error stx := models.SignedTx{} stx.Tx, err = proto.Marshal(&models.Tx{ Payload: &models.Tx_SendTokens{ SendTokens: &models.SendTokensTx{ Txtype: models.TxType_SET_ACCOUNT_INFO_URI, - Nonce: acc.Nonce, + Nonce: nonce, From: c.account.Address().Bytes(), To: to.Bytes(), Value: amount, diff --git a/cmd/end2endtest/account.go b/cmd/end2endtest/account.go index f3e379933..1e74ac415 100644 --- a/cmd/end2endtest/account.go +++ b/cmd/end2endtest/account.go @@ -6,6 +6,7 @@ import ( "fmt" "os" "strings" + "sync" "time" "github.com/ethereum/go-ethereum/common" @@ -150,6 +151,8 @@ func testSendTokens(api *apiclient.HTTPclient, aliceKeys, bobKeys *ethereum.Sign // both pay 2 for each tx // resulting in balance 52 for alice // and 44 for bob + // In addition, we send a couple of token txs to burn address to increase the nonce, + // without waiting for them to be mined (this tests that the mempool transactions are properly ordered). txCost, err := api.TransactionCost(models.TxType_SEND_TOKENS) if err != nil { @@ -181,23 +184,53 @@ func testSendTokens(api *apiclient.HTTPclient, aliceKeys, bobKeys *ethereum.Sign // try to send tokens at the same time: // alice sends 1/4 of her balance to bob // sends 1/3 of his balance to alice - amountAtoB := aliceAcc.Balance / 4 - amountBtoA := bobAcc.Balance / 3 + // Subtract 1 + txCost from each since we are sending an extra tx to increase the nonce to the burn address + amountAtoB := (aliceAcc.Balance) / 4 + amountBtoA := (bobAcc.Balance) / 3 - txhasha, err := alice.Transfer(bobKeys.Address(), amountAtoB) + _, err = alice.TransferWithNonce(state.BurnAddress, 2, aliceAcc.Nonce+10) if err != nil { - return fmt.Errorf("cannot send tokens: %v", err) + log.Fatalf("cannot burn tokens: %v", err) } - log.Infof("alice sent %d tokens to bob", amountAtoB) - log.Debugf("tx hash is %x", txhasha) - txhashb, err := bob.Transfer(aliceKeys.Address(), amountBtoA) - if err != nil { - return fmt.Errorf("cannot send tokens: %v", err) - } - log.Infof("bob sent %d tokens to alice", amountBtoA) - log.Debugf("tx hash is %x", txhashb) + // send a couple of token txs to increase the nonce, without waiting for them to be mined + // this tests that the mempool transactions are properly ordered. + wg := sync.WaitGroup{} + wg.Add(1) + go func() { + // send 1 token to burn address with nonce + 1 (should be mined after the other txs) + _, err = alice.TransferWithNonce(state.BurnAddress, 1, aliceAcc.Nonce+1) + if err != nil { + log.Fatalf("cannot burn tokens: %v", err) + } + _, err = bob.TransferWithNonce(state.BurnAddress, 1, bobAcc.Nonce+1) + if err != nil { + log.Fatalf("cannot burn tokens: %v", err) + } + + wg.Done() + }() + + var txhasha, txhashb []byte + wg.Add(1) + go func() { + txhasha, err = alice.TransferWithNonce(bobKeys.Address(), amountAtoB, aliceAcc.Nonce) + if err != nil { + log.Fatalf("cannot send tokens: %v", err) + } + log.Infof("alice sent %d tokens to bob", amountAtoB) + log.Debugf("tx hash is %x", txhasha) + + txhashb, err = bob.TransferWithNonce(aliceKeys.Address(), amountBtoA, bobAcc.Nonce) + if err != nil { + log.Fatalf("cannot send tokens: %v", err) + } + log.Infof("bob sent %d tokens to alice", amountBtoA) + log.Debugf("tx hash is %x", txhashb) + wg.Done() + }() + wg.Wait() ctx, cancel := context.WithTimeout(context.Background(), time.Second*40) defer cancel() txrefa, err := api.WaitUntilTxIsMined(ctx, txhasha) @@ -216,12 +249,12 @@ func testSendTokens(api *apiclient.HTTPclient, aliceKeys, bobKeys *ethereum.Sign _ = api.WaitUntilNextBlock() // now check the resulting state - if err := checkAccountNonceAndBalance(alice, aliceAcc.Nonce+1, - aliceAcc.Balance-amountAtoB-txCost+amountBtoA); err != nil { + if err := checkAccountNonceAndBalance(alice, aliceAcc.Nonce+2, + aliceAcc.Balance-amountAtoB-(2*txCost+1)+amountBtoA); err != nil { return err } - if err := checkAccountNonceAndBalance(bob, bobAcc.Nonce+1, - bobAcc.Balance-amountBtoA-txCost+amountAtoB); err != nil { + if err := checkAccountNonceAndBalance(bob, bobAcc.Nonce+2, + bobAcc.Balance-amountBtoA-(2*txCost+1)+amountAtoB); err != nil { return err } diff --git a/dockerfiles/testsuite/env.gateway0 b/dockerfiles/testsuite/env.gateway0 index 7d4d833e9..82862cbfd 100755 --- a/dockerfiles/testsuite/env.gateway0 +++ b/dockerfiles/testsuite/env.gateway0 @@ -10,4 +10,4 @@ VOCDONI_VOCHAIN_NOWAITSYNC=True VOCDONI_METRICS_ENABLED=True VOCDONI_METRICS_REFRESHINTERVAL=5 VOCDONI_CHAIN=dev -VOCDONI_SIGNINGKEY=e0f1412b86d6ca9f2b318f1d243ef50be23d315a2e6c1c3035bc72d44c8b2f90 +VOCDONI_SIGNINGKEY=e0f1412b86d6ca9f2b318f1d243ef50be23d315a2e6c1c3035bc72d44c8b2f90 # 0x88a499cEf9D1330111b41360173967c9C1bf703f diff --git a/dockerfiles/testsuite/genesis.json b/dockerfiles/testsuite/genesis.json index 98d395039..cee349070 100755 --- a/dockerfiles/testsuite/genesis.json +++ b/dockerfiles/testsuite/genesis.json @@ -57,20 +57,8 @@ ], "accounts":[ { - "address":"0xccEc2c2D658261Fbdc40b04FEc06d49057242D39", - "balance":10000000 - }, - { - "address":"0x776d858D17C8018F07899dB535866EBf805a32E0", - "balance":10000000 - }, - { - "address":"0x074fcAacb8B01850539eaE7E9fEE8dc94549db96", - "balance":10000000 - }, - { "address":"0x88a499cEf9D1330111b41360173967c9C1bf703f", - "balance":10000000 + "balance":1000000000000 } ], "treasurer": "0xfe10DAB06D636647f4E40dFd56599da9eF66Db1c", diff --git a/vochain/app.go b/vochain/app.go index 6121778eb..e70fab109 100644 --- a/vochain/app.go +++ b/vochain/app.go @@ -7,6 +7,8 @@ import ( "errors" "fmt" "path/filepath" + "sort" + "sync" "sync/atomic" "time" @@ -16,6 +18,7 @@ import ( tmcli "github.com/cometbft/cometbft/rpc/client/local" ctypes "github.com/cometbft/cometbft/rpc/core/types" tmtypes "github.com/cometbft/cometbft/types" + "github.com/ethereum/go-ethereum/common" ethcommon "github.com/ethereum/go-ethereum/common" lru "github.com/hashicorp/golang-lru/v2" "go.vocdoni.io/dvote/crypto/zk/circuit" @@ -33,7 +36,10 @@ import ( const ( // recheckTxHeightInterval is the number of blocks after which the mempool is // checked for transactions to be rechecked. - recheckTxHeightInterval = 12 + recheckTxHeightInterval = 6 * 5 // 5 minutes + // transactionBlocksTTL is the number of blocks after which a transaction is + // removed from the mempool. + transactionBlocksTTL = 6 * 10 // 10 minutes ) var ( @@ -64,6 +70,8 @@ type BaseApplication struct { fnMempoolSize func() int fnMempoolPrune func(txKey [32]byte) error blockCache *lru.Cache[int64, *tmtypes.Block] + // txTTLReferences is a map of tx hashes to the block height where they failed. + txTTLReferences sync.Map // endBlockTimestamp is the last block end timestamp calculated from local time. endBlockTimestamp atomic.Int64 // startBlockTimestamp is the current block timestamp from tendermint's @@ -74,6 +82,9 @@ type BaseApplication struct { dataDir string genesisInfo *tmtypes.GenesisDoc + // prepareProposalLock is used to avoid concurrent calls between PrepareProposal and FinalizeBlock + prepareProposalLock sync.Mutex + // testMockBlockStore is used for testing purposes only testMockBlockStore *testutil.MockBlockStore } @@ -257,11 +268,23 @@ func (app *BaseApplication) InitChain(_ context.Context, // CheckTx unmarshals req.Tx and checks its validity func (app *BaseApplication) CheckTx(_ context.Context, req *abcitypes.RequestCheckTx) (*abcitypes.ResponseCheckTx, error) { + txReference := vochaintx.TxKey(req.Tx) + // store the initial height of the tx + initialTTLheight, isReferenced := app.txTTLReferences.LoadOrStore(txReference, app.Height()) + // check if the tx is referenced by a previous block and the TTL has expired + if isReferenced && app.Height() > initialTTLheight.(uint32)+transactionBlocksTTL { + // remove tx reference and return checkTx error + log.Debugw("pruning expired tx from mempool", "height", app.Height(), "hash", fmt.Sprintf("%x", txReference)) + app.txTTLReferences.Delete(txReference) + return &abcitypes.ResponseCheckTx{Code: 1, Data: []byte(fmt.Sprintf("tx expired %x", txReference))}, nil + } + // execute recheck mempool every recheckTxHeightInterval blocks if req.Type == abcitypes.CheckTxType_Recheck { if app.Height()%recheckTxHeightInterval != 0 { return &abcitypes.ResponseCheckTx{Code: 0}, nil } } + // unmarshal tx and check it tx := new(vochaintx.Tx) if err := tx.Unmarshal(req.Tx, app.ChainID()); err != nil { return &abcitypes.ResponseCheckTx{Code: 1, Data: []byte("unmarshalTx " + err.Error())}, err @@ -289,6 +312,8 @@ func (app *BaseApplication) CheckTx(_ context.Context, // CometBFT calls it when a new block is decided. func (app *BaseApplication) FinalizeBlock(_ context.Context, req *abcitypes.RequestFinalizeBlock) (*abcitypes.ResponseFinalizeBlock, error) { + app.prepareProposalLock.Lock() + defer app.prepareProposalLock.Unlock() height := uint32(req.GetHeight()) app.beginBlock(req.GetTime(), height) txResults := make([]*abcitypes.ExecTxResult, len(req.Txs)) @@ -314,7 +339,7 @@ func (app *BaseApplication) FinalizeBlock(_ context.Context, app.endBlock(req.GetTime(), height) return &abcitypes.ResponseFinalizeBlock{ AppHash: app.State.WorkingHash(), - TxResults: txResults, // TODO: check if we can remove this + TxResults: txResults, }, nil } @@ -363,6 +388,7 @@ func (app *BaseApplication) deliverTx(rawTx []byte) *DeliverTxResponse { log.Errorw(err, "rejected tx") return &DeliverTxResponse{Code: 1, Data: []byte(err.Error())} } + app.txTTLReferences.Delete(tx.TxID) // call event listeners for _, e := range app.State.EventListeners() { e.OnNewTx(tx, app.Height(), app.State.TxCounter()) @@ -474,25 +500,75 @@ func (*BaseApplication) Query(_ context.Context, // the ResponsePrepareProposal call. The logic modifying the raw proposal MAY be non-deterministic. func (app *BaseApplication) PrepareProposal(ctx context.Context, req *abcitypes.RequestPrepareProposal) (*abcitypes.ResponsePrepareProposal, error) { - // TODO: Prepare Proposal should check the validity of the transactions for the next block. - // Currently they are executed by CheckTx, but it does not allow height to be passed in. - validTxs := [][]byte{} + app.prepareProposalLock.Lock() + defer app.prepareProposalLock.Unlock() + type txInfo struct { + Data []byte + Addr *common.Address + Nonce uint32 + DecodedTx *vochaintx.Tx + } + + validTxInfos := []txInfo{} for _, tx := range req.GetTxs() { - resp, err := app.CheckTx(ctx, &abcitypes.RequestCheckTx{ - Tx: tx, Type: abcitypes.CheckTxType_New, + vtx := new(vochaintx.Tx) + if err := vtx.Unmarshal(tx, app.ChainID()); err != nil { + // invalid transaction + log.Warnw("could not unmarshal transaction", "err", err) + continue + } + senderAddr, nonce, err := app.TransactionHandler.ExtractNonceAndSender(vtx) + if err != nil { + log.Warnw("could not extract nonce and/or sender from transaction", "err", err) + continue + } + + validTxInfos = append(validTxInfos, txInfo{ + Data: tx, + Addr: senderAddr, + Nonce: nonce, + DecodedTx: vtx, }) - if err != nil || resp.Code != 0 { + } + + // Sort the transactions based on the sender's address and nonce + sort.Slice(validTxInfos, func(i, j int) bool { + if validTxInfos[i].Addr == nil && validTxInfos[j].Addr != nil { + return true + } + if validTxInfos[i].Addr != nil && validTxInfos[j].Addr == nil { + return false + } + if validTxInfos[i].Addr != nil && validTxInfos[j].Addr != nil { + if validTxInfos[i].Addr.String() == validTxInfos[j].Addr.String() { + return validTxInfos[i].Nonce < validTxInfos[j].Nonce + } + return validTxInfos[i].Addr.String() < validTxInfos[j].Addr.String() + } + return false + }) + + // Check the validity of the transactions + validTxs := [][]byte{} + for _, txInfo := range validTxInfos { + // Check the validity of the transaction using forCommit true + resp, err := app.TransactionHandler.CheckTx(txInfo.DecodedTx, true) + if err != nil { log.Warnw("discard invalid tx on prepare proposal", "err", err, - "code", resp.Code, - "data", string(resp.Data), - "info", resp.Info, - "log", resp.Log) + "hash", fmt.Sprintf("%x", txInfo.DecodedTx.TxID), + "data", func() string { + if resp != nil { + return string(resp.Data) + } + return "" + }()) continue } - validTxs = append(validTxs, tx) + validTxs = append(validTxs, txInfo.Data) } - + // Rollback the state to discard the changes made by CheckTx + app.State.Rollback() return &abcitypes.ResponsePrepareProposal{ Txs: validTxs, }, nil @@ -506,8 +582,42 @@ func (app *BaseApplication) PrepareProposal(ctx context.Context, // Application SHOULD accept a prepared proposal passed via ProcessProposal, even if a part of the proposal // is invalid (e.g., an invalid transaction); the Application can ignore the invalid part of the prepared // proposal at block execution time. The logic in ProcessProposal MUST be deterministic. -func (*BaseApplication) ProcessProposal(_ context.Context, - _ *abcitypes.RequestProcessProposal) (*abcitypes.ResponseProcessProposal, error) { +func (app *BaseApplication) ProcessProposal(_ context.Context, + req *abcitypes.RequestProcessProposal) (*abcitypes.ResponseProcessProposal, error) { + app.prepareProposalLock.Lock() + defer app.prepareProposalLock.Unlock() + valid := true + for _, tx := range req.Txs { + vtx := new(vochaintx.Tx) + if err := vtx.Unmarshal(tx, app.ChainID()); err != nil { + // invalid transaction + log.Warnw("could not unmarshal transaction", "err", err) + valid = false + break + } + // Check the validity of the transaction using forCommit true + resp, err := app.TransactionHandler.CheckTx(vtx, true) + if err != nil { + log.Warnw("discard invalid tx on process proposal", + "err", err, + "data", func() string { + if resp != nil { + return string(resp.Data) + } + return "" + }()) + valid = false + break + } + } + // Rollback the state to discard the changes made by CheckTx + app.State.Rollback() + + if !valid { + return &abcitypes.ResponseProcessProposal{ + Status: abcitypes.ResponseProcessProposal_REJECT, + }, nil + } return &abcitypes.ResponseProcessProposal{ Status: abcitypes.ResponseProcessProposal_ACCEPT, }, nil diff --git a/vochain/proposal_test.go b/vochain/proposal_test.go new file mode 100644 index 000000000..370c265d4 --- /dev/null +++ b/vochain/proposal_test.go @@ -0,0 +1,103 @@ +package vochain + +import ( + "context" + "encoding/hex" + "testing" + + abcitypes "github.com/cometbft/cometbft/abci/types" + "github.com/frankban/quicktest" + "go.vocdoni.io/dvote/crypto/ethereum" + vstate "go.vocdoni.io/dvote/vochain/state" + "go.vocdoni.io/dvote/vochain/transaction/vochaintx" + "go.vocdoni.io/proto/build/go/models" + "google.golang.org/protobuf/proto" +) + +// To test if the PrepareProposal method correctly sorts the transactions based on the sender's address and nonce +func TestTransactionsSorted(t *testing.T) { + qt := quicktest.New(t) + app := TestBaseApplication(t) + keys := ethereum.NewSignKeysBatch(50) + txs := [][]byte{} + // create the accounts + for i, key := range keys { + err := app.State.SetAccount(key.Address(), &vstate.Account{ + Account: models.Account{ + Balance: 500, + Nonce: uint32(i), + }, + }) + qt.Assert(err, quicktest.IsNil) + } + + // add first the transactions with nonce+1 + for i, key := range keys { + tx := models.Tx{ + Payload: &models.Tx_SendTokens{SendTokens: &models.SendTokensTx{ + Nonce: uint32(i), + From: key.Address().Bytes(), + To: keys[(i+1)%50].Address().Bytes(), + Value: 1, + }}} + txBytes, err := proto.Marshal(&tx) + qt.Assert(err, quicktest.IsNil) + + signature, err := key.SignVocdoniTx(txBytes, app.chainID) + qt.Assert(err, quicktest.IsNil) + + stx, err := proto.Marshal(&models.SignedTx{ + Tx: txBytes, + Signature: signature, + }) + qt.Assert(err, quicktest.IsNil) + + txs = append(txs, stx) + } + + // add the transactions with current once + for i, key := range keys { + tx := models.Tx{ + Payload: &models.Tx_SendTokens{SendTokens: &models.SendTokensTx{ + Nonce: uint32(i + 1), + From: key.Address().Bytes(), + To: keys[(i+1)%50].Address().Bytes(), + Value: 1, + }}} + txBytes, err := proto.Marshal(&tx) + qt.Assert(err, quicktest.IsNil) + + signature, err := key.SignVocdoniTx(txBytes, app.chainID) + qt.Assert(err, quicktest.IsNil) + + stx, err := proto.Marshal(&models.SignedTx{ + Tx: txBytes, + Signature: signature, + }) + qt.Assert(err, quicktest.IsNil) + + txs = append(txs, stx) + } + + req := &abcitypes.RequestPrepareProposal{ + Txs: txs, + } + + resp, err := app.PrepareProposal(context.Background(), req) + qt.Assert(err, quicktest.IsNil) + + txAddresses := make(map[string]uint32) + for _, tx := range resp.GetTxs() { + vtx := new(vochaintx.Tx) + err := vtx.Unmarshal(tx, app.chainID) + qt.Assert(err, quicktest.IsNil) + txSendTokens := vtx.Tx.GetSendTokens() + nonce, ok := txAddresses[string(txSendTokens.From)] + if ok && nonce >= txSendTokens.Nonce { + qt.Errorf("nonce is not sorted: %d, %d", nonce, txSendTokens.Nonce) + } + txAddresses[string(txSendTokens.From)] = txSendTokens.Nonce + t.Logf("Address: %s Nonce: %d\n", hex.EncodeToString(txSendTokens.From), txSendTokens.Nonce) + } + qt.Assert(len(txs), quicktest.Equals, len(resp.Txs)) +} diff --git a/vochain/transaction/account_tx.go b/vochain/transaction/account_tx.go index 0660126ef..4eb62c28b 100644 --- a/vochain/transaction/account_tx.go +++ b/vochain/transaction/account_tx.go @@ -122,9 +122,6 @@ func (t *TransactionHandler) SetAccountDelegateTxCheck(vtx *vochaintx.Tx) error tx.Txtype != models.TxType_DEL_DELEGATE_FOR_ACCOUNT { return fmt.Errorf("invalid tx type") } - if tx.Nonce == nil { - return fmt.Errorf("invalid nonce") - } if len(tx.Delegates) == 0 { return fmt.Errorf("invalid delegates") } @@ -135,9 +132,6 @@ func (t *TransactionHandler) SetAccountDelegateTxCheck(vtx *vochaintx.Tx) error if err := vstate.CheckDuplicateDelegates(tx.Delegates, txSenderAddress); err != nil { return fmt.Errorf("checkDuplicateDelegates: %w", err) } - if tx.GetNonce() != txSenderAccount.Nonce { - return fmt.Errorf("invalid nonce, expected %d got %d", txSenderAccount.Nonce, tx.Nonce) - } cost, err := t.state.TxBaseCost(tx.Txtype, false) if err != nil { return fmt.Errorf("cannot get tx cost: %w", err) @@ -196,14 +190,6 @@ func (t *TransactionHandler) SetAccountInfoTxCheck(vtx *vochaintx.Tx) error { if txSenderAccount == nil { return vstate.ErrAccountNotExist } - // check txSender nonce - if tx.GetNonce() != txSenderAccount.Nonce { - return fmt.Errorf( - "invalid nonce, expected %d got %d", - txSenderAccount.Nonce, - tx.GetNonce(), - ) - } // get setAccount tx cost costSetAccountInfoURI, err := t.state.TxBaseCost(models.TxType_SET_ACCOUNT_INFO_URI, false) if err != nil { diff --git a/vochain/transaction/election_tx.go b/vochain/transaction/election_tx.go index 7f85f30e6..7e8f4ac37 100644 --- a/vochain/transaction/election_tx.go +++ b/vochain/transaction/election_tx.go @@ -94,16 +94,11 @@ func (t *TransactionHandler) NewProcessTxCheck(vtx *vochaintx.Tx) (*models.Proce if acc.Balance < cost { return nil, ethereum.Address{}, fmt.Errorf("%w: required %d, got %d", vstate.ErrNotEnoughBalance, cost, acc.Balance) } - if acc.Nonce != tx.Nonce { - return nil, ethereum.Address{}, fmt.Errorf("%w: expected %d, got %d", vstate.ErrAccountNonceInvalid, acc.Nonce, tx.Nonce) - } // if organization ID is not set, use the sender address if tx.Process.EntityId == nil { tx.Process.EntityId = addr.Bytes() - } else if !bytes.Equal(tx.Process.EntityId, addr.Bytes()) { // check if process entityID matches tx sender - // check for a delegate entityAddress := ethereum.AddrFromBytes(tx.Process.EntityId) entityAccount, err := t.state.GetAccount(entityAddress, false) @@ -163,9 +158,6 @@ func (t *TransactionHandler) SetProcessTxCheck(vtx *vochaintx.Tx) (ethereum.Addr if acc.Balance < cost { return ethereum.Address{}, vstate.ErrNotEnoughBalance } - if acc.Nonce != tx.Nonce { - return ethereum.Address{}, vstate.ErrAccountNonceInvalid - } // get process process, err := t.state.Process(tx.ProcessId, false) if err != nil { diff --git a/vochain/transaction/nonce.go b/vochain/transaction/nonce.go new file mode 100644 index 000000000..c0307b5a3 --- /dev/null +++ b/vochain/transaction/nonce.go @@ -0,0 +1,111 @@ +package transaction + +import ( + "fmt" + + "github.com/ethereum/go-ethereum/common" + "go.vocdoni.io/dvote/crypto/ethereum" + "go.vocdoni.io/dvote/vochain/transaction/vochaintx" + "go.vocdoni.io/proto/build/go/models" +) + +// ExtractNonceAndSender extracts the nonce and sender address from a given Vochain transaction. +// The function uses the signature of the transaction to derive the sender's public key and subsequently +// the Ethereum address. The nonce is extracted based on the specific payload type of the transaction. +// If the transaction does not contain signature or nonce, it returns the default values (nil and 0). +func (t *TransactionHandler) ExtractNonceAndSender(vtx *vochaintx.Tx) (*common.Address, uint32, error) { + type txWithNonce interface { + GetNonce() uint32 + } + var txNonce txWithNonce + var err error + switch payload := vtx.Tx.Payload.(type) { + case *models.Tx_NewProcess: + ptx := payload.NewProcess + if ptx == nil { + err = fmt.Errorf("new process payload is nil") + break + } + txNonce = txWithNonce(ptx) + case *models.Tx_SetProcess: + ptx := payload.SetProcess + if ptx == nil { + err = fmt.Errorf("set process payload is nil") + break + } + txNonce = txWithNonce(ptx) + case *models.Tx_SendTokens: + ptx := payload.SendTokens + if ptx == nil { + err = fmt.Errorf("send tokens payload is nil") + break + } + txNonce = txWithNonce(ptx) + case *models.Tx_SetAccount: + ptx := payload.SetAccount + if ptx == nil { + err = fmt.Errorf("set account payload is nil") + break + } + if ptx.Txtype == models.TxType_CREATE_ACCOUNT { + // create account tx is a special case where the nonce is not relevant + return nil, 0, nil + } + txNonce = txWithNonce(ptx) + case *models.Tx_CollectFaucet: + ptx := payload.CollectFaucet + if ptx == nil { + err = fmt.Errorf("collect faucet payload is nil") + break + } + txNonce = txWithNonce(ptx) + case *models.Tx_Vote, *models.Tx_Admin, *models.Tx_MintTokens, *models.Tx_SetKeykeeper, + *models.Tx_SetTransactionCosts, *models.Tx_DelSIK, *models.Tx_RegisterKey, *models.Tx_SetSIK: + // these tx does not have incremental nonce + return nil, 0, nil + default: + // force panic to detect new tx types + panic(fmt.Sprintf("unknown payload type on extract nonce: %T", payload)) + } + + if err != nil { + return nil, 0, err + } + pubKey, err := ethereum.PubKeyFromSignature(vtx.SignedBody, vtx.Signature) + if err != nil { + return nil, 0, fmt.Errorf("cannot extract public key from vtx.Signature: %w", err) + } + addr, err := ethereum.AddrFromPublicKey(pubKey) + if err != nil { + return nil, 0, fmt.Errorf("cannot extract address from public key: %w", err) + } + return &addr, txNonce.GetNonce(), nil +} + +// checkAccountNonce checks if the nonce of the given transaction matches the nonce of the sender account. +// If the transactions does not require a nonce, it returns nil. +// The check is performed against the current (not committed) state. +func (t *TransactionHandler) checkAccountNonce(vtx *vochaintx.Tx) error { + addr, nonce, err := t.ExtractNonceAndSender(vtx) + if err != nil { + return err + } + if addr == nil && nonce == 0 { + // no nonce required + return nil + } + if addr == nil { + return fmt.Errorf("could not check nonce, address is nil") + } + account, err := t.state.GetAccount(*addr, false) + if err != nil { + return fmt.Errorf("could not check nonce, error getting account: %w", err) + } + if account == nil { + return fmt.Errorf("could not check nonce, account does not exist") + } + if account.Nonce != nonce { + return fmt.Errorf("nonce mismatch, expected %d, got %d", account.Nonce, nonce) + } + return nil +} diff --git a/vochain/transaction/tokens_tx.go b/vochain/transaction/tokens_tx.go index 8f78d440e..445fefd4a 100644 --- a/vochain/transaction/tokens_tx.go +++ b/vochain/transaction/tokens_tx.go @@ -27,10 +27,6 @@ func (t *TransactionHandler) SetTransactionCostsTxCheck(vtx *vochaintx.Tx) (uint if err != nil { return 0, err } - // check nonce - if tx.Nonce != treasurer.Nonce { - return 0, fmt.Errorf("invalid nonce %d, expected: %d", tx.Nonce, treasurer.Nonce) - } // check valid tx type if _, ok := vstate.TxTypeCostToStateKey[tx.Txtype]; !ok { return 0, fmt.Errorf("tx type not supported") @@ -86,9 +82,6 @@ func (t *TransactionHandler) MintTokensTxCheck(vtx *vochaintx.Tx) error { txSenderAddress.String(), ) } - if tx.Nonce != treasurer.Nonce { - return fmt.Errorf("invalid nonce %d, expected: %d", tx.Nonce, treasurer.Nonce) - } toAddr := common.BytesToAddress(tx.To) toAcc, err := t.state.GetAccount(toAddr, false) if err != nil { @@ -148,9 +141,6 @@ func (t *TransactionHandler) SendTokensTxCheck(vtx *vochaintx.Tx) error { if acc == nil { return vstate.ErrAccountNotExist } - if tx.Nonce != acc.Nonce { - return fmt.Errorf("invalid nonce, expected %d got %d", acc.Nonce, tx.Nonce) - } cost, err := t.state.TxBaseCost(models.TxType_SEND_TOKENS, false) if err != nil { return err diff --git a/vochain/transaction/transaction.go b/vochain/transaction/transaction.go index 795a6f8cb..0f3261ec9 100644 --- a/vochain/transaction/transaction.go +++ b/vochain/transaction/transaction.go @@ -23,7 +23,7 @@ var ( ErrInvalidURILength = fmt.Errorf("invalid URI length") // ErrorAlreadyExistInCache is returned if the transaction has been already processed // and stored in the vote cache. - ErrorAlreadyExistInCache = fmt.Errorf("vote already exist in cache") + ErrorAlreadyExistInCache = fmt.Errorf("transaction already exist in cache") ) // TransactionResponse is the response of a transaction check. @@ -77,6 +77,11 @@ func (t *TransactionHandler) CheckTx(vtx *vochaintx.Tx, forCommit bool) (*Transa response := &TransactionResponse{ TxHash: vtx.TxID[:], } + if forCommit { + if err := t.checkAccountNonce(vtx); err != nil { + return nil, fmt.Errorf("checkAccountNonce: %w", err) + } + } switch vtx.Tx.Payload.(type) { case *models.Tx_Vote: v, err := t.VoteTxCheck(vtx, forCommit) diff --git a/vochain/transaction/vochaintx/vochaintx.go b/vochain/transaction/vochaintx/vochaintx.go index 139eeaedd..8045214cb 100644 --- a/vochain/transaction/vochaintx/vochaintx.go +++ b/vochain/transaction/vochaintx/vochaintx.go @@ -20,9 +20,12 @@ type Tx struct { TxModelType string } -// Unmarshal unarshal the content of a bytes serialized transaction. -// Returns the transaction struct, the original bytes and the signature -// of those bytes. +// Unmarshal decodes the content of a serialized transaction into the Tx struct. +// +// The function determines the type of the transaction using Protocol Buffers +// reflection and sets it to the TxModelType field. +// Extracts the signature. Prepares the signed body (ready to be checked) and +// computes the transaction ID (a hash of the data). func (tx *Tx) Unmarshal(content []byte, chainID string) error { stx := new(models.SignedTx) if err := proto.Unmarshal(content, stx); err != nil { From cff28c00e6c153a9265dc33f9d5f820fa402bdfe Mon Sep 17 00:00:00 2001 From: p4u Date: Wed, 27 Sep 2023 11:06:39 +0200 Subject: [PATCH 02/16] vochain: update sik roots only once per block Signed-off-by: p4u --- cmd/end2endtest/account.go | 5 ----- cmd/end2endtest/helpers.go | 4 ++++ vochain/app.go | 6 +----- vochain/state/sik.go | 40 ++++++++++++++++++++++---------------- vochain/state/state.go | 8 ++++++-- 5 files changed, 34 insertions(+), 29 deletions(-) diff --git a/cmd/end2endtest/account.go b/cmd/end2endtest/account.go index 1e74ac415..ebc10084f 100644 --- a/cmd/end2endtest/account.go +++ b/cmd/end2endtest/account.go @@ -188,11 +188,6 @@ func testSendTokens(api *apiclient.HTTPclient, aliceKeys, bobKeys *ethereum.Sign amountAtoB := (aliceAcc.Balance) / 4 amountBtoA := (bobAcc.Balance) / 3 - _, err = alice.TransferWithNonce(state.BurnAddress, 2, aliceAcc.Nonce+10) - if err != nil { - log.Fatalf("cannot burn tokens: %v", err) - } - // send a couple of token txs to increase the nonce, without waiting for them to be mined // this tests that the mempool transactions are properly ordered. wg := sync.WaitGroup{} diff --git a/cmd/end2endtest/helpers.go b/cmd/end2endtest/helpers.go index 507e9ff51..da6b17c9f 100644 --- a/cmd/end2endtest/helpers.go +++ b/cmd/end2endtest/helpers.go @@ -186,6 +186,10 @@ func (t *e2eElection) generateProofs(csp *ethereum.SignKeys, voterAccts []*ether wg sync.WaitGroup vcount int32 ) + // Wait for the next block to assure the SIK root is updated + if err := t.api.WaitUntilNextBlock(); err != nil { + return err + } errorChan := make(chan error) t.voters = new(sync.Map) addNaccounts := func(accounts []*ethereum.SignKeys) { diff --git a/vochain/app.go b/vochain/app.go index e70fab109..ebfb4541b 100644 --- a/vochain/app.go +++ b/vochain/app.go @@ -18,7 +18,6 @@ import ( tmcli "github.com/cometbft/cometbft/rpc/client/local" ctypes "github.com/cometbft/cometbft/rpc/core/types" tmtypes "github.com/cometbft/cometbft/types" - "github.com/ethereum/go-ethereum/common" ethcommon "github.com/ethereum/go-ethereum/common" lru "github.com/hashicorp/golang-lru/v2" "go.vocdoni.io/dvote/crypto/zk/circuit" @@ -414,9 +413,6 @@ func (app *BaseApplication) beginBlock(t time.Time, height uint32) { app.startBlockTimestamp.Store(t.Unix()) app.State.SetHeight(height) go app.State.CachePurge(height) - if err := app.State.FetchValidSIKRoots(); err != nil { - log.Errorw(err, "error fetching valid SIK roots") - } app.State.OnBeginBlock(vstate.BeginBlock{ Height: int64(height), Time: t, @@ -504,7 +500,7 @@ func (app *BaseApplication) PrepareProposal(ctx context.Context, defer app.prepareProposalLock.Unlock() type txInfo struct { Data []byte - Addr *common.Address + Addr *ethcommon.Address Nonce uint32 DecodedTx *vochaintx.Tx } diff --git a/vochain/state/sik.go b/vochain/state/sik.go index 20bb091c5..194f6996d 100644 --- a/vochain/state/sik.go +++ b/vochain/state/sik.go @@ -82,7 +82,7 @@ func (v *State) SetAddressSIK(address common.Address, newSIK SIK) error { if err != nil { return fmt.Errorf("%w: %w", ErrSIKSet, err) } - return v.UpdateSIKRoots() + return nil } if err != nil { return fmt.Errorf("%w: %w", ErrSIKGet, err) @@ -101,7 +101,7 @@ func (v *State) SetAddressSIK(address common.Address, newSIK SIK) error { if err != nil { return fmt.Errorf("%w: %w", ErrSIKSet, err) } - return v.UpdateSIKRoots() + return nil } // InvalidateSIK function removes logically the registered SIK for the address @@ -126,7 +126,7 @@ func (v *State) InvalidateSIK(address common.Address) error { if err != nil { return fmt.Errorf("%w: %w", ErrSIKDelete, err) } - return v.UpdateSIKRoots() + return nil } // ValidSIKRoots method returns the current valid SIK roots that are cached in @@ -181,8 +181,28 @@ func (v *State) ExpiredSIKRoot(candidateRoot []byte) bool { func (v *State) UpdateSIKRoots() error { // instance the SIK's key-value DB and set the current block to the current // network height. + v.mtxValidSIKRoots.Lock() + defer v.mtxValidSIKRoots.Unlock() sikNoStateDB := v.NoState(false) currentBlock := v.CurrentHeight() + + // get sik roots key-value database associated to the siks tree + siksTree, err := v.tx.DeepSubTree(StateTreeCfg(TreeSIK)) + if err != nil { + return fmt.Errorf("%w: %w", ErrSIKSubTree, err) + } + // get new sik tree root hash + newSikRoot, err := siksTree.Root() + if err != nil { + return fmt.Errorf("%w: %w", ErrSIKRootsGet, err) + } + // check if the new sik root is already in the list of valid roots, if so return + for _, sikRoot := range v.validSIKRoots { + if bytes.Equal(sikRoot, newSikRoot) { + return nil + } + } + // purge the oldest sikRoots if the hysteresis is reached if currentBlock > SIKROOT_HYSTERESIS_BLOCKS { // calculate the current minimun block to purge useless sik roots minBlock := currentBlock - SIKROOT_HYSTERESIS_BLOCKS @@ -219,18 +239,6 @@ func (v *State) UpdateSIKRoots() error { "blockNumber", binary.LittleEndian.Uint32(blockToDelete)) } } - // get sik roots key-value database associated to the siks tree - v.tx.Lock() - defer v.tx.Unlock() - siksTree, err := v.tx.DeepSubTree(StateTreeCfg(TreeSIK)) - if err != nil { - return fmt.Errorf("%w: %w", ErrSIKSubTree, err) - } - // get new sik tree root hash - newSikRoot, err := siksTree.Root() - if err != nil { - return fmt.Errorf("%w: %w", ErrSIKRootsGet, err) - } // encode current blockNumber as key blockKey := make([]byte, 32) binary.LittleEndian.PutUint32(blockKey, currentBlock) @@ -240,9 +248,7 @@ func (v *State) UpdateSIKRoots() error { return fmt.Errorf("%w: %w", ErrSIKRootsSet, err) } // include the new root into the cached list - v.mtxValidSIKRoots.Lock() v.validSIKRoots = append(v.validSIKRoots, newSikRoot) - v.mtxValidSIKRoots.Unlock() log.Debugw("updateSIKRoots (created)", "newSikRoot", hex.EncodeToString(newSikRoot), "blockNumber", currentBlock) diff --git a/vochain/state/state.go b/vochain/state/state.go index 3cb98a019..c70e81889 100644 --- a/vochain/state/state.go +++ b/vochain/state/state.go @@ -144,8 +144,10 @@ func NewState(dbType, dataDir string) (*State, error) { db: s.NoState(true), state: s, } - s.validSIKRoots = [][]byte{} s.mtxValidSIKRoots = &sync.Mutex{} + if err := s.FetchValidSIKRoots(); err != nil { + return nil, fmt.Errorf("cannot update valid SIK roots: %w", err) + } return s, os.MkdirAll(filepath.Join(dataDir, storageDirectory, snapshotsDirectory), 0750) } @@ -425,7 +427,9 @@ func (v *State) Save() ([]byte, error) { return nil, fmt.Errorf("cannot get statedb mainTreeView: %w", err) } v.setMainTreeView(mainTreeView) - + if err := v.UpdateSIKRoots(); err != nil { + return nil, fmt.Errorf("cannot update SIK roots: %w", err) + } return v.store.Hash() } From cdde4e384a3ed4b30532c14c05e23842d649eb99 Mon Sep 17 00:00:00 2001 From: p4u Date: Thu, 28 Sep 2023 17:58:54 +0200 Subject: [PATCH 03/16] fix tests regading the SIK changes Signed-off-by: p4u --- cmd/end2endtest/account.go | 7 ++----- vochain/account_test.go | 4 ---- vochain/app.go | 2 +- vochain/state/sik_test.go | 3 +++ vochain/transaction_zk_test.go | 3 +++ 5 files changed, 9 insertions(+), 10 deletions(-) diff --git a/cmd/end2endtest/account.go b/cmd/end2endtest/account.go index ebc10084f..ff1c61b8e 100644 --- a/cmd/end2endtest/account.go +++ b/cmd/end2endtest/account.go @@ -194,15 +194,12 @@ func testSendTokens(api *apiclient.HTTPclient, aliceKeys, bobKeys *ethereum.Sign wg.Add(1) go func() { // send 1 token to burn address with nonce + 1 (should be mined after the other txs) - _, err = alice.TransferWithNonce(state.BurnAddress, 1, aliceAcc.Nonce+1) - if err != nil { + if _, err = alice.TransferWithNonce(state.BurnAddress, 1, aliceAcc.Nonce+1); err != nil { log.Fatalf("cannot burn tokens: %v", err) } - _, err = bob.TransferWithNonce(state.BurnAddress, 1, bobAcc.Nonce+1) - if err != nil { + if _, err = bob.TransferWithNonce(state.BurnAddress, 1, bobAcc.Nonce+1); err != nil { log.Fatalf("cannot burn tokens: %v", err) } - wg.Done() }() diff --git a/vochain/account_test.go b/vochain/account_test.go index 5146966a7..2a8044214 100644 --- a/vochain/account_test.go +++ b/vochain/account_test.go @@ -626,10 +626,6 @@ func TestMintTokensTx(t *testing.T) { if err := testMintTokensTx(t, ¬Treasurer, app, toAccAddr, 100, 1); err == nil { t.Fatal(err) } - // should fail minting if invalid nonce - if err := testMintTokensTx(t, &signer, app, toAccAddr, 100, rand.Uint32()); err == nil { - t.Fatal(err) - } // get account toAcc, err := app.State.GetAccount(toAccAddr, false) diff --git a/vochain/app.go b/vochain/app.go index ebfb4541b..cf858e2c6 100644 --- a/vochain/app.go +++ b/vochain/app.go @@ -81,7 +81,7 @@ type BaseApplication struct { dataDir string genesisInfo *tmtypes.GenesisDoc - // prepareProposalLock is used to avoid concurrent calls between PrepareProposal and FinalizeBlock + // prepareProposalLock is used to avoid concurrent calls between Prepare/Process Proposal and FinalizeBlock prepareProposalLock sync.Mutex // testMockBlockStore is used for testing purposes only diff --git a/vochain/state/sik_test.go b/vochain/state/sik_test.go index 6cff2c78b..012a64d07 100644 --- a/vochain/state/sik_test.go +++ b/vochain/state/sik_test.go @@ -98,6 +98,7 @@ func Test_sikRoots(t *testing.T) { sik1, _ := hex.DecodeString("3a7806f4e0b5bda625d465abf5639ba42ac9b91bafea3b800a4a") s.SetHeight(1) c.Assert(s.SetAddressSIK(address1, sik1), qt.IsNil) + c.Assert(s.UpdateSIKRoots(), qt.IsNil) // check the results c.Assert(s.FetchValidSIKRoots(), qt.IsNil) validSIKs := s.ValidSIKRoots() @@ -113,6 +114,7 @@ func Test_sikRoots(t *testing.T) { sik2, _ := hex.DecodeString("5fb53c1f9b53fba0296f4e8306802d44235c1a11becc4e6853d0") s.SetHeight(33) c.Assert(s.SetAddressSIK(address2, sik2), qt.IsNil) + c.Assert(s.UpdateSIKRoots(), qt.IsNil) // check the results c.Assert(s.FetchValidSIKRoots(), qt.IsNil) validSIKs = s.ValidSIKRoots() @@ -127,6 +129,7 @@ func Test_sikRoots(t *testing.T) { sik3, _ := hex.DecodeString("7ccbc0da9e8d7e469ba60cd898a5b881c99a960c1e69990a3196") s.SetHeight(66) c.Assert(s.SetAddressSIK(address3, sik3), qt.IsNil) + c.Assert(s.UpdateSIKRoots(), qt.IsNil) // check the results c.Assert(s.FetchValidSIKRoots(), qt.IsNil) validSIKs = s.ValidSIKRoots() diff --git a/vochain/transaction_zk_test.go b/vochain/transaction_zk_test.go index bdaeb4e56..74b70bfb4 100644 --- a/vochain/transaction_zk_test.go +++ b/vochain/transaction_zk_test.go @@ -66,6 +66,9 @@ func TestVoteCheckZkSNARK(t *testing.T) { c.Assert(err, qt.IsNil) _, err = app.State.Process(electionId, false) c.Assert(err, qt.IsNil) + // advance the app block so the SIK tree is updated + app.AdvanceTestBlock() + // generate circuit inputs and the zk proof sikRoot, err := app.State.SIKRoot() c.Assert(err, qt.IsNil) From 641ca99557b332757c1771c5b08a4fec5f4a6ca1 Mon Sep 17 00:00:00 2001 From: p4u Date: Fri, 29 Sep 2023 01:05:10 +0200 Subject: [PATCH 04/16] add missing RegisterSIK tx on extract nonce Signed-off-by: p4u --- vochain/transaction/nonce.go | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/vochain/transaction/nonce.go b/vochain/transaction/nonce.go index c0307b5a3..664d4f924 100644 --- a/vochain/transaction/nonce.go +++ b/vochain/transaction/nonce.go @@ -60,7 +60,8 @@ func (t *TransactionHandler) ExtractNonceAndSender(vtx *vochaintx.Tx) (*common.A } txNonce = txWithNonce(ptx) case *models.Tx_Vote, *models.Tx_Admin, *models.Tx_MintTokens, *models.Tx_SetKeykeeper, - *models.Tx_SetTransactionCosts, *models.Tx_DelSIK, *models.Tx_RegisterKey, *models.Tx_SetSIK: + *models.Tx_SetTransactionCosts, *models.Tx_DelSIK, *models.Tx_RegisterKey, *models.Tx_SetSIK, + *models.Tx_RegisterSIK: // these tx does not have incremental nonce return nil, 0, nil default: From 51d2efceba2c69532b04a0cf2387cec69b5792e0 Mon Sep 17 00:00:00 2001 From: p4u Date: Fri, 29 Sep 2023 17:58:05 +0200 Subject: [PATCH 05/16] vochain: set isSync to true while bootstraping the blockchain Signed-off-by: p4u --- vochain/app.go | 1 + 1 file changed, 1 insertion(+) diff --git a/vochain/app.go b/vochain/app.go index cf858e2c6..e50133fc8 100644 --- a/vochain/app.go +++ b/vochain/app.go @@ -143,6 +143,7 @@ func NewBaseApplication(dbType, dbpath string) (*BaseApplication, error) { // We use this method to initialize some state variables. func (app *BaseApplication) Info(_ context.Context, req *abcitypes.RequestInfo) (*abcitypes.ResponseInfo, error) { + app.isSynchronizing.Store(true) lastHeight, err := app.State.LastHeight() if err != nil { return nil, fmt.Errorf("cannot get State.LastHeight: %w", err) From 57d00b182123354bacb4c3a782558308ed903971 Mon Sep 17 00:00:00 2001 From: p4u Date: Fri, 29 Sep 2023 09:59:43 +0200 Subject: [PATCH 06/16] log: if output is a json file, format propperly --- log/log.go | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) diff --git a/log/log.go b/log/log.go index 0bdf13b42..d86e23b22 100644 --- a/log/log.go +++ b/log/log.go @@ -7,6 +7,7 @@ import ( "os" "path" "runtime/debug" + "strings" "time" "github.com/rs/zerolog" @@ -89,11 +90,9 @@ func (*invalidCharChecker) Write(p []byte) (int, error) { return len(p), nil } -// Init initializes the logger. Output can be either "stdout/stderr/". -// Log level can be "debug/info/warn/error". -// errorOutput is an optional filename which only receives Warning and Error messages. func Init(level, output string, errorOutput io.Writer) { var out io.Writer + outputs := []io.Writer{} switch output { case "stdout": out = os.Stdout @@ -107,12 +106,16 @@ func Init(level, output string, errorOutput io.Writer) { panic(fmt.Sprintf("cannot create log output: %v", err)) } out = f + if strings.HasSuffix(output, ".json") { + outputs = append(outputs, f) + out = os.Stdout + } } out = zerolog.ConsoleWriter{ Out: out, TimeFormat: time.RFC3339Nano, } - outputs := []io.Writer{out} + outputs = append(outputs, out) if errorOutput != nil { outputs = append(outputs, &errorLevelWriter{zerolog.ConsoleWriter{ From 8b9b9e95b2bbe10d55f2480995acd0e10b1bfff0 Mon Sep 17 00:00:00 2001 From: p4u Date: Tue, 3 Oct 2023 12:11:19 +0200 Subject: [PATCH 07/16] some fixes on consensus stability 1. UpdateSIKRoots must be executed on the current State transaction, so when the commit is done for te block, it is also stored. 2. Prepare/Process proposal check (with panic) if there are pending transactions to commit. 3. NoState transaction is correctly discarted now when Rollback Signed-off-by: p4u --- vochain/app.go | 10 +++++ vochain/hysteresis_test.go | 78 +++++++++++++++++++++----------------- vochain/state/state.go | 9 +++-- 3 files changed, 60 insertions(+), 37 deletions(-) diff --git a/vochain/app.go b/vochain/app.go index e50133fc8..4673fc839 100644 --- a/vochain/app.go +++ b/vochain/app.go @@ -506,6 +506,11 @@ func (app *BaseApplication) PrepareProposal(ctx context.Context, DecodedTx *vochaintx.Tx } + // ensure the pending state is clean + if app.State.TxCounter() > 0 { + panic("found existing pending transactions on prepare proposal") + } + validTxInfos := []txInfo{} for _, tx := range req.GetTxs() { vtx := new(vochaintx.Tx) @@ -583,6 +588,11 @@ func (app *BaseApplication) ProcessProposal(_ context.Context, req *abcitypes.RequestProcessProposal) (*abcitypes.ResponseProcessProposal, error) { app.prepareProposalLock.Lock() defer app.prepareProposalLock.Unlock() + // ensure the pending state is clean + if app.State.TxCounter() > 0 { + panic("found existing pending transactions on process proposal") + } + valid := true for _, tx := range req.Txs { vtx := new(vochaintx.Tx) diff --git a/vochain/hysteresis_test.go b/vochain/hysteresis_test.go index 5ac6c7227..2b476a113 100644 --- a/vochain/hysteresis_test.go +++ b/vochain/hysteresis_test.go @@ -3,6 +3,7 @@ package vochain import ( "encoding/json" "math/big" + "sync" "testing" qt "github.com/frankban/quicktest" @@ -28,7 +29,7 @@ func TestHysteresis(t *testing.T) { // initial accounts testWeight := big.NewInt(10) - accounts, censusRoot, proofs := testCreateKeysAndBuildWeightedZkCensus(t, 10, testWeight) + accounts, censusRoot, proofs := testCreateKeysAndBuildWeightedZkCensus(t, 3, testWeight) // add the test accounts siks to the test app for _, account := range accounts { @@ -64,38 +65,47 @@ func TestHysteresis(t *testing.T) { c.Assert(err, qt.IsNil) sikRoot, err := sikTree.Root() c.Assert(err, qt.IsNil) - for i, account := range accounts { - _, sikProof, err := sikTree.GenProof(account.Address().Bytes()) - c.Assert(err, qt.IsNil) - - sikSiblings, err := zk.ProofToCircomSiblings(sikProof) - c.Assert(err, qt.IsNil) - - censusSiblings, err := zk.ProofToCircomSiblings(proofs[i]) - c.Assert(err, qt.IsNil) - - // get zkproof - inputs, err := circuit.GenerateCircuitInput(circuit.CircuitInputsParameters{ - Account: account, - ElectionId: pid, - CensusRoot: censusRoot, - SIKRoot: sikRoot, - CensusSiblings: censusSiblings, - SIKSiblings: sikSiblings, - AvailableWeight: testWeight, - }) - c.Assert(err, qt.IsNil) - encInputs, err := json.Marshal(inputs) - c.Assert(err, qt.IsNil) - - zkProof, err := prover.Prove(devCircuit.ProvingKey, devCircuit.Wasm, encInputs) - c.Assert(err, qt.IsNil) - - protoZkProof, err := zk.ProverProofToProtobufZKProof(zkProof, nil, nil, nil, nil, nil) - c.Assert(err, qt.IsNil) - - zkProofs = append(zkProofs, protoZkProof) + wg := sync.WaitGroup{} + mtx := sync.Mutex{} + for i := range accounts { + wg.Add(1) + go func() { + i := i + _, sikProof, err := sikTree.GenProof(accounts[i].Address().Bytes()) + c.Assert(err, qt.IsNil) + + sikSiblings, err := zk.ProofToCircomSiblings(sikProof) + c.Assert(err, qt.IsNil) + + censusSiblings, err := zk.ProofToCircomSiblings(proofs[i]) + c.Assert(err, qt.IsNil) + + // get zkproof + inputs, err := circuit.GenerateCircuitInput(circuit.CircuitInputsParameters{ + Account: accounts[i], + ElectionId: pid, + CensusRoot: censusRoot, + SIKRoot: sikRoot, + CensusSiblings: censusSiblings, + SIKSiblings: sikSiblings, + AvailableWeight: testWeight, + }) + c.Assert(err, qt.IsNil) + encInputs, err := json.Marshal(inputs) + c.Assert(err, qt.IsNil) + + zkProof, err := prover.Prove(devCircuit.ProvingKey, devCircuit.Wasm, encInputs) + c.Assert(err, qt.IsNil) + + protoZkProof, err := zk.ProverProofToProtobufZKProof(zkProof, nil, nil, nil, nil, nil) + c.Assert(err, qt.IsNil) + mtx.Lock() + zkProofs = append(zkProofs, protoZkProof) + mtx.Unlock() + wg.Done() + }() } + wg.Wait() validVotes := len(accounts) / 2 for i, account := range accounts[:validVotes] { @@ -131,14 +141,14 @@ func TestHysteresis(t *testing.T) { } for i := 0; i < state.SIKROOT_HYSTERESIS_BLOCKS; i++ { + mockNewSIK() app.AdvanceTestBlock() } - mockNewSIK() for i := 0; i < state.SIKROOT_HYSTERESIS_BLOCKS; i++ { + mockNewSIK() app.AdvanceTestBlock() } - mockNewSIK() for i, account := range accounts[validVotes:] { nullifier, err := account.AccountSIKnullifier(pid, nil) diff --git a/vochain/state/state.go b/vochain/state/state.go index c70e81889..6112ab03e 100644 --- a/vochain/state/state.go +++ b/vochain/state/state.go @@ -407,6 +407,10 @@ func (v *State) Save() ([]byte, error) { // the listeners may need to get the previous (not committed) state. v.tx.Lock() defer v.tx.Unlock() + // Update the SIK merkle-tree roots + if err := v.UpdateSIKRoots(); err != nil { + return nil, fmt.Errorf("cannot update SIK roots: %w", err) + } err := func() error { var err error if err := v.tx.Commit(height); err != nil { @@ -427,9 +431,6 @@ func (v *State) Save() ([]byte, error) { return nil, fmt.Errorf("cannot get statedb mainTreeView: %w", err) } v.setMainTreeView(mainTreeView) - if err := v.UpdateSIKRoots(); err != nil { - return nil, fmt.Errorf("cannot update SIK roots: %w", err) - } return v.store.Hash() } @@ -441,6 +442,7 @@ func (v *State) Rollback() { v.tx.Lock() defer v.tx.Unlock() v.tx.Discard() + v.store.NoStateWriteTx.Discard() var err error if v.tx.TreeTx, err = v.store.BeginTx(); err != nil { log.Errorf("cannot begin statedb tx: %s", err) @@ -453,6 +455,7 @@ func (v *State) Rollback() { func (v *State) Close() error { v.tx.Lock() v.tx.Discard() + v.store.NoStateWriteTx.Discard() v.tx.Unlock() return v.db.Close() From cc9ab8a26603e0255a74b8d8e82f49067d7b7388 Mon Sep 17 00:00:00 2001 From: p4u Date: Tue, 3 Oct 2023 00:27:09 +0200 Subject: [PATCH 08/16] arbo: add a cache for the current Root and coupe of locks more Signed-off-by: p4u --- tree/arbo/tree.go | 25 ++++++++++++++++++++++--- 1 file changed, 22 insertions(+), 3 deletions(-) diff --git a/tree/arbo/tree.go b/tree/arbo/tree.go index bc07c430e..c0a78e83a 100644 --- a/tree/arbo/tree.go +++ b/tree/arbo/tree.go @@ -84,7 +84,7 @@ var ( // Tree defines the struct that implements the MerkleTree functionalities type Tree struct { - sync.Mutex + sync.RWMutex db db.Database maxLevels int @@ -104,6 +104,9 @@ type Tree struct { emptyNode []byte dbg *dbgStats + + currentRoot []byte + currentRootLock sync.RWMutex } // Config defines the configuration for calling NewTree & NewTreeWithTx methods @@ -178,10 +181,18 @@ func (t *Tree) RootWithTx(rTx db.Reader) ([]byte, error) { return t.snapshotRoot, nil } // get db root + t.currentRootLock.RLock() + defer t.currentRootLock.RUnlock() + if len(t.currentRoot) > 0 { + return t.currentRoot, nil + } return rTx.Get(dbKeyRoot) } -func (*Tree) setRoot(wTx db.WriteTx, root []byte) error { +func (t *Tree) setRoot(wTx db.WriteTx, root []byte) error { + t.currentRootLock.Lock() + defer t.currentRootLock.Unlock() + t.currentRoot = root return wTx.Set(dbKeyRoot, root) } @@ -325,7 +336,6 @@ func (t *Tree) Update(k, v []byte) error { func (t *Tree) UpdateWithTx(wTx db.WriteTx, k, v []byte) error { t.Lock() defer t.Unlock() - if !t.editable() { return ErrSnapshotNotEditable } @@ -521,6 +531,9 @@ func (t *Tree) Get(k []byte) ([]byte, []byte, error) { // ErrKeyNotFound, and in the leafK & leafV parameters will be placed the data // found in the tree in the leaf that was on the path going to the input key. func (t *Tree) GetWithTx(rTx db.Reader, k []byte) ([]byte, []byte, error) { + t.RLock() + defer t.RUnlock() + keyPath, err := keyPathFromKey(t.maxLevels, k) if err != nil { return nil, nil, err @@ -602,6 +615,8 @@ func (t *Tree) SetRoot(root []byte) error { // SetRootWithTx sets the root to the given root using the given db.WriteTx func (t *Tree) SetRootWithTx(wTx db.WriteTx, root []byte) error { + t.Lock() + defer t.Unlock() if !t.editable() { return ErrSnapshotNotEditable } @@ -620,6 +635,8 @@ func (t *Tree) SetRootWithTx(wTx db.WriteTx, root []byte) error { // The provided root must be a valid existing intermediate node in the tree. // The list of roots for a level can be obtained using tree.RootsFromLevel(). func (t *Tree) Snapshot(fromRoot []byte) (*Tree, error) { + t.Lock() + defer t.Unlock() // allow to define which root to use if fromRoot == nil { var err error @@ -672,6 +689,8 @@ func (t *Tree) IterateWithTx(rTx db.Reader, fromRoot []byte, f func([]byte, []by return err } } + t.Lock() + defer t.Unlock() return t.iter(rTx, fromRoot, f) } From 8a3be13dcbc5f0aa3122e67867a4159731eebabe Mon Sep 17 00:00:00 2001 From: p4u Date: Tue, 3 Oct 2023 00:30:48 +0200 Subject: [PATCH 09/16] end2endtest: tokentxs better check nonce ordering --- cmd/end2endtest/account.go | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/cmd/end2endtest/account.go b/cmd/end2endtest/account.go index ff1c61b8e..929a46977 100644 --- a/cmd/end2endtest/account.go +++ b/cmd/end2endtest/account.go @@ -193,6 +193,7 @@ func testSendTokens(api *apiclient.HTTPclient, aliceKeys, bobKeys *ethereum.Sign wg := sync.WaitGroup{} wg.Add(1) go func() { + log.Warnf("send transactions with nonce+1, should not be mined before the others") // send 1 token to burn address with nonce + 1 (should be mined after the other txs) if _, err = alice.TransferWithNonce(state.BurnAddress, 1, aliceAcc.Nonce+1); err != nil { log.Fatalf("cannot burn tokens: %v", err) @@ -202,7 +203,8 @@ func testSendTokens(api *apiclient.HTTPclient, aliceKeys, bobKeys *ethereum.Sign } wg.Done() }() - + log.Warnf("waiting 6 seconds to let the burn txs be sent") + time.Sleep(6 * time.Second) var txhasha, txhashb []byte wg.Add(1) go func() { From 66086b6b4147d2fa5fc64af3bde6625018a5dde7 Mon Sep 17 00:00:00 2001 From: p4u Date: Tue, 3 Oct 2023 12:23:01 +0200 Subject: [PATCH 10/16] bump new dev genesis Signed-off-by: p4u --- vochain/genesis/genesis.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/vochain/genesis/genesis.go b/vochain/genesis/genesis.go index c0c018e81..fc59e4a2f 100644 --- a/vochain/genesis/genesis.go +++ b/vochain/genesis/genesis.go @@ -40,8 +40,8 @@ var Genesis = map[string]VochainGenesis{ } var devGenesis = GenesisDoc{ - GenesisTime: time.Date(2023, time.September, 21, 1, 0, 0, 0, time.UTC), - ChainID: "vocdoni-dev-20", + GenesisTime: time.Date(2023, time.October, 3, 1, 0, 0, 0, time.UTC), + ChainID: "vocdoni-dev-21", ConsensusParams: &ConsensusParams{ Block: BlockParams{ MaxBytes: 2097152, From b6e1c01806249f9e1a77da46ca45e26741e384d5 Mon Sep 17 00:00:00 2001 From: p4u Date: Tue, 3 Oct 2023 12:47:57 +0200 Subject: [PATCH 11/16] apply PR suggestions --- vochain/app.go | 4 +-- vochain/transaction/nonce.go | 54 ++++++++++-------------------------- 2 files changed, 17 insertions(+), 41 deletions(-) diff --git a/vochain/app.go b/vochain/app.go index 4673fc839..ddd81cc50 100644 --- a/vochain/app.go +++ b/vochain/app.go @@ -270,9 +270,9 @@ func (app *BaseApplication) CheckTx(_ context.Context, req *abcitypes.RequestCheckTx) (*abcitypes.ResponseCheckTx, error) { txReference := vochaintx.TxKey(req.Tx) // store the initial height of the tx - initialTTLheight, isReferenced := app.txTTLReferences.LoadOrStore(txReference, app.Height()) + initialTTLheight, _ := app.txTTLReferences.LoadOrStore(txReference, app.Height()) // check if the tx is referenced by a previous block and the TTL has expired - if isReferenced && app.Height() > initialTTLheight.(uint32)+transactionBlocksTTL { + if app.Height() > initialTTLheight.(uint32)+transactionBlocksTTL { // remove tx reference and return checkTx error log.Debugw("pruning expired tx from mempool", "height", app.Height(), "hash", fmt.Sprintf("%x", txReference)) app.txTTLReferences.Delete(txReference) diff --git a/vochain/transaction/nonce.go b/vochain/transaction/nonce.go index 664d4f924..69f5ce652 100644 --- a/vochain/transaction/nonce.go +++ b/vochain/transaction/nonce.go @@ -5,6 +5,7 @@ import ( "github.com/ethereum/go-ethereum/common" "go.vocdoni.io/dvote/crypto/ethereum" + "go.vocdoni.io/dvote/log" "go.vocdoni.io/dvote/vochain/transaction/vochaintx" "go.vocdoni.io/proto/build/go/models" ) @@ -14,64 +15,38 @@ import ( // the Ethereum address. The nonce is extracted based on the specific payload type of the transaction. // If the transaction does not contain signature or nonce, it returns the default values (nil and 0). func (t *TransactionHandler) ExtractNonceAndSender(vtx *vochaintx.Tx) (*common.Address, uint32, error) { - type txWithNonce interface { + var ptx interface { GetNonce() uint32 } - var txNonce txWithNonce - var err error + switch payload := vtx.Tx.Payload.(type) { case *models.Tx_NewProcess: - ptx := payload.NewProcess - if ptx == nil { - err = fmt.Errorf("new process payload is nil") - break - } - txNonce = txWithNonce(ptx) + ptx = payload.NewProcess case *models.Tx_SetProcess: - ptx := payload.SetProcess - if ptx == nil { - err = fmt.Errorf("set process payload is nil") - break - } - txNonce = txWithNonce(ptx) + ptx = payload.SetProcess case *models.Tx_SendTokens: - ptx := payload.SendTokens - if ptx == nil { - err = fmt.Errorf("send tokens payload is nil") - break - } - txNonce = txWithNonce(ptx) + ptx = payload.SendTokens case *models.Tx_SetAccount: - ptx := payload.SetAccount - if ptx == nil { - err = fmt.Errorf("set account payload is nil") - break - } - if ptx.Txtype == models.TxType_CREATE_ACCOUNT { + if payload.SetAccount.Txtype == models.TxType_CREATE_ACCOUNT { // create account tx is a special case where the nonce is not relevant return nil, 0, nil } - txNonce = txWithNonce(ptx) + ptx = payload.SetAccount case *models.Tx_CollectFaucet: - ptx := payload.CollectFaucet - if ptx == nil { - err = fmt.Errorf("collect faucet payload is nil") - break - } - txNonce = txWithNonce(ptx) + ptx = payload.CollectFaucet case *models.Tx_Vote, *models.Tx_Admin, *models.Tx_MintTokens, *models.Tx_SetKeykeeper, *models.Tx_SetTransactionCosts, *models.Tx_DelSIK, *models.Tx_RegisterKey, *models.Tx_SetSIK, *models.Tx_RegisterSIK: // these tx does not have incremental nonce return nil, 0, nil default: - // force panic to detect new tx types - panic(fmt.Sprintf("unknown payload type on extract nonce: %T", payload)) + log.Errorf("unknown payload type on extract nonce: %T", payload) } - if err != nil { - return nil, 0, err + if ptx == nil { + return nil, 0, fmt.Errorf("payload is nil") } + pubKey, err := ethereum.PubKeyFromSignature(vtx.SignedBody, vtx.Signature) if err != nil { return nil, 0, fmt.Errorf("cannot extract public key from vtx.Signature: %w", err) @@ -80,7 +55,8 @@ func (t *TransactionHandler) ExtractNonceAndSender(vtx *vochaintx.Tx) (*common.A if err != nil { return nil, 0, fmt.Errorf("cannot extract address from public key: %w", err) } - return &addr, txNonce.GetNonce(), nil + + return &addr, ptx.GetNonce(), nil } // checkAccountNonce checks if the nonce of the given transaction matches the nonce of the sender account. From eb64225b10ea2e58c51ad177615d82fcb0e83edb Mon Sep 17 00:00:00 2001 From: p4u Date: Tue, 3 Oct 2023 12:51:50 +0200 Subject: [PATCH 12/16] arbo: remove the root cache --- tree/arbo/tree.go | 11 ----------- 1 file changed, 11 deletions(-) diff --git a/tree/arbo/tree.go b/tree/arbo/tree.go index c0a78e83a..2e2050898 100644 --- a/tree/arbo/tree.go +++ b/tree/arbo/tree.go @@ -104,9 +104,6 @@ type Tree struct { emptyNode []byte dbg *dbgStats - - currentRoot []byte - currentRootLock sync.RWMutex } // Config defines the configuration for calling NewTree & NewTreeWithTx methods @@ -181,18 +178,10 @@ func (t *Tree) RootWithTx(rTx db.Reader) ([]byte, error) { return t.snapshotRoot, nil } // get db root - t.currentRootLock.RLock() - defer t.currentRootLock.RUnlock() - if len(t.currentRoot) > 0 { - return t.currentRoot, nil - } return rTx.Get(dbKeyRoot) } func (t *Tree) setRoot(wTx db.WriteTx, root []byte) error { - t.currentRootLock.Lock() - defer t.currentRootLock.Unlock() - t.currentRoot = root return wTx.Set(dbKeyRoot, root) } From 9ddf0b07e7b74b5cb3345507a902f2c3182d6489 Mon Sep 17 00:00:00 2001 From: p4u Date: Tue, 3 Oct 2023 13:06:54 +0200 Subject: [PATCH 13/16] make vet happy --- vochain/hysteresis_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/vochain/hysteresis_test.go b/vochain/hysteresis_test.go index 2b476a113..64e5b9cb3 100644 --- a/vochain/hysteresis_test.go +++ b/vochain/hysteresis_test.go @@ -69,8 +69,8 @@ func TestHysteresis(t *testing.T) { mtx := sync.Mutex{} for i := range accounts { wg.Add(1) + i := i go func() { - i := i _, sikProof, err := sikTree.GenProof(accounts[i].Address().Bytes()) c.Assert(err, qt.IsNil) From d97288ce3fdec9c80f83e0d92409fb542b999da2 Mon Sep 17 00:00:00 2001 From: p4u Date: Tue, 3 Oct 2023 15:17:58 +0200 Subject: [PATCH 14/16] vochain: reset txCounter on commit Signed-off-by: p4u --- vochain/app.go | 6 +++--- vochain/state/state.go | 1 + 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/vochain/app.go b/vochain/app.go index ddd81cc50..712c2bf2b 100644 --- a/vochain/app.go +++ b/vochain/app.go @@ -345,6 +345,9 @@ func (app *BaseApplication) FinalizeBlock(_ context.Context, // Commit saves the current vochain state and returns a commit hash func (app *BaseApplication) Commit(_ context.Context, _ *abcitypes.RequestCommit) (*abcitypes.ResponseCommit, error) { + if app.State.TxCounter() > 0 { + log.Infow("commit block", "height", app.Height(), "txs", app.State.TxCounter()) + } // save state _, err := app.State.Save() if err != nil { @@ -360,9 +363,6 @@ func (app *BaseApplication) Commit(_ context.Context, _ *abcitypes.RequestCommit log.Infof("snapshot created successfully, took %s", time.Since(startTime)) log.Debugf("%+v", app.State.ListSnapshots()) } - if app.State.TxCounter() > 0 { - log.Infow("commit block", "height", app.Height(), "txs", app.State.TxCounter()) - } return &abcitypes.ResponseCommit{ RetainHeight: 0, // When snapshot sync enabled, we can start to remove old blocks }, nil diff --git a/vochain/state/state.go b/vochain/state/state.go index 6112ab03e..a847a068e 100644 --- a/vochain/state/state.go +++ b/vochain/state/state.go @@ -419,6 +419,7 @@ func (v *State) Save() ([]byte, error) { if v.tx.TreeTx, err = v.store.BeginTx(); err != nil { return fmt.Errorf("cannot begin statedb tx: %w", err) } + v.txCounter.Store(0) return nil }() if err != nil { From bc8636b71b45b11168a4beed4af606a68fe01b08 Mon Sep 17 00:00:00 2001 From: p4u Date: Tue, 3 Oct 2023 15:23:20 +0200 Subject: [PATCH 15/16] arbo: move some locks to read-locks Signed-off-by: p4u --- tree/arbo/tree.go | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/tree/arbo/tree.go b/tree/arbo/tree.go index 2e2050898..18cac5c26 100644 --- a/tree/arbo/tree.go +++ b/tree/arbo/tree.go @@ -678,8 +678,8 @@ func (t *Tree) IterateWithTx(rTx db.Reader, fromRoot []byte, f func([]byte, []by return err } } - t.Lock() - defer t.Unlock() + t.RLock() + defer t.RUnlock() return t.iter(rTx, fromRoot, f) } @@ -695,6 +695,8 @@ func (t *Tree) IterateWithStop(fromRoot []byte, f func(int, []byte, []byte) bool return err } } + t.RLock() + defer t.RUnlock() return t.iterWithStop(t.db, fromRoot, 0, f) } @@ -709,6 +711,8 @@ func (t *Tree) IterateWithStopWithTx(rTx db.Reader, fromRoot []byte, f func(int, return err } } + t.RLock() + defer t.RUnlock() return t.iterWithStop(rTx, fromRoot, 0, f) } From 596f2eb559daf110fe031d47652e357d6dbe870f Mon Sep 17 00:00:00 2001 From: "Lucas\\ Menendez" Date: Thu, 5 Oct 2023 13:20:25 +0200 Subject: [PATCH 16/16] increasing the nonce for SetSIK and DelSIK tx, pending to update protobuf models --- vochain/transaction/nonce.go | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/vochain/transaction/nonce.go b/vochain/transaction/nonce.go index 69f5ce652..6e455c1b3 100644 --- a/vochain/transaction/nonce.go +++ b/vochain/transaction/nonce.go @@ -34,9 +34,12 @@ func (t *TransactionHandler) ExtractNonceAndSender(vtx *vochaintx.Tx) (*common.A ptx = payload.SetAccount case *models.Tx_CollectFaucet: ptx = payload.CollectFaucet + case *models.Tx_SetSIK: + ptx = payload.SetSIK + case *models.Tx_DelSIK: + ptx = payload.DelSIK case *models.Tx_Vote, *models.Tx_Admin, *models.Tx_MintTokens, *models.Tx_SetKeykeeper, - *models.Tx_SetTransactionCosts, *models.Tx_DelSIK, *models.Tx_RegisterKey, *models.Tx_SetSIK, - *models.Tx_RegisterSIK: + *models.Tx_SetTransactionCosts, *models.Tx_RegisterKey, *models.Tx_RegisterSIK: // these tx does not have incremental nonce return nil, 0, nil default: