From a81be2f30e22b2e1e6c60d91b972afd56b02d808 Mon Sep 17 00:00:00 2001 From: codchen Date: Fri, 17 Nov 2023 12:09:54 +0800 Subject: [PATCH 01/21] Make ReadMaxTxs atomic (#166) --- internal/mempool/mempool.go | 21 +++------------------ internal/mempool/priority_queue.go | 28 ++++++++++++++++++++++++++++ 2 files changed, 31 insertions(+), 18 deletions(-) diff --git a/internal/mempool/mempool.go b/internal/mempool/mempool.go index ae7b3c7c5..b57603b50 100644 --- a/internal/mempool/mempool.go +++ b/internal/mempool/mempool.go @@ -15,7 +15,6 @@ import ( "github.com/tendermint/tendermint/config" "github.com/tendermint/tendermint/internal/libs/clist" "github.com/tendermint/tendermint/libs/log" - tmmath "github.com/tendermint/tendermint/libs/math" "github.com/tendermint/tendermint/types" ) @@ -420,24 +419,10 @@ func (txmp *TxMempool) ReapMaxTxs(max int) types.Txs { txmp.mtx.RLock() defer txmp.mtx.RUnlock() - numTxs := txmp.priorityIndex.NumTxs() - if max < 0 { - max = numTxs - } - - cap := tmmath.MinInt(numTxs, max) - - // wTxs contains a list of *WrappedTx retrieved from the priority queue that - // need to be re-enqueued prior to returning. - wTxs := make([]*WrappedTx, 0, cap) - txs := make([]types.Tx, 0, cap) - for txmp.priorityIndex.NumTxs() > 0 && len(txs) < max { - wtx := txmp.priorityIndex.PopTx() - txs = append(txs, wtx.tx) - wTxs = append(wTxs, wtx) - } + wTxs := txmp.priorityIndex.PeekTxs(max) + txs := make([]types.Tx, 0, len(wTxs)) for _, wtx := range wTxs { - txmp.priorityIndex.PushTx(wtx) + txs = append(txs, wtx.tx) } return txs } diff --git a/internal/mempool/priority_queue.go b/internal/mempool/priority_queue.go index e31997397..ad3a347a3 100644 --- a/internal/mempool/priority_queue.go +++ b/internal/mempool/priority_queue.go @@ -4,6 +4,8 @@ import ( "container/heap" "sort" "sync" + + tmmath "github.com/tendermint/tendermint/libs/math" ) var _ heap.Interface = (*TxPriorityQueue)(nil) @@ -106,6 +108,32 @@ func (pq *TxPriorityQueue) PopTx() *WrappedTx { return nil } +// dequeue up to `max` transactions and reenqueue while locked +func (pq *TxPriorityQueue) PeekTxs(max int) []*WrappedTx { + pq.mtx.Lock() + defer pq.mtx.Unlock() + + numTxs := len(pq.txs) + if max < 0 { + max = numTxs + } + + cap := tmmath.MinInt(numTxs, max) + res := make([]*WrappedTx, 0, cap) + for i := 0; i < cap; i++ { + popped := heap.Pop(pq) + if popped == nil { + break + } + res = append(res, popped.(*WrappedTx)) + } + + for _, tx := range res { + heap.Push(pq, tx) + } + return res +} + // Push implements the Heap interface. // // NOTE: A caller should never call Push. Use PushTx instead. From e1640c3f8309cf11e2b6aad28fcdf3e880dea65a Mon Sep 17 00:00:00 2001 From: codchen Date: Sat, 16 Dec 2023 22:22:55 +0800 Subject: [PATCH 02/21] Support pending transaction in mempool (#169) --- abci/client/grpc_client.go | 5 ++- abci/client/mocks/client.go | 8 ++-- abci/client/socket_client.go | 4 +- abci/example/kvstore/kvstore.go | 4 +- abci/server/grpc_server.go | 8 ++++ abci/types/application.go | 6 +-- abci/types/messages.go | 4 +- abci/types/mocks/application.go | 8 ++-- abci/types/types.go | 18 ++++++++ internal/consensus/mempool_test.go | 8 ++-- internal/mempool/mempool.go | 45 +++++++++++++++---- internal/mempool/mempool_test.go | 16 +++---- internal/mempool/tx.go | 71 ++++++++++++++++++++++++++++-- internal/proxy/client.go | 2 +- internal/rpc/core/mempool.go | 2 +- internal/state/helpers_test.go | 4 +- rpc/client/mock/abci.go | 2 +- test/e2e/app/app.go | 10 +++-- 18 files changed, 174 insertions(+), 51 deletions(-) diff --git a/abci/client/grpc_client.go b/abci/client/grpc_client.go index f1be2c120..8eec13ef8 100644 --- a/abci/client/grpc_client.go +++ b/abci/client/grpc_client.go @@ -130,8 +130,9 @@ func (cli *grpcClient) Info(ctx context.Context, params *types.RequestInfo) (*ty return cli.client.Info(ctx, types.ToRequestInfo(params).GetInfo(), grpc.WaitForReady(true)) } -func (cli *grpcClient) CheckTx(ctx context.Context, params *types.RequestCheckTx) (*types.ResponseCheckTx, error) { - return cli.client.CheckTx(ctx, types.ToRequestCheckTx(params).GetCheckTx(), grpc.WaitForReady(true)) +func (cli *grpcClient) CheckTx(ctx context.Context, params *types.RequestCheckTx) (*types.ResponseCheckTxV2, error) { + resCheckTx, err := cli.client.CheckTx(ctx, types.ToRequestCheckTx(params).GetCheckTx(), grpc.WaitForReady(true)) + return &types.ResponseCheckTxV2{ResponseCheckTx: resCheckTx}, err } func (cli *grpcClient) Query(ctx context.Context, params *types.RequestQuery) (*types.ResponseQuery, error) { diff --git a/abci/client/mocks/client.go b/abci/client/mocks/client.go index 44a5eb00f..a0626cff0 100644 --- a/abci/client/mocks/client.go +++ b/abci/client/mocks/client.go @@ -38,15 +38,15 @@ func (_m *Client) ApplySnapshotChunk(_a0 context.Context, _a1 *types.RequestAppl } // CheckTx provides a mock function with given fields: _a0, _a1 -func (_m *Client) CheckTx(_a0 context.Context, _a1 *types.RequestCheckTx) (*types.ResponseCheckTx, error) { +func (_m *Client) CheckTx(_a0 context.Context, _a1 *types.RequestCheckTx) (*types.ResponseCheckTxV2, error) { ret := _m.Called(_a0, _a1) - var r0 *types.ResponseCheckTx - if rf, ok := ret.Get(0).(func(context.Context, *types.RequestCheckTx) *types.ResponseCheckTx); ok { + var r0 *types.ResponseCheckTxV2 + if rf, ok := ret.Get(0).(func(context.Context, *types.RequestCheckTx) *types.ResponseCheckTxV2); ok { r0 = rf(_a0, _a1) } else { if ret.Get(0) != nil { - r0 = ret.Get(0).(*types.ResponseCheckTx) + r0 = ret.Get(0).(*types.ResponseCheckTxV2) } } diff --git a/abci/client/socket_client.go b/abci/client/socket_client.go index 80f106333..35f244666 100644 --- a/abci/client/socket_client.go +++ b/abci/client/socket_client.go @@ -257,12 +257,12 @@ func (cli *socketClient) Info(ctx context.Context, req *types.RequestInfo) (*typ return res.GetInfo(), nil } -func (cli *socketClient) CheckTx(ctx context.Context, req *types.RequestCheckTx) (*types.ResponseCheckTx, error) { +func (cli *socketClient) CheckTx(ctx context.Context, req *types.RequestCheckTx) (*types.ResponseCheckTxV2, error) { res, err := cli.doRequest(ctx, types.ToRequestCheckTx(req)) if err != nil { return nil, err } - return res.GetCheckTx(), nil + return &types.ResponseCheckTxV2{ResponseCheckTx: res.GetCheckTx()}, nil } func (cli *socketClient) Query(ctx context.Context, req *types.RequestQuery) (*types.ResponseQuery, error) { diff --git a/abci/example/kvstore/kvstore.go b/abci/example/kvstore/kvstore.go index f33ff2be1..ac0d122b1 100644 --- a/abci/example/kvstore/kvstore.go +++ b/abci/example/kvstore/kvstore.go @@ -205,8 +205,8 @@ func (app *Application) FinalizeBlock(_ context.Context, req *types.RequestFinal return &types.ResponseFinalizeBlock{TxResults: respTxs, ValidatorUpdates: app.ValUpdates, AppHash: appHash}, nil } -func (*Application) CheckTx(_ context.Context, req *types.RequestCheckTx) (*types.ResponseCheckTx, error) { - return &types.ResponseCheckTx{Code: code.CodeTypeOK, GasWanted: 1}, nil +func (*Application) CheckTx(_ context.Context, req *types.RequestCheckTx) (*types.ResponseCheckTxV2, error) { + return &types.ResponseCheckTxV2{ResponseCheckTx: &types.ResponseCheckTx{Code: code.CodeTypeOK, GasWanted: 1}}, nil } func (app *Application) Commit(_ context.Context) (*types.ResponseCommit, error) { diff --git a/abci/server/grpc_server.go b/abci/server/grpc_server.go index 0dfee8169..6f7caf39c 100644 --- a/abci/server/grpc_server.go +++ b/abci/server/grpc_server.go @@ -81,3 +81,11 @@ func (app *gRPCApplication) Flush(_ context.Context, req *types.RequestFlush) (* func (app *gRPCApplication) Commit(ctx context.Context, req *types.RequestCommit) (*types.ResponseCommit, error) { return app.Application.Commit(ctx) } + +func (app *gRPCApplication) CheckTx(ctx context.Context, req *types.RequestCheckTx) (*types.ResponseCheckTx, error) { + resV2, err := app.Application.CheckTx(ctx, req) + if err != nil { + return &types.ResponseCheckTx{}, err + } + return resV2.ResponseCheckTx, nil +} diff --git a/abci/types/application.go b/abci/types/application.go index 9915d16a1..5f98bdc9f 100644 --- a/abci/types/application.go +++ b/abci/types/application.go @@ -12,7 +12,7 @@ type Application interface { Query(context.Context, *RequestQuery) (*ResponseQuery, error) // Query for state // Mempool Connection - CheckTx(context.Context, *RequestCheckTx) (*ResponseCheckTx, error) // Validate a tx for the mempool + CheckTx(context.Context, *RequestCheckTx) (*ResponseCheckTxV2, error) // Validate a tx for the mempool // Consensus Connection InitChain(context.Context, *RequestInitChain) (*ResponseInitChain, error) // Initialize blockchain w validators/other info from TendermintCore @@ -51,8 +51,8 @@ func (BaseApplication) Info(_ context.Context, req *RequestInfo) (*ResponseInfo, return &ResponseInfo{}, nil } -func (BaseApplication) CheckTx(_ context.Context, req *RequestCheckTx) (*ResponseCheckTx, error) { - return &ResponseCheckTx{Code: CodeTypeOK}, nil +func (BaseApplication) CheckTx(_ context.Context, req *RequestCheckTx) (*ResponseCheckTxV2, error) { + return &ResponseCheckTxV2{ResponseCheckTx: &ResponseCheckTx{Code: CodeTypeOK}}, nil } func (BaseApplication) Commit(_ context.Context) (*ResponseCommit, error) { diff --git a/abci/types/messages.go b/abci/types/messages.go index 2ab6ac1df..f6a6c12d1 100644 --- a/abci/types/messages.go +++ b/abci/types/messages.go @@ -155,9 +155,9 @@ func ToResponseInfo(res *ResponseInfo) *Response { } } -func ToResponseCheckTx(res *ResponseCheckTx) *Response { +func ToResponseCheckTx(res *ResponseCheckTxV2) *Response { return &Response{ - Value: &Response_CheckTx{res}, + Value: &Response_CheckTx{res.ResponseCheckTx}, } } diff --git a/abci/types/mocks/application.go b/abci/types/mocks/application.go index c6eb29219..08ef9ca5c 100644 --- a/abci/types/mocks/application.go +++ b/abci/types/mocks/application.go @@ -38,15 +38,15 @@ func (_m *Application) ApplySnapshotChunk(_a0 context.Context, _a1 *types.Reques } // CheckTx provides a mock function with given fields: _a0, _a1 -func (_m *Application) CheckTx(_a0 context.Context, _a1 *types.RequestCheckTx) (*types.ResponseCheckTx, error) { +func (_m *Application) CheckTx(_a0 context.Context, _a1 *types.RequestCheckTx) (*types.ResponseCheckTxV2, error) { ret := _m.Called(_a0, _a1) - var r0 *types.ResponseCheckTx - if rf, ok := ret.Get(0).(func(context.Context, *types.RequestCheckTx) *types.ResponseCheckTx); ok { + var r0 *types.ResponseCheckTxV2 + if rf, ok := ret.Get(0).(func(context.Context, *types.RequestCheckTx) *types.ResponseCheckTxV2); ok { r0 = rf(_a0, _a1) } else { if ret.Get(0) != nil { - r0 = ret.Get(0).(*types.ResponseCheckTx) + r0 = ret.Get(0).(*types.ResponseCheckTxV2) } } diff --git a/abci/types/types.go b/abci/types/types.go index 121e72159..4df6d4751 100644 --- a/abci/types/types.go +++ b/abci/types/types.go @@ -237,3 +237,21 @@ func MarshalTxResults(r []*ExecTxResult) ([][]byte, error) { } return s, nil } + +type PendingTxCheckerResponse int + +const ( + Accepted PendingTxCheckerResponse = iota + Rejected + Pending +) + +type PendingTxChecker func() PendingTxCheckerResponse + +// V2 response type contains non-protobuf fields, so non-local ABCI clients will not be able +// to utilize the new fields in V2 type (but still be backwards-compatible) +type ResponseCheckTxV2 struct { + *ResponseCheckTx + IsPendingTransaction bool + Checker PendingTxChecker // must not be nil if IsPendingTransaction is true +} diff --git a/internal/consensus/mempool_test.go b/internal/consensus/mempool_test.go index e634d4b22..84badbeeb 100644 --- a/internal/consensus/mempool_test.go +++ b/internal/consensus/mempool_test.go @@ -308,18 +308,18 @@ func (app *CounterApplication) FinalizeBlock(_ context.Context, req *abci.Reques return res, nil } -func (app *CounterApplication) CheckTx(_ context.Context, req *abci.RequestCheckTx) (*abci.ResponseCheckTx, error) { +func (app *CounterApplication) CheckTx(_ context.Context, req *abci.RequestCheckTx) (*abci.ResponseCheckTxV2, error) { app.mu.Lock() defer app.mu.Unlock() txValue := txAsUint64(req.Tx) if txValue != uint64(app.mempoolTxCount) { - return &abci.ResponseCheckTx{ + return &abci.ResponseCheckTxV2{ResponseCheckTx: &abci.ResponseCheckTx{ Code: code.CodeTypeBadNonce, - }, nil + }}, nil } app.mempoolTxCount++ - return &abci.ResponseCheckTx{Code: code.CodeTypeOK}, nil + return &abci.ResponseCheckTxV2{ResponseCheckTx: &abci.ResponseCheckTx{Code: code.CodeTypeOK}}, nil } func txAsUint64(tx []byte) uint64 { diff --git a/internal/mempool/mempool.go b/internal/mempool/mempool.go index b57603b50..d2b84910c 100644 --- a/internal/mempool/mempool.go +++ b/internal/mempool/mempool.go @@ -81,6 +81,10 @@ type TxMempool struct { // index. i.e. older transactions are first. timestampIndex *WrappedTxList + // pendingTxs stores transactions that are not valid yet but might become valid + // if its checker returns Accepted + pendingTxs *PendingTxs + // A read/write lock is used to safe guard updates, insertions and deletions // from the mempool. A read-lock is implicitly acquired when executing CheckTx, // however, a caller must explicitly grab a write-lock via Lock when updating @@ -120,6 +124,7 @@ func NewTxMempool( timestampIndex: NewWrappedTxList(func(wtx1, wtx2 *WrappedTx) bool { return wtx1.timestamp.After(wtx2.timestamp) || wtx1.timestamp.Equal(wtx2.timestamp) }), + pendingTxs: NewPendingTxs(), failedCheckTxCounts: map[types.NodeID]uint64{}, peerManager: peerManager, } @@ -286,17 +291,25 @@ func (txmp *TxMempool) CheckTx( height: txmp.height, } - // only add new transaction if checkTx passes if err == nil { - err = txmp.addNewTransaction(wtx, res, txInfo) + // only add new transaction if checkTx passes and is not pending + if !res.IsPendingTransaction { + err = txmp.addNewTransaction(wtx, res.ResponseCheckTx, txInfo) - if err != nil { - return err + if err != nil { + return err + } + } else { + // otherwise add to pending txs store + if res.Checker == nil { + return errors.New("no checker available for pending transaction") + } + txmp.pendingTxs.Insert(wtx, res, txInfo) } } if cb != nil { - cb(res) + cb(res.ResponseCheckTx) } return nil @@ -470,6 +483,7 @@ func (txmp *TxMempool) Update( } } + txmp.handlePendingTransactions() txmp.purgeExpiredTxs(blockHeight) // If there any uncommitted transactions left in the mempool, we either @@ -633,7 +647,7 @@ func (txmp *TxMempool) addNewTransaction(wtx *WrappedTx, res *abci.ResponseCheck // // This method is NOT executed for the initial CheckTx on a new transaction; // that case is handled by addNewTransaction instead. -func (txmp *TxMempool) handleRecheckResult(tx types.Tx, res *abci.ResponseCheckTx) { +func (txmp *TxMempool) handleRecheckResult(tx types.Tx, res *abci.ResponseCheckTxV2) { if txmp.recheckCursor == nil { return } @@ -676,10 +690,11 @@ func (txmp *TxMempool) handleRecheckResult(tx types.Tx, res *abci.ResponseCheckT if !txmp.txStore.IsTxRemoved(wtx.hash) { var err error if txmp.postCheck != nil { - err = txmp.postCheck(tx, res) + err = txmp.postCheck(tx, res.ResponseCheckTx) } - if res.Code == abci.CodeTypeOK && err == nil { + // we will treat a transaction that turns pending in a recheck as invalid and evict it + if res.Code == abci.CodeTypeOK && err == nil && !res.IsPendingTransaction { wtx.priority = res.Priority } else { txmp.logger.Debug( @@ -904,3 +919,17 @@ func (txmp *TxMempool) AppendCheckTxErr(existingLogs string, log string) string jsonData, _ := json.Marshal(logs) return string(jsonData) } + +func (txmp *TxMempool) handlePendingTransactions() { + accepted, rejected := txmp.pendingTxs.EvaluatePendingTransactions() + for _, tx := range accepted { + if err := txmp.addNewTransaction(tx.tx, tx.checkTxResponse.ResponseCheckTx, tx.txInfo); err != nil { + txmp.logger.Error(fmt.Sprintf("error adding pending transaction: %s", err)) + } + } + if !txmp.config.KeepInvalidTxsInCache { + for _, tx := range rejected { + txmp.cache.Remove(tx.tx.tx) + } + } +} diff --git a/internal/mempool/mempool_test.go b/internal/mempool/mempool_test.go index d86ba8f7e..336357aa2 100644 --- a/internal/mempool/mempool_test.go +++ b/internal/mempool/mempool_test.go @@ -37,7 +37,7 @@ type testTx struct { priority int64 } -func (app *application) CheckTx(_ context.Context, req *abci.RequestCheckTx) (*abci.ResponseCheckTx, error) { +func (app *application) CheckTx(_ context.Context, req *abci.RequestCheckTx) (*abci.ResponseCheckTxV2, error) { var ( priority int64 sender string @@ -48,29 +48,29 @@ func (app *application) CheckTx(_ context.Context, req *abci.RequestCheckTx) (*a if len(parts) == 3 { v, err := strconv.ParseInt(string(parts[2]), 10, 64) if err != nil { - return &abci.ResponseCheckTx{ + return &abci.ResponseCheckTxV2{ResponseCheckTx: &abci.ResponseCheckTx{ Priority: priority, Code: 100, GasWanted: 1, - }, nil + }}, nil } priority = v sender = string(parts[0]) } else { - return &abci.ResponseCheckTx{ + return &abci.ResponseCheckTxV2{ResponseCheckTx: &abci.ResponseCheckTx{ Priority: priority, Code: 101, GasWanted: 1, - }, nil + }}, nil } - return &abci.ResponseCheckTx{ + return &abci.ResponseCheckTxV2{ResponseCheckTx: &abci.ResponseCheckTx{ Priority: priority, Sender: sender, Code: code.CodeTypeOK, GasWanted: 1, - }, nil + }}, nil } func setup(t testing.TB, app abciclient.Client, cacheSize int, options ...TxMempoolOption) *TxMempool { @@ -727,4 +727,4 @@ func TestAppendCheckTxErr(t *testing.T) { require.NoError(t, err) require.Equal(t, len(data), 1) require.Equal(t, data[0]["log"], "sample error msg") -} \ No newline at end of file +} diff --git a/internal/mempool/tx.go b/internal/mempool/tx.go index c7113c951..9570935eb 100644 --- a/internal/mempool/tx.go +++ b/internal/mempool/tx.go @@ -5,6 +5,7 @@ import ( "sync" "time" + abci "github.com/tendermint/tendermint/abci/types" "github.com/tendermint/tendermint/internal/libs/clist" "github.com/tendermint/tendermint/types" ) @@ -72,9 +73,9 @@ func (wtx *WrappedTx) Size() int { // TxStore implements a thread-safe mapping of valid transaction(s). // // NOTE: -// - Concurrent read-only access to a *WrappedTx object is OK. However, mutative -// access is not allowed. Regardless, it is not expected for the mempool to -// need mutative access. +// - Concurrent read-only access to a *WrappedTx object is OK. However, mutative +// access is not allowed. Regardless, it is not expected for the mempool to +// need mutative access. type TxStore struct { mtx sync.RWMutex hashTxs map[types.TxKey]*WrappedTx // primary index @@ -291,3 +292,67 @@ func (wtl *WrappedTxList) Remove(wtx *WrappedTx) { i++ } } + +type PendingTxs struct { + mtx *sync.Mutex + txs []PendingTxInfo +} + +type PendingTxInfo struct { + tx *WrappedTx + checkTxResponse *abci.ResponseCheckTxV2 + txInfo TxInfo +} + +func NewPendingTxs() *PendingTxs { + return &PendingTxs{ + mtx: &sync.Mutex{}, + txs: []PendingTxInfo{}, + } +} + +func (p *PendingTxs) EvaluatePendingTransactions() ( + acceptedTxs []PendingTxInfo, + rejectedTxs []PendingTxInfo, +) { + poppedIndices := []int{} + p.mtx.Lock() + defer p.mtx.Unlock() + for i := 0; i < len(p.txs); i++ { + switch p.txs[i].checkTxResponse.Checker() { + case abci.Accepted: + acceptedTxs = append(acceptedTxs, p.txs[i]) + poppedIndices = append(poppedIndices, i) + case abci.Rejected: + rejectedTxs = append(rejectedTxs, p.txs[i]) + poppedIndices = append(poppedIndices, i) + } + } + p.popTxsAtIndices(poppedIndices) + return +} + +// assume mtx is already acquired +func (p *PendingTxs) popTxsAtIndices(indices []int) { + if len(indices) == 0 { + return + } + newTxs := []PendingTxInfo{} + start := 0 + for _, idx := range indices { + newTxs = append(newTxs, p.txs[start:idx]...) + start = idx + } + newTxs = append(newTxs, p.txs[indices[len(indices)-1]:]...) + p.txs = newTxs +} + +func (p *PendingTxs) Insert(tx *WrappedTx, resCheckTx *abci.ResponseCheckTxV2, txInfo TxInfo) { + p.mtx.Lock() + defer p.mtx.Unlock() + p.txs = append(p.txs, PendingTxInfo{ + tx: tx, + checkTxResponse: resCheckTx, + txInfo: txInfo, + }) +} diff --git a/internal/proxy/client.go b/internal/proxy/client.go index fa1d8c66e..f3ab28f5c 100644 --- a/internal/proxy/client.go +++ b/internal/proxy/client.go @@ -169,7 +169,7 @@ func (app *proxyClient) Flush(ctx context.Context) error { return app.client.Flush(ctx) } -func (app *proxyClient) CheckTx(ctx context.Context, req *types.RequestCheckTx) (*types.ResponseCheckTx, error) { +func (app *proxyClient) CheckTx(ctx context.Context, req *types.RequestCheckTx) (*types.ResponseCheckTxV2, error) { defer addTimeSample(app.metrics.MethodTiming.With("method", "check_tx", "type", "sync"))() return app.client.CheckTx(ctx, req) } diff --git a/internal/rpc/core/mempool.go b/internal/rpc/core/mempool.go index 7cd1bcd7d..917e00b47 100644 --- a/internal/rpc/core/mempool.go +++ b/internal/rpc/core/mempool.go @@ -185,7 +185,7 @@ func (env *Environment) CheckTx(ctx context.Context, req *coretypes.RequestCheck if err != nil { return nil, err } - return &coretypes.ResultCheckTx{ResponseCheckTx: *res}, nil + return &coretypes.ResultCheckTx{ResponseCheckTx: *res.ResponseCheckTx}, nil } func (env *Environment) RemoveTx(ctx context.Context, req *coretypes.RequestRemoveTx) error { diff --git a/internal/state/helpers_test.go b/internal/state/helpers_test.go index 4284bad68..1d0d59441 100644 --- a/internal/state/helpers_test.go +++ b/internal/state/helpers_test.go @@ -302,8 +302,8 @@ func (app *testApp) FinalizeBlock(_ context.Context, req *abci.RequestFinalizeBl }, nil } -func (app *testApp) CheckTx(_ context.Context, req *abci.RequestCheckTx) (*abci.ResponseCheckTx, error) { - return &abci.ResponseCheckTx{}, nil +func (app *testApp) CheckTx(_ context.Context, req *abci.RequestCheckTx) (*abci.ResponseCheckTxV2, error) { + return &abci.ResponseCheckTxV2{ResponseCheckTx: &abci.ResponseCheckTx{}}, nil } func (app *testApp) Commit(context.Context) (*abci.ResponseCommit, error) { diff --git a/rpc/client/mock/abci.go b/rpc/client/mock/abci.go index 142d64d19..27d779554 100644 --- a/rpc/client/mock/abci.go +++ b/rpc/client/mock/abci.go @@ -60,7 +60,7 @@ func (a ABCIApp) BroadcastTxCommit(ctx context.Context, tx types.Tx) (*coretypes return nil, err } - res := &coretypes.ResultBroadcastTxCommit{CheckTx: *resp} + res := &coretypes.ResultBroadcastTxCommit{CheckTx: *resp.ResponseCheckTx} if res.CheckTx.IsErr() { return res, nil } diff --git a/test/e2e/app/app.go b/test/e2e/app/app.go index 885b49f39..b127fa744 100644 --- a/test/e2e/app/app.go +++ b/test/e2e/app/app.go @@ -163,14 +163,14 @@ func (app *Application) InitChain(_ context.Context, req *abci.RequestInitChain) } // CheckTx implements ABCI. -func (app *Application) CheckTx(_ context.Context, req *abci.RequestCheckTx) (*abci.ResponseCheckTx, error) { +func (app *Application) CheckTx(_ context.Context, req *abci.RequestCheckTx) (*abci.ResponseCheckTxV2, error) { app.mu.Lock() defer app.mu.Unlock() _, _, err := parseTx(req.Tx) if err != nil { - return &abci.ResponseCheckTx{ - Code: code.CodeTypeEncodingError, + return &abci.ResponseCheckTxV2{ + ResponseCheckTx: &abci.ResponseCheckTx{Code: code.CodeTypeEncodingError}, }, nil } @@ -178,7 +178,9 @@ func (app *Application) CheckTx(_ context.Context, req *abci.RequestCheckTx) (*a time.Sleep(time.Duration(app.cfg.CheckTxDelayMS) * time.Millisecond) } - return &abci.ResponseCheckTx{Code: code.CodeTypeOK, GasWanted: 1}, nil + return &abci.ResponseCheckTxV2{ + ResponseCheckTx: &abci.ResponseCheckTx{Code: code.CodeTypeOK, GasWanted: 1}, + }, nil } // FinalizeBlock implements ABCI. From 56bd6348f8939fcc2516904fcef9735a2347cc49 Mon Sep 17 00:00:00 2001 From: codchen Date: Wed, 20 Dec 2023 21:43:55 +0800 Subject: [PATCH 03/21] fix unconfirmed tx to consider pending txs (#172) --- internal/mempool/mempool.go | 9 ++++++++- internal/mempool/tx.go | 20 ++++++++++++++++++-- internal/rpc/core/mempool.go | 2 ++ 3 files changed, 28 insertions(+), 3 deletions(-) diff --git a/internal/mempool/mempool.go b/internal/mempool/mempool.go index d2b84910c..d38fe9e5c 100644 --- a/internal/mempool/mempool.go +++ b/internal/mempool/mempool.go @@ -177,7 +177,7 @@ func (txmp *TxMempool) Unlock() { // Size returns the number of valid transactions in the mempool. It is // thread-safe. func (txmp *TxMempool) Size() int { - return txmp.txStore.Size() + return txmp.txStore.Size() + txmp.pendingTxs.Size() } // SizeBytes return the total sum in bytes of all the valid transactions in the @@ -437,6 +437,13 @@ func (txmp *TxMempool) ReapMaxTxs(max int) types.Txs { for _, wtx := range wTxs { txs = append(txs, wtx.tx) } + if len(txs) < max { + // retrieve more from pending txs + pending := txmp.pendingTxs.Peek(max - len(txs)) + for _, ptx := range pending { + txs = append(txs, ptx.tx.tx) + } + } return txs } diff --git a/internal/mempool/tx.go b/internal/mempool/tx.go index 9570935eb..e5fd7b16f 100644 --- a/internal/mempool/tx.go +++ b/internal/mempool/tx.go @@ -294,7 +294,7 @@ func (wtl *WrappedTxList) Remove(wtx *WrappedTx) { } type PendingTxs struct { - mtx *sync.Mutex + mtx *sync.RWMutex txs []PendingTxInfo } @@ -306,7 +306,7 @@ type PendingTxInfo struct { func NewPendingTxs() *PendingTxs { return &PendingTxs{ - mtx: &sync.Mutex{}, + mtx: &sync.RWMutex{}, txs: []PendingTxInfo{}, } } @@ -356,3 +356,19 @@ func (p *PendingTxs) Insert(tx *WrappedTx, resCheckTx *abci.ResponseCheckTxV2, t txInfo: txInfo, }) } + +func (p *PendingTxs) Peek(max int) []PendingTxInfo { + p.mtx.RLock() + defer p.mtx.RUnlock() + // priority is fifo + if max > len(p.txs) { + return p.txs + } + return p.txs[:max] +} + +func (p *PendingTxs) Size() int { + p.mtx.RLock() + defer p.mtx.RUnlock() + return len(p.txs) +} diff --git a/internal/rpc/core/mempool.go b/internal/rpc/core/mempool.go index 917e00b47..12fa2d339 100644 --- a/internal/rpc/core/mempool.go +++ b/internal/rpc/core/mempool.go @@ -149,6 +149,7 @@ func (env *Environment) BroadcastTxCommit(ctx context.Context, req *coretypes.Re // More: https://docs.tendermint.com/master/rpc/#/Info/unconfirmed_txs func (env *Environment) UnconfirmedTxs(ctx context.Context, req *coretypes.RequestUnconfirmedTxs) (*coretypes.ResultUnconfirmedTxs, error) { totalCount := env.Mempool.Size() + fmt.Printf("EVMTEST mempool size: %d\n", totalCount) perPage := env.validatePerPage(req.PerPage.IntPtr()) page, err := validatePage(req.Page.IntPtr(), perPage, totalCount) if err != nil { @@ -160,6 +161,7 @@ func (env *Environment) UnconfirmedTxs(ctx context.Context, req *coretypes.Reque txs := env.Mempool.ReapMaxTxs(skipCount + tmmath.MinInt(perPage, totalCount-skipCount)) result := txs[skipCount:] + fmt.Printf("EVMTEST unconfirmed tx result count: %d, skipping %d\n", len(result), skipCount) return &coretypes.ResultUnconfirmedTxs{ Count: len(result), Total: totalCount, From e475a94bb958c0e3c47720fd0d87dcb9b7c9895a Mon Sep 17 00:00:00 2001 From: codchen Date: Wed, 20 Dec 2023 22:25:47 +0800 Subject: [PATCH 04/21] fix pending pop (#173) --- internal/mempool/mempool.go | 4 +++- internal/mempool/tx.go | 2 +- internal/rpc/core/mempool.go | 2 -- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/internal/mempool/mempool.go b/internal/mempool/mempool.go index d38fe9e5c..41f5b587c 100644 --- a/internal/mempool/mempool.go +++ b/internal/mempool/mempool.go @@ -177,7 +177,9 @@ func (txmp *TxMempool) Unlock() { // Size returns the number of valid transactions in the mempool. It is // thread-safe. func (txmp *TxMempool) Size() int { - return txmp.txStore.Size() + txmp.pendingTxs.Size() + txSize := txmp.txStore.Size() + pendingSize := txmp.pendingTxs.Size() + return txSize + pendingSize } // SizeBytes return the total sum in bytes of all the valid transactions in the diff --git a/internal/mempool/tx.go b/internal/mempool/tx.go index e5fd7b16f..c74872054 100644 --- a/internal/mempool/tx.go +++ b/internal/mempool/tx.go @@ -343,7 +343,7 @@ func (p *PendingTxs) popTxsAtIndices(indices []int) { newTxs = append(newTxs, p.txs[start:idx]...) start = idx } - newTxs = append(newTxs, p.txs[indices[len(indices)-1]:]...) + newTxs = append(newTxs, p.txs[start+1:]...) p.txs = newTxs } diff --git a/internal/rpc/core/mempool.go b/internal/rpc/core/mempool.go index 12fa2d339..917e00b47 100644 --- a/internal/rpc/core/mempool.go +++ b/internal/rpc/core/mempool.go @@ -149,7 +149,6 @@ func (env *Environment) BroadcastTxCommit(ctx context.Context, req *coretypes.Re // More: https://docs.tendermint.com/master/rpc/#/Info/unconfirmed_txs func (env *Environment) UnconfirmedTxs(ctx context.Context, req *coretypes.RequestUnconfirmedTxs) (*coretypes.ResultUnconfirmedTxs, error) { totalCount := env.Mempool.Size() - fmt.Printf("EVMTEST mempool size: %d\n", totalCount) perPage := env.validatePerPage(req.PerPage.IntPtr()) page, err := validatePage(req.Page.IntPtr(), perPage, totalCount) if err != nil { @@ -161,7 +160,6 @@ func (env *Environment) UnconfirmedTxs(ctx context.Context, req *coretypes.Reque txs := env.Mempool.ReapMaxTxs(skipCount + tmmath.MinInt(perPage, totalCount-skipCount)) result := txs[skipCount:] - fmt.Printf("EVMTEST unconfirmed tx result count: %d, skipping %d\n", len(result), skipCount) return &coretypes.ResultUnconfirmedTxs{ Count: len(result), Total: totalCount, From 20b4b85659fec86062ae9195d3e44b62bed50df2 Mon Sep 17 00:00:00 2001 From: codchen Date: Tue, 2 Jan 2024 23:45:44 +0800 Subject: [PATCH 05/21] add TTL for pending txs (#174) --- internal/mempool/mempool.go | 6 ++++-- internal/mempool/tx.go | 38 +++++++++++++++++++++++++++++++++++++ 2 files changed, 42 insertions(+), 2 deletions(-) diff --git a/internal/mempool/mempool.go b/internal/mempool/mempool.go index 41f5b587c..56d0cb6eb 100644 --- a/internal/mempool/mempool.go +++ b/internal/mempool/mempool.go @@ -492,8 +492,8 @@ func (txmp *TxMempool) Update( } } - txmp.handlePendingTransactions() txmp.purgeExpiredTxs(blockHeight) + txmp.handlePendingTransactions() // If there any uncommitted transactions left in the mempool, we either // initiate re-CheckTx per remaining transaction or notify that remaining @@ -844,7 +844,7 @@ func (txmp *TxMempool) removeTx(wtx *WrappedTx, removeFromCache bool) { // purgeExpiredTxs removes all transactions that have exceeded their respective // height- and/or time-based TTLs from their respective indexes. Every expired -// transaction will be removed from the mempool, but preserved in the cache. +// transaction will be removed from the mempool, but preserved in the cache (except for pending txs). // // NOTE: purgeExpiredTxs must only be called during TxMempool#Update in which // the caller has a write-lock on the mempool and so we can safely iterate over @@ -890,6 +890,8 @@ func (txmp *TxMempool) purgeExpiredTxs(blockHeight int64) { for _, wtx := range expiredTxs { txmp.removeTx(wtx, false) } + + txmp.pendingTxs.PurgeExpired(txmp.config.TTLNumBlocks, blockHeight, txmp.config.TTLDuration, now, txmp.cache.Remove) } func (txmp *TxMempool) notifyTxsAvailable() { diff --git a/internal/mempool/tx.go b/internal/mempool/tx.go index c74872054..2f3ec7b8f 100644 --- a/internal/mempool/tx.go +++ b/internal/mempool/tx.go @@ -372,3 +372,41 @@ func (p *PendingTxs) Size() int { defer p.mtx.RUnlock() return len(p.txs) } + +func (p *PendingTxs) PurgeExpired(ttlNumBlock int64, blockHeight int64, ttlDuration time.Duration, now time.Time, cb func(types.Tx)) { + p.mtx.Lock() + defer p.mtx.Unlock() + + if len(p.txs) == 0 { + return + } + + // txs retains the ordering of insertion + if ttlNumBlock > 0 { + idxFirstNotExpiredTx := len(p.txs) + for i, ptx := range p.txs { + if (blockHeight - ptx.tx.height) <= ttlNumBlock { + idxFirstNotExpiredTx = i + } else { + cb(ptx.tx.tx) + } + } + p.txs = p.txs[idxFirstNotExpiredTx:] + } + + if len(p.txs) == 0 { + return + } + + if ttlDuration > 0 { + idxFirstNotExpiredTx := len(p.txs) + for i, ptx := range p.txs { + if now.Sub(ptx.tx.timestamp) <= ttlDuration { + idxFirstNotExpiredTx = i + } else { + cb(ptx.tx.tx) + } + } + p.txs = p.txs[idxFirstNotExpiredTx:] + } +} From fa152a0786cdd0e1fc6b3ebf9bb908926880d967 Mon Sep 17 00:00:00 2001 From: Steven Landers Date: Thu, 4 Jan 2024 09:22:26 -0500 Subject: [PATCH 06/21] [EVM] Fix evm pending nonce (#179) * Perf: Increase buffer size for pubsub server to boost performance (#167) * Increase buffer size for pubsub server * Add more timeout for test failure * Add more timeout * Fix test split scripts * Fix test split * Fix unit test * Unit test * Unit test * [P2P] Optimize block pool requester retry and peer pick up logic (#170) * P2P Improvements: Fix block sync reactor and block pool retry logic * Revert "Add event data to result event (#165)" (#176) This reverts commit 72bb29caf821ed03900bcd74f4ddb439df85f5f1. * Fix block sync auto restart not working as expected (#175) * Fix edge case for blocksync (#178) * fix evm pending nonce * fix test * deflake a test * de-flake test * Revert "merge main" This reverts commit 58b94248ac561583b918775c0b0e220d4225fbc0, reversing changes made to 02d14782edc6c58d59db2658a4190dbdc5c7280f. * consider keep-in-cache logic when removing from cache * undo test tweaks --------- Co-authored-by: Yiming Zang <50607998+yzang2019@users.noreply.github.com> Co-authored-by: Jeremy Wei --- abci/types/types.go | 4 +++- internal/mempool/mempool.go | 20 ++++++++++++++------ internal/mempool/tx.go | 30 ++++++++++++++++++------------ 3 files changed, 35 insertions(+), 19 deletions(-) diff --git a/abci/types/types.go b/abci/types/types.go index 4df6d4751..b40865524 100644 --- a/abci/types/types.go +++ b/abci/types/types.go @@ -247,11 +247,13 @@ const ( ) type PendingTxChecker func() PendingTxCheckerResponse +type ExpireTxHandler func() -// V2 response type contains non-protobuf fields, so non-local ABCI clients will not be able +// ResponseCheckTxV2 response type contains non-protobuf fields, so non-local ABCI clients will not be able // to utilize the new fields in V2 type (but still be backwards-compatible) type ResponseCheckTxV2 struct { *ResponseCheckTx IsPendingTransaction bool Checker PendingTxChecker // must not be nil if IsPendingTransaction is true + ExpireTxHandler ExpireTxHandler } diff --git a/internal/mempool/mempool.go b/internal/mempool/mempool.go index 56d0cb6eb..084313cec 100644 --- a/internal/mempool/mempool.go +++ b/internal/mempool/mempool.go @@ -291,13 +291,20 @@ func (txmp *TxMempool) CheckTx( hash: txHash, timestamp: time.Now().UTC(), height: txmp.height, + expiredCallback: func(removeFromCache bool) { + if removeFromCache { + txmp.cache.Remove(tx) + } + if res.ExpireTxHandler != nil { + res.ExpireTxHandler() + } + }, } if err == nil { // only add new transaction if checkTx passes and is not pending if !res.IsPendingTransaction { err = txmp.addNewTransaction(wtx, res.ResponseCheckTx, txInfo) - if err != nil { return err } @@ -837,9 +844,7 @@ func (txmp *TxMempool) removeTx(wtx *WrappedTx, removeFromCache bool) { atomic.AddInt64(&txmp.sizeBytes, int64(-wtx.Size())) - if removeFromCache { - txmp.cache.Remove(wtx.tx) - } + wtx.expiredCallback(removeFromCache) } // purgeExpiredTxs removes all transactions that have exceeded their respective @@ -888,10 +893,13 @@ func (txmp *TxMempool) purgeExpiredTxs(blockHeight int64) { } for _, wtx := range expiredTxs { - txmp.removeTx(wtx, false) + txmp.removeTx(wtx, !txmp.config.KeepInvalidTxsInCache) } - txmp.pendingTxs.PurgeExpired(txmp.config.TTLNumBlocks, blockHeight, txmp.config.TTLDuration, now, txmp.cache.Remove) + // remove pending txs that have expired + txmp.pendingTxs.PurgeExpired(txmp.config.TTLNumBlocks, blockHeight, txmp.config.TTLDuration, now, func(wtx *WrappedTx) { + wtx.expiredCallback(!txmp.config.KeepInvalidTxsInCache) + }) } func (txmp *TxMempool) notifyTxsAvailable() { diff --git a/internal/mempool/tx.go b/internal/mempool/tx.go index 2f3ec7b8f..4d5762be5 100644 --- a/internal/mempool/tx.go +++ b/internal/mempool/tx.go @@ -64,6 +64,9 @@ type WrappedTx struct { // transaction in the mempool can be evicted when it is simultaneously having // a reCheckTx callback executed. removed bool + + // this is the callback that can be called when a transaction expires + expiredCallback func(removeFromCache bool) } func (wtx *WrappedTx) Size() int { @@ -295,10 +298,10 @@ func (wtl *WrappedTxList) Remove(wtx *WrappedTx) { type PendingTxs struct { mtx *sync.RWMutex - txs []PendingTxInfo + txs []TxWithResponse } -type PendingTxInfo struct { +type TxWithResponse struct { tx *WrappedTx checkTxResponse *abci.ResponseCheckTxV2 txInfo TxInfo @@ -307,13 +310,12 @@ type PendingTxInfo struct { func NewPendingTxs() *PendingTxs { return &PendingTxs{ mtx: &sync.RWMutex{}, - txs: []PendingTxInfo{}, + txs: []TxWithResponse{}, } } - func (p *PendingTxs) EvaluatePendingTransactions() ( - acceptedTxs []PendingTxInfo, - rejectedTxs []PendingTxInfo, + acceptedTxs []TxWithResponse, + rejectedTxs []TxWithResponse, ) { poppedIndices := []int{} p.mtx.Lock() @@ -337,7 +339,7 @@ func (p *PendingTxs) popTxsAtIndices(indices []int) { if len(indices) == 0 { return } - newTxs := []PendingTxInfo{} + newTxs := []TxWithResponse{} start := 0 for _, idx := range indices { newTxs = append(newTxs, p.txs[start:idx]...) @@ -350,14 +352,14 @@ func (p *PendingTxs) popTxsAtIndices(indices []int) { func (p *PendingTxs) Insert(tx *WrappedTx, resCheckTx *abci.ResponseCheckTxV2, txInfo TxInfo) { p.mtx.Lock() defer p.mtx.Unlock() - p.txs = append(p.txs, PendingTxInfo{ + p.txs = append(p.txs, TxWithResponse{ tx: tx, checkTxResponse: resCheckTx, txInfo: txInfo, }) } -func (p *PendingTxs) Peek(max int) []PendingTxInfo { +func (p *PendingTxs) Peek(max int) []TxWithResponse { p.mtx.RLock() defer p.mtx.RUnlock() // priority is fifo @@ -373,7 +375,7 @@ func (p *PendingTxs) Size() int { return len(p.txs) } -func (p *PendingTxs) PurgeExpired(ttlNumBlock int64, blockHeight int64, ttlDuration time.Duration, now time.Time, cb func(types.Tx)) { +func (p *PendingTxs) PurgeExpired(ttlNumBlock int64, blockHeight int64, ttlDuration time.Duration, now time.Time, cb func(wtx *WrappedTx)) { p.mtx.Lock() defer p.mtx.Unlock() @@ -385,10 +387,12 @@ func (p *PendingTxs) PurgeExpired(ttlNumBlock int64, blockHeight int64, ttlDurat if ttlNumBlock > 0 { idxFirstNotExpiredTx := len(p.txs) for i, ptx := range p.txs { + // once found, we can break because these are ordered if (blockHeight - ptx.tx.height) <= ttlNumBlock { idxFirstNotExpiredTx = i + break } else { - cb(ptx.tx.tx) + cb(ptx.tx) } } p.txs = p.txs[idxFirstNotExpiredTx:] @@ -401,10 +405,12 @@ func (p *PendingTxs) PurgeExpired(ttlNumBlock int64, blockHeight int64, ttlDurat if ttlDuration > 0 { idxFirstNotExpiredTx := len(p.txs) for i, ptx := range p.txs { + // once found, we can break because these are ordered if now.Sub(ptx.tx.timestamp) <= ttlDuration { idxFirstNotExpiredTx = i + break } else { - cb(ptx.tx.tx) + cb(ptx.tx) } } p.txs = p.txs[idxFirstNotExpiredTx:] From c95208eb30f2a2aeeec21d89c4199877ed526cd7 Mon Sep 17 00:00:00 2001 From: codchen Date: Mon, 15 Jan 2024 21:11:47 +0800 Subject: [PATCH 07/21] Fix bug when popping pending TXs (#188) --- internal/mempool/tx.go | 10 ++++-- internal/mempool/tx_test.go | 72 +++++++++++++++++++++++++++++++++++++ 2 files changed, 80 insertions(+), 2 deletions(-) diff --git a/internal/mempool/tx.go b/internal/mempool/tx.go index 4d5762be5..be1d9290f 100644 --- a/internal/mempool/tx.go +++ b/internal/mempool/tx.go @@ -342,10 +342,16 @@ func (p *PendingTxs) popTxsAtIndices(indices []int) { newTxs := []TxWithResponse{} start := 0 for _, idx := range indices { + if idx <= start-1 { + panic("indices popped from pending tx store should be sorted without duplicate") + } + if idx >= len(p.txs) { + panic("indices popped from pending tx store out of range") + } newTxs = append(newTxs, p.txs[start:idx]...) - start = idx + start = idx + 1 } - newTxs = append(newTxs, p.txs[start+1:]...) + newTxs = append(newTxs, p.txs[start:]...) p.txs = newTxs } diff --git a/internal/mempool/tx_test.go b/internal/mempool/tx_test.go index c6d494b04..77a49c276 100644 --- a/internal/mempool/tx_test.go +++ b/internal/mempool/tx_test.go @@ -229,3 +229,75 @@ func TestWrappedTxList_Remove(t *testing.T) { sort.Ints(expected) require.Equal(t, expected, got) } + +func TestPendingTxsPopTxsGood(t *testing.T) { + pendingTxs := NewPendingTxs() + for _, test := range []struct { + origLen int + popIndices []int + expected []int + }{ + { + origLen: 1, + popIndices: []int{}, + expected: []int{0}, + }, { + origLen: 1, + popIndices: []int{0}, + expected: []int{}, + }, { + origLen: 2, + popIndices: []int{0}, + expected: []int{1}, + }, { + origLen: 2, + popIndices: []int{1}, + expected: []int{0}, + }, { + origLen: 2, + popIndices: []int{0, 1}, + expected: []int{}, + }, { + origLen: 3, + popIndices: []int{1}, + expected: []int{0, 2}, + }, { + origLen: 3, + popIndices: []int{0, 2}, + expected: []int{1}, + }, { + origLen: 3, + popIndices: []int{0, 1, 2}, + expected: []int{}, + }, { + origLen: 5, + popIndices: []int{0, 1, 4}, + expected: []int{2, 3}, + }, { + origLen: 5, + popIndices: []int{1, 3}, + expected: []int{0, 2, 4}, + }, + } { + pendingTxs.txs = []TxWithResponse{} + for i := 0; i < test.origLen; i++ { + pendingTxs.txs = append(pendingTxs.txs, TxWithResponse{txInfo: TxInfo{SenderID: uint16(i)}}) + } + pendingTxs.popTxsAtIndices(test.popIndices) + require.Equal(t, len(test.expected), len(pendingTxs.txs)) + for i, e := range test.expected { + require.Equal(t, e, int(pendingTxs.txs[i].txInfo.SenderID)) + } + } +} + +func TestPendingTxsPopTxsBad(t *testing.T) { + pendingTxs := NewPendingTxs() + // out of range + require.Panics(t, func() { pendingTxs.popTxsAtIndices([]int{0}) }) + // out of order + pendingTxs.txs = []TxWithResponse{{}, {}, {}} + require.Panics(t, func() { pendingTxs.popTxsAtIndices([]int{1, 0}) }) + // duplicate + require.Panics(t, func() { pendingTxs.popTxsAtIndices([]int{2, 2}) }) +} From 8528f3a480cbe34d05a66cf5688c4a67f9f63a78 Mon Sep 17 00:00:00 2001 From: Yiming Zang <50607998+yzang2019@users.noreply.github.com> Date: Wed, 17 Jan 2024 09:42:08 -0800 Subject: [PATCH 08/21] Add mempool metrics for number of pending tx and expired txs (#189) * Add metrics for mempool pending transaction size * Add expired tx count metrics --- internal/mempool/mempool.go | 9 +++++++++ internal/mempool/mempool_test.go | 1 + internal/mempool/metrics.gen.go | 14 ++++++++++++++ internal/mempool/metrics.go | 9 +++++++++ 4 files changed, 33 insertions(+) diff --git a/internal/mempool/mempool.go b/internal/mempool/mempool.go index 084313cec..b76e1f007 100644 --- a/internal/mempool/mempool.go +++ b/internal/mempool/mempool.go @@ -182,6 +182,11 @@ func (txmp *TxMempool) Size() int { return txSize + pendingSize } +// PendingSize returns the number of pending transactions in the mempool. +func (txmp *TxMempool) PendingSize() int { + return txmp.pendingTxs.Size() +} + // SizeBytes return the total sum in bytes of all the valid transactions in the // mempool. It is thread-safe. func (txmp *TxMempool) SizeBytes() int64 { @@ -292,6 +297,7 @@ func (txmp *TxMempool) CheckTx( timestamp: time.Now().UTC(), height: txmp.height, expiredCallback: func(removeFromCache bool) { + txmp.metrics.ExpiredTxs.Add(1) if removeFromCache { txmp.cache.Remove(tx) } @@ -519,6 +525,7 @@ func (txmp *TxMempool) Update( } txmp.metrics.Size.Set(float64(txmp.Size())) + txmp.metrics.PendingSize.Set(float64(txmp.PendingSize())) return nil } @@ -638,6 +645,7 @@ func (txmp *TxMempool) addNewTransaction(wtx *WrappedTx, res *abci.ResponseCheck txmp.metrics.TxSizeBytes.Observe(float64(wtx.Size())) txmp.metrics.Size.Set(float64(txmp.Size())) + txmp.metrics.PendingSize.Set(float64(txmp.PendingSize())) txmp.insertTx(wtx) txmp.logger.Debug( @@ -745,6 +753,7 @@ func (txmp *TxMempool) handleRecheckResult(tx types.Tx, res *abci.ResponseCheckT } txmp.metrics.Size.Set(float64(txmp.Size())) + txmp.metrics.PendingSize.Set(float64(txmp.PendingSize())) } // updateReCheckTxs updates the recheck cursors using the gossipIndex. For diff --git a/internal/mempool/mempool_test.go b/internal/mempool/mempool_test.go index 336357aa2..d90a12277 100644 --- a/internal/mempool/mempool_test.go +++ b/internal/mempool/mempool_test.go @@ -215,6 +215,7 @@ func TestTxMempool_Size(t *testing.T) { txmp := setup(t, client, 0) txs := checkTxs(ctx, t, txmp, 100, 0) require.Equal(t, len(txs), txmp.Size()) + require.Equal(t, 0, txmp.PendingSize()) require.Equal(t, int64(5690), txmp.SizeBytes()) rawTxs := make([]types.Tx, len(txs)) diff --git a/internal/mempool/metrics.gen.go b/internal/mempool/metrics.gen.go index 100c5e71c..c5a44f029 100644 --- a/internal/mempool/metrics.gen.go +++ b/internal/mempool/metrics.gen.go @@ -20,6 +20,12 @@ func PrometheusMetrics(namespace string, labelsAndValues ...string) *Metrics { Name: "size", Help: "Number of uncommitted transactions in the mempool.", }, labels).With(labelsAndValues...), + PendingSize: prometheus.NewGaugeFrom(stdprometheus.GaugeOpts{ + Namespace: namespace, + Subsystem: MetricsSubsystem, + Name: "pending_size", + Help: "Number of pending transactions in mempool", + }, labels).With(labelsAndValues...), TxSizeBytes: prometheus.NewHistogramFrom(stdprometheus.HistogramOpts{ Namespace: namespace, Subsystem: MetricsSubsystem, @@ -46,6 +52,12 @@ func PrometheusMetrics(namespace string, labelsAndValues ...string) *Metrics { Name: "evicted_txs", Help: "Number of evicted transactions.", }, labels).With(labelsAndValues...), + ExpiredTxs: prometheus.NewCounterFrom(stdprometheus.CounterOpts{ + Namespace: namespace, + Subsystem: MetricsSubsystem, + Name: "expired_txs", + Help: "Number of expired transactions.", + }, labels).With(labelsAndValues...), RecheckTimes: prometheus.NewCounterFrom(stdprometheus.CounterOpts{ Namespace: namespace, Subsystem: MetricsSubsystem, @@ -58,10 +70,12 @@ func PrometheusMetrics(namespace string, labelsAndValues ...string) *Metrics { func NopMetrics() *Metrics { return &Metrics{ Size: discard.NewGauge(), + PendingSize: discard.NewGauge(), TxSizeBytes: discard.NewHistogram(), FailedTxs: discard.NewCounter(), RejectedTxs: discard.NewCounter(), EvictedTxs: discard.NewCounter(), + ExpiredTxs: discard.NewCounter(), RecheckTimes: discard.NewCounter(), } } diff --git a/internal/mempool/metrics.go b/internal/mempool/metrics.go index 532307635..eb296182d 100644 --- a/internal/mempool/metrics.go +++ b/internal/mempool/metrics.go @@ -18,6 +18,9 @@ type Metrics struct { // Number of uncommitted transactions in the mempool. Size metrics.Gauge + // Number of pending transactions in mempool + PendingSize metrics.Gauge + // Histogram of transaction sizes in bytes. TxSizeBytes metrics.Histogram `metrics_buckettype:"exp" metrics_bucketsizes:"1,3,7"` @@ -38,6 +41,12 @@ type Metrics struct { //metrics:Number of evicted transactions. EvictedTxs metrics.Counter + // ExpiredTxs defines the number of expired transactions. These are valid + // transactions that passed CheckTx and existed in the mempool but were not + // get picked up in time and eventually got expired and removed from mempool + //metrics:Number of expired transactions. + ExpiredTxs metrics.Counter + // Number of times transactions are rechecked in the mempool. RecheckTimes metrics.Counter } From 7ec8f9488804902d7f8ba93f7da5bd2b379576c1 Mon Sep 17 00:00:00 2001 From: Steven Landers Date: Fri, 19 Jan 2024 10:24:27 -0500 Subject: [PATCH 09/21] [EVM] Allow multiple txs from same account in a block (#190) * add mempool prioritization with evm nonce * fix priority stability * index fixes * replace with binary search insert * impl binary search --- abci/types/types.go | 5 + internal/mempool/mempool.go | 11 +- internal/mempool/mempool_test.go | 94 +++++++++++++++- internal/mempool/priority_queue.go | 138 ++++++++++++++++++++---- internal/mempool/priority_queue_test.go | 122 +++++++++++++++++++++ internal/mempool/tx.go | 5 + 6 files changed, 352 insertions(+), 23 deletions(-) diff --git a/abci/types/types.go b/abci/types/types.go index b40865524..1ecb0a9f3 100644 --- a/abci/types/types.go +++ b/abci/types/types.go @@ -256,4 +256,9 @@ type ResponseCheckTxV2 struct { IsPendingTransaction bool Checker PendingTxChecker // must not be nil if IsPendingTransaction is true ExpireTxHandler ExpireTxHandler + + // helper properties for prioritization in mempool + EVMNonce uint64 + EVMSenderAddress string + IsEVM bool } diff --git a/internal/mempool/mempool.go b/internal/mempool/mempool.go index b76e1f007..7bccb5479 100644 --- a/internal/mempool/mempool.go +++ b/internal/mempool/mempool.go @@ -292,10 +292,13 @@ func (txmp *TxMempool) CheckTx( } wtx := &WrappedTx{ - tx: tx, - hash: txHash, - timestamp: time.Now().UTC(), - height: txmp.height, + tx: tx, + hash: txHash, + timestamp: time.Now().UTC(), + height: txmp.height, + evmNonce: res.EVMNonce, + evmAddress: res.EVMSenderAddress, + isEVM: res.IsEVM, expiredCallback: func(removeFromCache bool) { txmp.metrics.ExpiredTxs.Add(1) if removeFromCache { diff --git a/internal/mempool/mempool_test.go b/internal/mempool/mempool_test.go index d90a12277..dd4874417 100644 --- a/internal/mempool/mempool_test.go +++ b/internal/mempool/mempool_test.go @@ -43,6 +43,42 @@ func (app *application) CheckTx(_ context.Context, req *abci.RequestCheckTx) (*a sender string ) + if strings.HasPrefix(string(req.Tx), "evm") { + // format is evm-sender-0=account=priority=nonce + // split into respective vars + parts := bytes.Split(req.Tx, []byte("=")) + sender = string(parts[0]) + account := string(parts[1]) + v, err := strconv.ParseInt(string(parts[2]), 10, 64) + if err != nil { + // could not parse + return &abci.ResponseCheckTxV2{ResponseCheckTx: &abci.ResponseCheckTx{ + Priority: priority, + Code: 100, + GasWanted: 1, + }}, nil + } + nonce, err := strconv.ParseInt(string(parts[3]), 10, 64) + if err != nil { + // could not parse + return &abci.ResponseCheckTxV2{ResponseCheckTx: &abci.ResponseCheckTx{ + Priority: priority, + Code: 101, + GasWanted: 1, + }}, nil + } + return &abci.ResponseCheckTxV2{ + ResponseCheckTx: &abci.ResponseCheckTx{ + Priority: v, + Code: code.CodeTypeOK, + GasWanted: 1, + }, + EVMNonce: uint64(nonce), + EVMSenderAddress: account, + IsEVM: true, + }, nil + } + // infer the priority from the raw transaction value (sender=key=value) parts := bytes.Split(req.Tx, []byte("=")) if len(parts) == 3 { @@ -64,7 +100,6 @@ func (app *application) CheckTx(_ context.Context, req *abci.RequestCheckTx) (*a GasWanted: 1, }}, nil } - return &abci.ResponseCheckTxV2{ResponseCheckTx: &abci.ResponseCheckTx{ Priority: priority, Sender: sender, @@ -412,6 +447,63 @@ func TestTxMempool_CheckTxExceedsMaxSize(t *testing.T) { require.NoError(t, txmp.CheckTx(ctx, tx, nil, TxInfo{SenderID: 0})) } +func TestTxMempool_Prioritization(t *testing.T) { + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + client := abciclient.NewLocalClient(log.NewNopLogger(), &application{Application: kvstore.NewApplication()}) + if err := client.Start(ctx); err != nil { + t.Fatal(err) + } + t.Cleanup(client.Wait) + + txmp := setup(t, client, 100) + peerID := uint16(1) + + address1 := "0xeD23B3A9DE15e92B9ef9540E587B3661E15A12fA" + address2 := "0xfD23B3A9DE15e92B9ef9540E587B3661E15A12fA" + + // Generate transactions with different priorities + // there are two formats to comply with the above mocked CheckTX + // EVM: evm-sender=account=priority=nonce + // Non-EVM: sender=peer=priority + txs := [][]byte{ + []byte(fmt.Sprintf("sender-0-1=peer=%d", 9)), + []byte(fmt.Sprintf("sender-1-1=peer=%d", 8)), + []byte(fmt.Sprintf("evm-sender=%s=%d=%d", address1, 7, 0)), + []byte(fmt.Sprintf("evm-sender=%s=%d=%d", address1, 9, 1)), + []byte(fmt.Sprintf("evm-sender=%s=%d=%d", address2, 6, 0)), + []byte(fmt.Sprintf("sender-2-1=peer=%d", 5)), + []byte(fmt.Sprintf("sender-3-1=peer=%d", 4)), + } + + // copy the slice of txs and shuffle the order randomly + txsCopy := make([][]byte, len(txs)) + copy(txsCopy, txs) + rng := rand.New(rand.NewSource(time.Now().UnixNano())) + rng.Shuffle(len(txsCopy), func(i, j int) { + txsCopy[i], txsCopy[j] = txsCopy[j], txsCopy[i] + }) + + for i := range txsCopy { + require.NoError(t, txmp.CheckTx(ctx, txsCopy[i], nil, TxInfo{SenderID: peerID})) + } + + // Reap the transactions + reapedTxs := txmp.ReapMaxTxs(len(txs)) + // Check if the reaped transactions are in the correct order of their priorities + for _, tx := range txs { + fmt.Printf("expected: %s\n", string(tx)) + } + fmt.Println("**************") + for _, reapedTx := range reapedTxs { + fmt.Printf("received: %s\n", string(reapedTx)) + } + for i, reapedTx := range reapedTxs { + require.Equal(t, txs[i], []byte(reapedTx)) + } +} + func TestTxMempool_CheckTxSamePeer(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) defer cancel() diff --git a/internal/mempool/priority_queue.go b/internal/mempool/priority_queue.go index ad3a347a3..3886da0cf 100644 --- a/internal/mempool/priority_queue.go +++ b/internal/mempool/priority_queue.go @@ -12,13 +12,40 @@ var _ heap.Interface = (*TxPriorityQueue)(nil) // TxPriorityQueue defines a thread-safe priority queue for valid transactions. type TxPriorityQueue struct { - mtx sync.RWMutex - txs []*WrappedTx + mtx sync.RWMutex + txs []*WrappedTx + evmQueue map[string][]*WrappedTx +} + +func insertToEVMQueue(queue []*WrappedTx, tx *WrappedTx) []*WrappedTx { + // Using BinarySearch to find the appropriate index to insert tx + i := binarySearch(queue, tx) + + // Make room for new value and add it + queue = append(queue, nil) + copy(queue[i+1:], queue[i:]) + queue[i] = tx + return queue +} + +// binarySearch finds the index at which tx should be inserted in queue +func binarySearch(queue []*WrappedTx, tx *WrappedTx) int { + low, high := 0, len(queue) + for low < high { + mid := low + (high-low)/2 + if queue[mid].evmNonce < tx.evmNonce { + low = mid + 1 + } else { + high = mid + } + } + return low } func NewTxPriorityQueue() *TxPriorityQueue { pq := &TxPriorityQueue{ - txs: make([]*WrappedTx, 0), + txs: make([]*WrappedTx, 0), + evmQueue: make(map[string][]*WrappedTx), } heap.Init(pq) @@ -68,13 +95,46 @@ func (pq *TxPriorityQueue) GetEvictableTxs(priority, txSize, totalSize, cap int6 return nil } +// requires read lock +func (pq *TxPriorityQueue) numQueuedUnsafe() int { + var result int + for _, queue := range pq.evmQueue { + result += len(queue) + } + // first items in queue are also in heap, subtract one + return result - len(pq.evmQueue) +} + // NumTxs returns the number of transactions in the priority queue. It is // thread safe. func (pq *TxPriorityQueue) NumTxs() int { pq.mtx.RLock() defer pq.mtx.RUnlock() - return len(pq.txs) + return len(pq.txs) + pq.numQueuedUnsafe() +} + +func (pq *TxPriorityQueue) removeQueuedEvmTxUnsafe(tx *WrappedTx) { + if queue, ok := pq.evmQueue[tx.evmAddress]; ok { + for i, t := range queue { + if t.evmNonce == tx.evmNonce { + pq.evmQueue[tx.evmAddress] = append(queue[:i], queue[i+1:]...) + if len(pq.evmQueue[tx.evmAddress]) == 0 { + delete(pq.evmQueue, tx.evmAddress) + } + break + } + } + } +} + +func (pq *TxPriorityQueue) findTxIndexUnsafe(tx *WrappedTx) (int, bool) { + for i, t := range pq.txs { + if t == tx { + return i, true + } + } + return 0, false } // RemoveTx removes a specific transaction from the priority queue. @@ -82,30 +142,71 @@ func (pq *TxPriorityQueue) RemoveTx(tx *WrappedTx) { pq.mtx.Lock() defer pq.mtx.Unlock() - if tx.heapIndex < len(pq.txs) { - heap.Remove(pq, tx.heapIndex) + if idx, ok := pq.findTxIndexUnsafe(tx); ok { + heap.Remove(pq, idx) + } + + if tx.isEVM { + pq.removeQueuedEvmTxUnsafe(tx) } } +func (pq *TxPriorityQueue) pushTxUnsafe(tx *WrappedTx) { + if !tx.isEVM { + heap.Push(pq, tx) + return + } + + queue, exists := pq.evmQueue[tx.evmAddress] + if !exists { + pq.evmQueue[tx.evmAddress] = []*WrappedTx{tx} + heap.Push(pq, tx) + return + } + + first := queue[0] + if tx.evmNonce < first.evmNonce { + if idx, ok := pq.findTxIndexUnsafe(first); ok { + heap.Remove(pq, idx) + } + heap.Push(pq, tx) + } + + pq.evmQueue[tx.evmAddress] = insertToEVMQueue(queue, tx) +} + // PushTx adds a valid transaction to the priority queue. It is thread safe. func (pq *TxPriorityQueue) PushTx(tx *WrappedTx) { pq.mtx.Lock() defer pq.mtx.Unlock() + pq.pushTxUnsafe(tx) +} + +func (pq *TxPriorityQueue) popTxUnsafe() *WrappedTx { + x := heap.Pop(pq) + if x == nil { + return nil + } - heap.Push(pq, tx) + tx := x.(*WrappedTx) + + if !tx.isEVM { + return tx + } + + pq.removeQueuedEvmTxUnsafe(tx) + if len(pq.evmQueue[tx.evmAddress]) > 0 { + heap.Push(pq, pq.evmQueue[tx.evmAddress][0]) + } + + return tx } // PopTx removes the top priority transaction from the queue. It is thread safe. func (pq *TxPriorityQueue) PopTx() *WrappedTx { pq.mtx.Lock() defer pq.mtx.Unlock() - - x := heap.Pop(pq) - if x != nil { - return x.(*WrappedTx) - } - - return nil + return pq.popTxUnsafe() } // dequeue up to `max` transactions and reenqueue while locked @@ -113,7 +214,7 @@ func (pq *TxPriorityQueue) PeekTxs(max int) []*WrappedTx { pq.mtx.Lock() defer pq.mtx.Unlock() - numTxs := len(pq.txs) + numTxs := len(pq.txs) + pq.numQueuedUnsafe() if max < 0 { max = numTxs } @@ -121,15 +222,16 @@ func (pq *TxPriorityQueue) PeekTxs(max int) []*WrappedTx { cap := tmmath.MinInt(numTxs, max) res := make([]*WrappedTx, 0, cap) for i := 0; i < cap; i++ { - popped := heap.Pop(pq) + popped := pq.popTxUnsafe() if popped == nil { break } - res = append(res, popped.(*WrappedTx)) + + res = append(res, popped) } for _, tx := range res { - heap.Push(pq, tx) + pq.pushTxUnsafe(tx) } return res } diff --git a/internal/mempool/priority_queue_test.go b/internal/mempool/priority_queue_test.go index ddc84806d..bd7dda126 100644 --- a/internal/mempool/priority_queue_test.go +++ b/internal/mempool/priority_queue_test.go @@ -1,6 +1,7 @@ package mempool import ( + "fmt" "math/rand" "sort" "sync" @@ -10,6 +11,127 @@ import ( "github.com/stretchr/testify/require" ) +// TxTestCase represents a single test case for the TxPriorityQueue +type TxTestCase struct { + name string + inputTxs []*WrappedTx // Input transactions + expectedOutput []int64 // Expected order of transaction IDs +} + +func TestTxPriorityQueue_ReapHalf(t *testing.T) { + pq := NewTxPriorityQueue() + + // Generate transactions with different priorities and nonces + txs := make([]*WrappedTx, 100) + for i := range txs { + txs[i] = &WrappedTx{ + tx: []byte(fmt.Sprintf("tx-%d", i)), + priority: int64(i), + } + + // Push the transaction + pq.PushTx(txs[i]) + } + + //reverse sort txs by priority + sort.Slice(txs, func(i, j int) bool { + return txs[i].priority > txs[j].priority + }) + + // Reap half of the transactions + reapedTxs := pq.PeekTxs(len(txs) / 2) + + // Check if the reaped transactions are in the correct order of their priorities and nonces + for i, reapedTx := range reapedTxs { + require.Equal(t, txs[i].priority, reapedTx.priority) + } +} + +func TestTxPriorityQueue_PriorityAndNonceOrdering(t *testing.T) { + testCases := []TxTestCase{ + { + name: "PriorityWithEVMAndNonEVM", + inputTxs: []*WrappedTx{ + {sender: "1", isEVM: true, evmAddress: "0xabc", evmNonce: 1, priority: 10}, + {sender: "2", isEVM: false, priority: 9}, + {sender: "4", isEVM: true, evmAddress: "0xabc", evmNonce: 0, priority: 9}, // Same EVM address as first, lower nonce + {sender: "5", isEVM: true, evmAddress: "0xdef", evmNonce: 1, priority: 7}, + {sender: "3", isEVM: true, evmAddress: "0xdef", evmNonce: 0, priority: 8}, + {sender: "6", isEVM: false, priority: 6}, + {sender: "7", isEVM: true, evmAddress: "0xghi", evmNonce: 2, priority: 5}, + }, + expectedOutput: []int64{2, 4, 1, 3, 5, 6, 7}, + }, + { + name: "IdenticalPrioritiesAndNoncesDifferentAddresses", + inputTxs: []*WrappedTx{ + {sender: "1", isEVM: true, evmAddress: "0xabc", evmNonce: 2, priority: 5}, + {sender: "2", isEVM: true, evmAddress: "0xdef", evmNonce: 2, priority: 5}, + {sender: "3", isEVM: true, evmAddress: "0xghi", evmNonce: 2, priority: 5}, + }, + expectedOutput: []int64{1, 2, 3}, + }, + { + name: "InterleavedEVAndNonEVMTransactions", + inputTxs: []*WrappedTx{ + {sender: "7", isEVM: false, priority: 15}, + {sender: "8", isEVM: true, evmAddress: "0xabc", evmNonce: 1, priority: 20}, + {sender: "9", isEVM: false, priority: 10}, + {sender: "10", isEVM: true, evmAddress: "0xdef", evmNonce: 2, priority: 20}, + }, + expectedOutput: []int64{8, 10, 7, 9}, + }, + { + name: "SameAddressPriorityDifferentNonces", + inputTxs: []*WrappedTx{ + {sender: "11", isEVM: true, evmAddress: "0xabc", evmNonce: 3, priority: 10}, + {sender: "12", isEVM: true, evmAddress: "0xabc", evmNonce: 1, priority: 10}, + {sender: "13", isEVM: true, evmAddress: "0xabc", evmNonce: 2, priority: 10}, + }, + expectedOutput: []int64{12, 13, 11}, + }, + } + + for _, tc := range testCases { + t.Run(tc.name, func(t *testing.T) { + pq := NewTxPriorityQueue() + now := time.Now() + + // Add input transactions to the queue and set timestamp to order inserted + for i, tx := range tc.inputTxs { + tx.timestamp = now.Add(time.Duration(i) * time.Second) + pq.PushTx(tx) + } + + results := pq.PeekTxs(len(tc.inputTxs)) + // Validate the order of transactions + require.Len(t, results, len(tc.expectedOutput)) + for i, expectedTxID := range tc.expectedOutput { + tx := results[i] + require.Equal(t, fmt.Sprintf("%d", expectedTxID), tx.sender) + } + }) + } +} + +func TestTxPriorityQueue_SameAddressDifferentNonces(t *testing.T) { + pq := NewTxPriorityQueue() + address := "0x123" + + // Insert transactions with the same address but different nonces and priorities + pq.PushTx(&WrappedTx{isEVM: true, evmAddress: address, evmNonce: 2, priority: 10}) + pq.PushTx(&WrappedTx{isEVM: true, evmAddress: address, evmNonce: 1, priority: 5}) + pq.PushTx(&WrappedTx{isEVM: true, evmAddress: address, evmNonce: 3, priority: 15}) + + // Pop transactions and verify they are in the correct order of nonce + tx1 := pq.PopTx() + require.Equal(t, uint64(1), tx1.evmNonce) + tx2 := pq.PopTx() + require.Equal(t, uint64(2), tx2.evmNonce) + tx3 := pq.PopTx() + require.Equal(t, uint64(3), tx3.evmNonce) +} + func TestTxPriorityQueue(t *testing.T) { pq := NewTxPriorityQueue() numTxs := 1000 diff --git a/internal/mempool/tx.go b/internal/mempool/tx.go index be1d9290f..411065db8 100644 --- a/internal/mempool/tx.go +++ b/internal/mempool/tx.go @@ -67,6 +67,11 @@ type WrappedTx struct { // this is the callback that can be called when a transaction expires expiredCallback func(removeFromCache bool) + + // evm properties that aid in prioritization + evmAddress string + evmNonce uint64 + isEVM bool } func (wtx *WrappedTx) Size() int { From 4aa38c03b52c885efeb0132a06b29874d66b5f32 Mon Sep 17 00:00:00 2001 From: Steven Landers Date: Fri, 19 Jan 2024 14:31:54 -0500 Subject: [PATCH 10/21] fix removeTx to push next queued evm tx (#191) --- internal/mempool/priority_queue.go | 5 ++-- internal/mempool/priority_queue_test.go | 32 +++++++++++++++++++++++++ 2 files changed, 34 insertions(+), 3 deletions(-) diff --git a/internal/mempool/priority_queue.go b/internal/mempool/priority_queue.go index 3886da0cf..6906a9717 100644 --- a/internal/mempool/priority_queue.go +++ b/internal/mempool/priority_queue.go @@ -121,6 +121,8 @@ func (pq *TxPriorityQueue) removeQueuedEvmTxUnsafe(tx *WrappedTx) { pq.evmQueue[tx.evmAddress] = append(queue[:i], queue[i+1:]...) if len(pq.evmQueue[tx.evmAddress]) == 0 { delete(pq.evmQueue, tx.evmAddress) + } else { + heap.Push(pq, pq.evmQueue[tx.evmAddress][0]) } break } @@ -195,9 +197,6 @@ func (pq *TxPriorityQueue) popTxUnsafe() *WrappedTx { } pq.removeQueuedEvmTxUnsafe(tx) - if len(pq.evmQueue[tx.evmAddress]) > 0 { - heap.Push(pq, pq.evmQueue[tx.evmAddress][0]) - } return tx } diff --git a/internal/mempool/priority_queue_test.go b/internal/mempool/priority_queue_test.go index bd7dda126..c3bc90853 100644 --- a/internal/mempool/priority_queue_test.go +++ b/internal/mempool/priority_queue_test.go @@ -90,6 +90,13 @@ func TestTxPriorityQueue_PriorityAndNonceOrdering(t *testing.T) { }, expectedOutput: []int64{12, 13, 11}, }, + { + name: "OneItem", + inputTxs: []*WrappedTx{ + {sender: "14", isEVM: true, evmAddress: "0xabc", evmNonce: 1, priority: 10}, + }, + expectedOutput: []int64{14}, + }, } for _, tc := range testCases { @@ -263,6 +270,31 @@ func TestTxPriorityQueue_GetEvictableTxs(t *testing.T) { } } +func TestTxPriorityQueue_RemoveTxEvm(t *testing.T) { + pq := NewTxPriorityQueue() + + tx1 := &WrappedTx{ + priority: 1, + isEVM: true, + evmAddress: "0xabc", + evmNonce: 1, + } + tx2 := &WrappedTx{ + priority: 1, + isEVM: true, + evmAddress: "0xabc", + evmNonce: 2, + } + + pq.PushTx(tx1) + pq.PushTx(tx2) + + pq.RemoveTx(tx1) + + result := pq.PopTx() + require.Equal(t, tx2, result) +} + func TestTxPriorityQueue_RemoveTx(t *testing.T) { pq := NewTxPriorityQueue() rng := rand.New(rand.NewSource(time.Now().UnixNano())) From b730c59a1ce4b7695467162a362bef9c90d9d75c Mon Sep 17 00:00:00 2001 From: Steven Landers Date: Wed, 24 Jan 2024 13:51:56 -0500 Subject: [PATCH 11/21] fix expire metric (#193) --- internal/mempool/mempool.go | 9 +++++---- internal/mempool/tx.go | 4 ++-- 2 files changed, 7 insertions(+), 6 deletions(-) diff --git a/internal/mempool/mempool.go b/internal/mempool/mempool.go index 7bccb5479..a00883b05 100644 --- a/internal/mempool/mempool.go +++ b/internal/mempool/mempool.go @@ -299,8 +299,7 @@ func (txmp *TxMempool) CheckTx( evmNonce: res.EVMNonce, evmAddress: res.EVMSenderAddress, isEVM: res.IsEVM, - expiredCallback: func(removeFromCache bool) { - txmp.metrics.ExpiredTxs.Add(1) + removeHandler: func(removeFromCache bool) { if removeFromCache { txmp.cache.Remove(tx) } @@ -856,7 +855,7 @@ func (txmp *TxMempool) removeTx(wtx *WrappedTx, removeFromCache bool) { atomic.AddInt64(&txmp.sizeBytes, int64(-wtx.Size())) - wtx.expiredCallback(removeFromCache) + wtx.removeHandler(removeFromCache) } // purgeExpiredTxs removes all transactions that have exceeded their respective @@ -905,12 +904,14 @@ func (txmp *TxMempool) purgeExpiredTxs(blockHeight int64) { } for _, wtx := range expiredTxs { + txmp.metrics.ExpiredTxs.Add(1) txmp.removeTx(wtx, !txmp.config.KeepInvalidTxsInCache) } // remove pending txs that have expired txmp.pendingTxs.PurgeExpired(txmp.config.TTLNumBlocks, blockHeight, txmp.config.TTLDuration, now, func(wtx *WrappedTx) { - wtx.expiredCallback(!txmp.config.KeepInvalidTxsInCache) + txmp.metrics.ExpiredTxs.Add(1) + wtx.removeHandler(!txmp.config.KeepInvalidTxsInCache) }) } diff --git a/internal/mempool/tx.go b/internal/mempool/tx.go index 411065db8..025cfda73 100644 --- a/internal/mempool/tx.go +++ b/internal/mempool/tx.go @@ -65,8 +65,8 @@ type WrappedTx struct { // a reCheckTx callback executed. removed bool - // this is the callback that can be called when a transaction expires - expiredCallback func(removeFromCache bool) + // this is the callback that can be called when a transaction is removed + removeHandler func(removeFromCache bool) // evm properties that aid in prioritization evmAddress string From d359eccb3597814b2e39eae96b44c1d42c9b3901 Mon Sep 17 00:00:00 2001 From: Steven Landers Date: Thu, 25 Jan 2024 11:36:08 -0500 Subject: [PATCH 12/21] [EVM] Fix duplicate evm txs from priority queue (#195) * debug duplicate evm tx * add more logs * add some \ns * more logs * fix swap check * add-lockable-reap-by-gas * add invariant checks * fix invariant parenthesis * fix log * remove invalid invariant * fix nonce ordering pain * handle ordering of insert * fix remove * cleanup * fix imports * cleanup * avoid getTransactionByHash(hash) panic due to index * use Key() to compare instead of pointer --- internal/mempool/mempool.go | 30 ++----- internal/mempool/priority_queue.go | 106 +++++++++++++++++++++--- internal/mempool/priority_queue_test.go | 34 +++++++- internal/rpc/core/mempool.go | 3 + types/tx.go | 2 +- 5 files changed, 139 insertions(+), 36 deletions(-) diff --git a/internal/mempool/mempool.go b/internal/mempool/mempool.go index a00883b05..9bed03a8a 100644 --- a/internal/mempool/mempool.go +++ b/internal/mempool/mempool.go @@ -399,42 +399,28 @@ func (txmp *TxMempool) ReapMaxBytesMaxGas(maxBytes, maxGas int64) types.Txs { totalSize int64 ) - // wTxs contains a list of *WrappedTx retrieved from the priority queue that - // need to be re-enqueued prior to returning. - wTxs := make([]*WrappedTx, 0, txmp.priorityIndex.NumTxs()) - defer func() { - for _, wtx := range wTxs { - txmp.priorityIndex.PushTx(wtx) - } - }() - - txs := make([]types.Tx, 0, txmp.priorityIndex.NumTxs()) + var txs []types.Tx if uint64(txmp.Size()) < txmp.config.TxNotifyThreshold { // do not reap anything if threshold is not met return txs } - for txmp.priorityIndex.NumTxs() > 0 { - wtx := txmp.priorityIndex.PopTx() - txs = append(txs, wtx.tx) - wTxs = append(wTxs, wtx) + txmp.priorityIndex.ForEachTx(func(wtx *WrappedTx) bool { size := types.ComputeProtoSizeForTxs([]types.Tx{wtx.tx}) - // Ensure we have capacity for the transaction with respect to the - // transaction size. if maxBytes > -1 && totalSize+size > maxBytes { - return txs[:len(txs)-1] + return false } - totalSize += size - - // ensure we have capacity for the transaction with respect to total gas gas := totalGas + wtx.gasWanted if maxGas > -1 && gas > maxGas { - return txs[:len(txs)-1] + return false } totalGas = gas - } + + txs = append(txs, wtx.tx) + return true + }) return txs } diff --git a/internal/mempool/priority_queue.go b/internal/mempool/priority_queue.go index 6906a9717..6dbbfe9b2 100644 --- a/internal/mempool/priority_queue.go +++ b/internal/mempool/priority_queue.go @@ -13,14 +13,11 @@ var _ heap.Interface = (*TxPriorityQueue)(nil) // TxPriorityQueue defines a thread-safe priority queue for valid transactions. type TxPriorityQueue struct { mtx sync.RWMutex - txs []*WrappedTx - evmQueue map[string][]*WrappedTx + txs []*WrappedTx // priority heap + evmQueue map[string][]*WrappedTx // sorted by nonce } -func insertToEVMQueue(queue []*WrappedTx, tx *WrappedTx) []*WrappedTx { - // Using BinarySearch to find the appropriate index to insert tx - i := binarySearch(queue, tx) - +func insertToEVMQueue(queue []*WrappedTx, tx *WrappedTx, i int) []*WrappedTx { // Make room for new value and add it queue = append(queue, nil) copy(queue[i+1:], queue[i:]) @@ -33,7 +30,7 @@ func binarySearch(queue []*WrappedTx, tx *WrappedTx) int { low, high := 0, len(queue) for low < high { mid := low + (high-low)/2 - if queue[mid].evmNonce < tx.evmNonce { + if queue[mid].evmNonce <= tx.evmNonce { low = mid + 1 } else { high = mid @@ -117,12 +114,15 @@ func (pq *TxPriorityQueue) NumTxs() int { func (pq *TxPriorityQueue) removeQueuedEvmTxUnsafe(tx *WrappedTx) { if queue, ok := pq.evmQueue[tx.evmAddress]; ok { for i, t := range queue { - if t.evmNonce == tx.evmNonce { + if t.tx.Key() == tx.tx.Key() { pq.evmQueue[tx.evmAddress] = append(queue[:i], queue[i+1:]...) if len(pq.evmQueue[tx.evmAddress]) == 0 { delete(pq.evmQueue, tx.evmAddress) } else { - heap.Push(pq, pq.evmQueue[tx.evmAddress][0]) + // only if removing the first item, then push next onto queue + if i == 0 { + heap.Push(pq, pq.evmQueue[tx.evmAddress][0]) + } } break } @@ -174,9 +174,67 @@ func (pq *TxPriorityQueue) pushTxUnsafe(tx *WrappedTx) { heap.Push(pq, tx) } - pq.evmQueue[tx.evmAddress] = insertToEVMQueue(queue, tx) + pq.evmQueue[tx.evmAddress] = insertToEVMQueue(queue, tx, binarySearch(queue, tx)) + } +// These are available if we need to test the invariant checks +// these can be used to troubleshoot invariant violations +//func (pq *TxPriorityQueue) checkInvariants(msg string) { +// +// uniqHashes := make(map[string]bool) +// for _, tx := range pq.txs { +// if _, ok := uniqHashes[fmt.Sprintf("%x", tx.tx.Key())]; ok { +// pq.print() +// panic(fmt.Sprintf("INVARIANT (%s): duplicate hash=%x in heap", msg, tx.tx.Key())) +// } +// uniqHashes[fmt.Sprintf("%x", tx.tx.Key())] = true +// if tx.isEVM { +// if queue, ok := pq.evmQueue[tx.evmAddress]; ok { +// if queue[0].tx.Key() != tx.tx.Key() { +// pq.print() +// panic(fmt.Sprintf("INVARIANT (%s): tx in heap but not at front of evmQueue hash=%x", msg, tx.tx.Key())) +// } +// } else { +// pq.print() +// panic(fmt.Sprintf("INVARIANT (%s): tx in heap but not in evmQueue hash=%x", msg, tx.tx.Key())) +// } +// } +// } +// +// // each item in all queues should be unique nonce +// for _, queue := range pq.evmQueue { +// hashes := make(map[string]bool) +// for idx, tx := range queue { +// if idx == 0 { +// _, ok := pq.findTxIndexUnsafe(tx) +// if !ok { +// pq.print() +// panic(fmt.Sprintf("INVARIANT (%s): did not find tx[0] hash=%x nonce=%d in heap", msg, tx.tx.Key(), tx.evmNonce)) +// } +// } +// if _, ok := hashes[fmt.Sprintf("%x", tx.tx.Key())]; ok { +// pq.print() +// panic(fmt.Sprintf("INVARIANT (%s): duplicate hash=%x in queue nonce=%d", msg, tx.tx.Key(), tx.evmNonce)) +// } +// hashes[fmt.Sprintf("%x", tx.tx.Key())] = true +// } +// } +//} + +// for debugging situations where invariant violations occur +//func (pq *TxPriorityQueue) print() { +// for _, tx := range pq.txs { +// fmt.Printf("DEBUG PRINT: heap: nonce=%d, hash=%x\n", tx.evmNonce, tx.tx.Key()) +// } +// +// for _, queue := range pq.evmQueue { +// for idx, tx := range queue { +// fmt.Printf("DEBUG PRINT: evmQueue[%d]: nonce=%d, hash=%x\n", idx, tx.evmNonce, tx.tx.Key()) +// } +// } +//} + // PushTx adds a valid transaction to the priority queue. It is thread safe. func (pq *TxPriorityQueue) PushTx(tx *WrappedTx) { pq.mtx.Lock() @@ -185,6 +243,9 @@ func (pq *TxPriorityQueue) PushTx(tx *WrappedTx) { } func (pq *TxPriorityQueue) popTxUnsafe() *WrappedTx { + if len(pq.txs) == 0 { + return nil + } x := heap.Pop(pq) if x == nil { return nil @@ -209,6 +270,31 @@ func (pq *TxPriorityQueue) PopTx() *WrappedTx { } // dequeue up to `max` transactions and reenqueue while locked +func (pq *TxPriorityQueue) ForEachTx(handler func(wtx *WrappedTx) bool) { + pq.mtx.Lock() + defer pq.mtx.Unlock() + + numTxs := len(pq.txs) + pq.numQueuedUnsafe() + + txs := make([]*WrappedTx, 0, numTxs) + + defer func() { + for _, tx := range txs { + pq.pushTxUnsafe(tx) + } + }() + + for i := 0; i < numTxs; i++ { + popped := pq.popTxUnsafe() + txs = append(txs, popped) + if !handler(popped) { + return + } + } +} + +// dequeue up to `max` transactions and reenqueue while locked +// TODO: use ForEachTx instead func (pq *TxPriorityQueue) PeekTxs(max int) []*WrappedTx { pq.mtx.Lock() defer pq.mtx.Unlock() diff --git a/internal/mempool/priority_queue_test.go b/internal/mempool/priority_queue_test.go index c3bc90853..c1e17d278 100644 --- a/internal/mempool/priority_queue_test.go +++ b/internal/mempool/priority_queue_test.go @@ -49,6 +49,29 @@ func TestTxPriorityQueue_ReapHalf(t *testing.T) { func TestTxPriorityQueue_PriorityAndNonceOrdering(t *testing.T) { testCases := []TxTestCase{ + { + name: "PriorityWithEVMAndNonEVMDuplicateNonce", + inputTxs: []*WrappedTx{ + {sender: "1", isEVM: true, evmAddress: "0xabc", evmNonce: 1, priority: 10}, + {sender: "3", isEVM: true, evmAddress: "0xabc", evmNonce: 3, priority: 9}, + {sender: "2", isEVM: true, evmAddress: "0xabc", evmNonce: 1, priority: 7}, + }, + expectedOutput: []int64{1, 2, 3}, + }, + { + name: "PriorityWithEVMAndNonEVMDuplicateNonce", + inputTxs: []*WrappedTx{ + {sender: "1", isEVM: true, evmAddress: "0xabc", evmNonce: 1, priority: 10}, + {sender: "2", isEVM: false, priority: 9}, + {sender: "4", isEVM: true, evmAddress: "0xabc", evmNonce: 0, priority: 9}, // Same EVM address as first, lower nonce + {sender: "5", isEVM: true, evmAddress: "0xdef", evmNonce: 1, priority: 7}, + {sender: "5", isEVM: true, evmAddress: "0xdef", evmNonce: 1, priority: 7}, + {sender: "3", isEVM: true, evmAddress: "0xdef", evmNonce: 0, priority: 8}, + {sender: "6", isEVM: false, priority: 6}, + {sender: "7", isEVM: true, evmAddress: "0xghi", evmNonce: 2, priority: 5}, + }, + expectedOutput: []int64{2, 4, 1, 3, 5, 5, 6, 7}, + }, { name: "PriorityWithEVMAndNonEVM", inputTxs: []*WrappedTx{ @@ -107,6 +130,7 @@ func TestTxPriorityQueue_PriorityAndNonceOrdering(t *testing.T) { // Add input transactions to the queue and set timestamp to order inserted for i, tx := range tc.inputTxs { tx.timestamp = now.Add(time.Duration(i) * time.Second) + tx.tx = []byte(fmt.Sprintf("%d", time.Now().UnixNano())) pq.PushTx(tx) } @@ -126,9 +150,9 @@ func TestTxPriorityQueue_SameAddressDifferentNonces(t *testing.T) { address := "0x123" // Insert transactions with the same address but different nonces and priorities - pq.PushTx(&WrappedTx{isEVM: true, evmAddress: address, evmNonce: 2, priority: 10}) - pq.PushTx(&WrappedTx{isEVM: true, evmAddress: address, evmNonce: 1, priority: 5}) - pq.PushTx(&WrappedTx{isEVM: true, evmAddress: address, evmNonce: 3, priority: 15}) + pq.PushTx(&WrappedTx{isEVM: true, evmAddress: address, evmNonce: 2, priority: 10, tx: []byte("tx1")}) + pq.PushTx(&WrappedTx{isEVM: true, evmAddress: address, evmNonce: 1, priority: 5, tx: []byte("tx2")}) + pq.PushTx(&WrappedTx{isEVM: true, evmAddress: address, evmNonce: 3, priority: 15, tx: []byte("tx3")}) // Pop transactions and verify they are in the correct order of nonce tx1 := pq.PopTx() @@ -154,6 +178,7 @@ func TestTxPriorityQueue(t *testing.T) { pq.PushTx(&WrappedTx{ priority: int64(i), timestamp: time.Now(), + tx: []byte(fmt.Sprintf("%d", i)), }) wg.Done() @@ -278,12 +303,14 @@ func TestTxPriorityQueue_RemoveTxEvm(t *testing.T) { isEVM: true, evmAddress: "0xabc", evmNonce: 1, + tx: []byte("tx1"), } tx2 := &WrappedTx{ priority: 1, isEVM: true, evmAddress: "0xabc", evmNonce: 2, + tx: []byte("tx2"), } pq.PushTx(tx1) @@ -306,6 +333,7 @@ func TestTxPriorityQueue_RemoveTx(t *testing.T) { x := rng.Intn(100000) pq.PushTx(&WrappedTx{ priority: int64(x), + tx: []byte(fmt.Sprintf("%d", i)), }) values[i] = x diff --git a/internal/rpc/core/mempool.go b/internal/rpc/core/mempool.go index 917e00b47..c351c5ba2 100644 --- a/internal/rpc/core/mempool.go +++ b/internal/rpc/core/mempool.go @@ -158,6 +158,9 @@ func (env *Environment) UnconfirmedTxs(ctx context.Context, req *coretypes.Reque skipCount := validateSkipCount(page, perPage) txs := env.Mempool.ReapMaxTxs(skipCount + tmmath.MinInt(perPage, totalCount-skipCount)) + if skipCount > len(txs) { + skipCount = len(txs) + } result := txs[skipCount:] return &coretypes.ResultUnconfirmedTxs{ diff --git a/types/tx.go b/types/tx.go index 0a37d1e9d..5b7c2377e 100644 --- a/types/tx.go +++ b/types/tx.go @@ -184,7 +184,7 @@ func (t TxRecordSet) Validate(maxSizeBytes int64, otxs Txs) error { for i, cur := range allCopy { // allCopy is sorted, so any duplicated data will be adjacent. if i+1 < len(allCopy) && bytes.Equal(cur, allCopy[i+1]) { - return fmt.Errorf("found duplicate transaction with hash: %x", cur.Hash()) + return fmt.Errorf("found duplicate transaction with hash: %x", cur.Key()) } } From cf41dabfee56546f92794bb5ffe5361d04b2f298 Mon Sep 17 00:00:00 2001 From: Steven Landers Date: Mon, 29 Jan 2024 02:29:17 -0500 Subject: [PATCH 13/21] [EVM] prevent duplicate txs from getting inserted (#196) * prevent duplicates in mempool * use timestamp in priority queue --- internal/consensus/mempool_test.go | 5 +- internal/mempool/mempool.go | 32 ++++++--- internal/mempool/priority_queue.go | 90 +++++++++++++++++++------ internal/mempool/priority_queue_test.go | 1 + internal/mempool/tx.go | 6 ++ 5 files changed, 102 insertions(+), 32 deletions(-) diff --git a/internal/consensus/mempool_test.go b/internal/consensus/mempool_test.go index 84badbeeb..6ef3849ad 100644 --- a/internal/consensus/mempool_test.go +++ b/internal/consensus/mempool_test.go @@ -139,7 +139,7 @@ func checkTxsRange(ctx context.Context, t *testing.T, cs *State, start, end int) var rCode uint32 err := assertMempool(t, cs.txNotifier).CheckTx(ctx, txBytes, func(r *abci.ResponseCheckTx) { rCode = r.Code }, mempool.TxInfo{}) require.NoError(t, err, "error after checkTx") - require.Equal(t, code.CodeTypeOK, rCode, "checkTx code is error, txBytes %X", txBytes) + require.Equal(t, code.CodeTypeOK, rCode, "checkTx code is error, txBytes %X, index=%d", txBytes, i) } } @@ -166,7 +166,7 @@ func TestMempoolTxConcurrentWithCommit(t *testing.T) { require.NoError(t, err) newBlockHeaderCh := subscribe(ctx, t, cs.eventBus, types.EventQueryNewBlockHeader) - const numTxs int64 = 100 + const numTxs int64 = 50 go checkTxsRange(ctx, t, cs, 0, int(numTxs)) startTestRound(ctx, cs, cs.roundState.Height(), cs.roundState.Round()) @@ -331,7 +331,6 @@ func txAsUint64(tx []byte) uint64 { func (app *CounterApplication) Commit(context.Context) (*abci.ResponseCommit, error) { app.mu.Lock() defer app.mu.Unlock() - app.mempoolTxCount = app.txCount return &abci.ResponseCommit{}, nil } diff --git a/internal/mempool/mempool.go b/internal/mempool/mempool.go index 9bed03a8a..612a0bc48 100644 --- a/internal/mempool/mempool.go +++ b/internal/mempool/mempool.go @@ -332,6 +332,11 @@ func (txmp *TxMempool) CheckTx( return nil } +func (txmp *TxMempool) isInMempool(tx types.Tx) bool { + existingTx := txmp.txStore.GetTxByHash(tx.Key()) + return existingTx != nil && !existingTx.removed +} + func (txmp *TxMempool) RemoveTxByKey(txKey types.TxKey) error { txmp.Lock() defer txmp.Unlock() @@ -635,15 +640,17 @@ func (txmp *TxMempool) addNewTransaction(wtx *WrappedTx, res *abci.ResponseCheck txmp.metrics.Size.Set(float64(txmp.Size())) txmp.metrics.PendingSize.Set(float64(txmp.PendingSize())) - txmp.insertTx(wtx) - txmp.logger.Debug( - "inserted good transaction", - "priority", wtx.priority, - "tx", fmt.Sprintf("%X", wtx.tx.Hash()), - "height", txmp.height, - "num_txs", txmp.Size(), - ) - txmp.notifyTxsAvailable() + if txmp.insertTx(wtx) { + txmp.logger.Debug( + "inserted good transaction", + "priority", wtx.priority, + "tx", fmt.Sprintf("%X", wtx.tx.Hash()), + "height", txmp.height, + "num_txs", txmp.Size(), + ) + txmp.notifyTxsAvailable() + } + return nil } @@ -809,7 +816,11 @@ func (txmp *TxMempool) canAddTx(wtx *WrappedTx) error { return nil } -func (txmp *TxMempool) insertTx(wtx *WrappedTx) { +func (txmp *TxMempool) insertTx(wtx *WrappedTx) bool { + if txmp.isInMempool(wtx.tx) { + return false + } + txmp.txStore.SetTx(wtx) txmp.priorityIndex.PushTx(wtx) txmp.heightIndex.Insert(wtx) @@ -822,6 +833,7 @@ func (txmp *TxMempool) insertTx(wtx *WrappedTx) { wtx.gossipEl = gossipEl atomic.AddInt64(&txmp.sizeBytes, int64(wtx.Size())) + return true } func (txmp *TxMempool) removeTx(wtx *WrappedTx, removeFromCache bool) { diff --git a/internal/mempool/priority_queue.go b/internal/mempool/priority_queue.go index 6dbbfe9b2..3de4e810f 100644 --- a/internal/mempool/priority_queue.go +++ b/internal/mempool/priority_queue.go @@ -30,7 +30,7 @@ func binarySearch(queue []*WrappedTx, tx *WrappedTx) int { low, high := 0, len(queue) for low < high { mid := low + (high-low)/2 - if queue[mid].evmNonce <= tx.evmNonce { + if queue[mid].IsBefore(tx) { low = mid + 1 } else { high = mid @@ -118,11 +118,6 @@ func (pq *TxPriorityQueue) removeQueuedEvmTxUnsafe(tx *WrappedTx) { pq.evmQueue[tx.evmAddress] = append(queue[:i], queue[i+1:]...) if len(pq.evmQueue[tx.evmAddress]) == 0 { delete(pq.evmQueue, tx.evmAddress) - } else { - // only if removing the first item, then push next onto queue - if i == 0 { - heap.Push(pq, pq.evmQueue[tx.evmAddress][0]) - } } break } @@ -132,7 +127,7 @@ func (pq *TxPriorityQueue) removeQueuedEvmTxUnsafe(tx *WrappedTx) { func (pq *TxPriorityQueue) findTxIndexUnsafe(tx *WrappedTx) (int, bool) { for i, t := range pq.txs { - if t == tx { + if t.tx.Key() == tx.tx.Key() { return i, true } } @@ -146,9 +141,13 @@ func (pq *TxPriorityQueue) RemoveTx(tx *WrappedTx) { if idx, ok := pq.findTxIndexUnsafe(tx); ok { heap.Remove(pq, idx) - } - - if tx.isEVM { + if tx.isEVM { + pq.removeQueuedEvmTxUnsafe(tx) + if len(pq.evmQueue[tx.evmAddress]) > 0 { + heap.Push(pq, pq.evmQueue[tx.evmAddress][0]) + } + } + } else if tx.isEVM { pq.removeQueuedEvmTxUnsafe(tx) } } @@ -159,6 +158,7 @@ func (pq *TxPriorityQueue) pushTxUnsafe(tx *WrappedTx) { return } + // if there aren't other waiting txs, init and return queue, exists := pq.evmQueue[tx.evmAddress] if !exists { pq.evmQueue[tx.evmAddress] = []*WrappedTx{tx} @@ -166,29 +166,45 @@ func (pq *TxPriorityQueue) pushTxUnsafe(tx *WrappedTx) { return } + // this item is on the heap at the moment first := queue[0] - if tx.evmNonce < first.evmNonce { + + // the queue's first item (and ONLY the first item) must be on the heap + // if this tx is before the first item, then we need to remove the first + // item from the heap + if tx.IsBefore(first) { if idx, ok := pq.findTxIndexUnsafe(first); ok { heap.Remove(pq, idx) } heap.Push(pq, tx) } - pq.evmQueue[tx.evmAddress] = insertToEVMQueue(queue, tx, binarySearch(queue, tx)) - } // These are available if we need to test the invariant checks // these can be used to troubleshoot invariant violations //func (pq *TxPriorityQueue) checkInvariants(msg string) { -// // uniqHashes := make(map[string]bool) -// for _, tx := range pq.txs { +// for idx, tx := range pq.txs { +// if tx == nil { +// pq.print() +// panic(fmt.Sprintf("DEBUG PRINT: found nil item on heap: idx=%d\n", idx)) +// } +// if tx.tx == nil { +// pq.print() +// panic(fmt.Sprintf("DEBUG PRINT: found nil tx.tx on heap: idx=%d\n", idx)) +// } // if _, ok := uniqHashes[fmt.Sprintf("%x", tx.tx.Key())]; ok { // pq.print() // panic(fmt.Sprintf("INVARIANT (%s): duplicate hash=%x in heap", msg, tx.tx.Key())) // } // uniqHashes[fmt.Sprintf("%x", tx.tx.Key())] = true +// +// //if _, ok := pq.keys[tx.tx.Key()]; !ok { +// // pq.print() +// // panic(fmt.Sprintf("INVARIANT (%s): tx in heap but not in keys hash=%x", msg, tx.tx.Key())) +// //} +// // if tx.isEVM { // if queue, ok := pq.evmQueue[tx.evmAddress]; ok { // if queue[0].tx.Key() != tx.tx.Key() { @@ -213,6 +229,10 @@ func (pq *TxPriorityQueue) pushTxUnsafe(tx *WrappedTx) { // panic(fmt.Sprintf("INVARIANT (%s): did not find tx[0] hash=%x nonce=%d in heap", msg, tx.tx.Key(), tx.evmNonce)) // } // } +// //if _, ok := pq.keys[tx.tx.Key()]; !ok { +// // pq.print() +// // panic(fmt.Sprintf("INVARIANT (%s): tx in heap but not in keys hash=%x", msg, tx.tx.Key())) +// //} // if _, ok := hashes[fmt.Sprintf("%x", tx.tx.Key())]; ok { // pq.print() // panic(fmt.Sprintf("INVARIANT (%s): duplicate hash=%x in queue nonce=%d", msg, tx.tx.Key(), tx.evmNonce)) @@ -224,13 +244,31 @@ func (pq *TxPriorityQueue) pushTxUnsafe(tx *WrappedTx) { // for debugging situations where invariant violations occur //func (pq *TxPriorityQueue) print() { +// fmt.Println("PRINT PRIORITY QUEUE ****************** ") // for _, tx := range pq.txs { -// fmt.Printf("DEBUG PRINT: heap: nonce=%d, hash=%x\n", tx.evmNonce, tx.tx.Key()) +// if tx == nil { +// fmt.Printf("DEBUG PRINT: heap (nil): nonce=?, hash=?\n") +// continue +// } +// if tx.tx == nil { +// fmt.Printf("DEBUG PRINT: heap (%s): nonce=%d, tx.tx is nil \n", tx.evmAddress, tx.evmNonce) +// continue +// } +// fmt.Printf("DEBUG PRINT: heap (%s): nonce=%d, hash=%x, time=%d\n", tx.evmAddress, tx.evmNonce, tx.tx.Key(), tx.timestamp.UnixNano()) // } // -// for _, queue := range pq.evmQueue { +// for addr, queue := range pq.evmQueue { // for idx, tx := range queue { -// fmt.Printf("DEBUG PRINT: evmQueue[%d]: nonce=%d, hash=%x\n", idx, tx.evmNonce, tx.tx.Key()) +// if tx == nil { +// fmt.Printf("DEBUG PRINT: found nil item on evmQueue(%s): idx=%d\n", addr, idx) +// continue +// } +// if tx.tx == nil { +// fmt.Printf("DEBUG PRINT: found nil tx.tx on evmQueue(%s): idx=%d\n", addr, idx) +// continue +// } +// +// fmt.Printf("DEBUG PRINT: evmQueue(%s)[%d]: nonce=%d, hash=%x, time=%d\n", tx.evmAddress, idx, tx.evmNonce, tx.tx.Key(), tx.timestamp.UnixNano()) // } // } //} @@ -239,6 +277,7 @@ func (pq *TxPriorityQueue) pushTxUnsafe(tx *WrappedTx) { func (pq *TxPriorityQueue) PushTx(tx *WrappedTx) { pq.mtx.Lock() defer pq.mtx.Unlock() + pq.pushTxUnsafe(tx) } @@ -246,19 +285,31 @@ func (pq *TxPriorityQueue) popTxUnsafe() *WrappedTx { if len(pq.txs) == 0 { return nil } + + // remove the first item from the heap x := heap.Pop(pq) if x == nil { return nil } - tx := x.(*WrappedTx) + // non-evm transactions do not have txs waiting on a nonce if !tx.isEVM { return tx } + // evm transactions can have txs waiting on this nonce + // if there are any, we should replace the heap with the next nonce + // for the address + + // remove the first item from the evmQueue pq.removeQueuedEvmTxUnsafe(tx) + // if there is a next item, now it can be added to the heap + if len(pq.evmQueue[tx.evmAddress]) > 0 { + heap.Push(pq, pq.evmQueue[tx.evmAddress][0]) + } + return tx } @@ -266,6 +317,7 @@ func (pq *TxPriorityQueue) popTxUnsafe() *WrappedTx { func (pq *TxPriorityQueue) PopTx() *WrappedTx { pq.mtx.Lock() defer pq.mtx.Unlock() + return pq.popTxUnsafe() } diff --git a/internal/mempool/priority_queue_test.go b/internal/mempool/priority_queue_test.go index c1e17d278..8cb6d2e1a 100644 --- a/internal/mempool/priority_queue_test.go +++ b/internal/mempool/priority_queue_test.go @@ -196,6 +196,7 @@ func TestTxPriorityQueue(t *testing.T) { pq.PushTx(&WrappedTx{ priority: 1000, timestamp: now, + tx: []byte(fmt.Sprintf("%d", time.Now().UnixNano())), }) require.Equal(t, 1001, pq.NumTxs()) diff --git a/internal/mempool/tx.go b/internal/mempool/tx.go index 025cfda73..13ddb0a12 100644 --- a/internal/mempool/tx.go +++ b/internal/mempool/tx.go @@ -74,6 +74,12 @@ type WrappedTx struct { isEVM bool } +// IsBefore returns true if the WrappedTx is before the given WrappedTx +// this applies to EVM transactions only +func (wtx *WrappedTx) IsBefore(tx *WrappedTx) bool { + return wtx.evmNonce < tx.evmNonce || (wtx.evmNonce == tx.evmNonce && wtx.timestamp.Before(tx.timestamp)) +} + func (wtx *WrappedTx) Size() int { return len(wtx.tx) } From 9f115c744974edc72b9ee20f9ad29d4a0a04c066 Mon Sep 17 00:00:00 2001 From: Steven Landers Date: Mon, 29 Jan 2024 17:33:31 -0500 Subject: [PATCH 14/21] [EVM] Add logging for expiration (#198) * add logging for expired txs * cleanup --- internal/mempool/mempool.go | 32 ++++++++++++++++++++++++++++---- 1 file changed, 28 insertions(+), 4 deletions(-) diff --git a/internal/mempool/mempool.go b/internal/mempool/mempool.go index 612a0bc48..df30dc7c0 100644 --- a/internal/mempool/mempool.go +++ b/internal/mempool/mempool.go @@ -856,6 +856,32 @@ func (txmp *TxMempool) removeTx(wtx *WrappedTx, removeFromCache bool) { wtx.removeHandler(removeFromCache) } +func (txmp *TxMempool) expire(blockHeight int64, wtx *WrappedTx) { + txmp.metrics.ExpiredTxs.Add(1) + txmp.logExpiredTx(blockHeight, wtx) + wtx.removeHandler(!txmp.config.KeepInvalidTxsInCache) +} + +func (txmp *TxMempool) logExpiredTx(blockHeight int64, wtx *WrappedTx) { + // defensive check + if wtx == nil { + return + } + + txmp.logger.Info( + "transaction expired", + "priority", wtx.priority, + "tx", fmt.Sprintf("%X", wtx.tx.Hash()), + "address", wtx.evmAddress, + "evm", wtx.isEVM, + "nonce", wtx.evmNonce, + "height", blockHeight, + "tx_height", wtx.height, + "tx_timestamp", wtx.timestamp, + "age", time.Since(wtx.timestamp), + ) +} + // purgeExpiredTxs removes all transactions that have exceeded their respective // height- and/or time-based TTLs from their respective indexes. Every expired // transaction will be removed from the mempool, but preserved in the cache (except for pending txs). @@ -902,14 +928,12 @@ func (txmp *TxMempool) purgeExpiredTxs(blockHeight int64) { } for _, wtx := range expiredTxs { - txmp.metrics.ExpiredTxs.Add(1) - txmp.removeTx(wtx, !txmp.config.KeepInvalidTxsInCache) + txmp.expire(blockHeight, wtx) } // remove pending txs that have expired txmp.pendingTxs.PurgeExpired(txmp.config.TTLNumBlocks, blockHeight, txmp.config.TTLDuration, now, func(wtx *WrappedTx) { - txmp.metrics.ExpiredTxs.Add(1) - wtx.removeHandler(!txmp.config.KeepInvalidTxsInCache) + txmp.expire(blockHeight, wtx) }) } From 585bed3cfe70096b90e3b61ffa31e2ad8ed0d0d9 Mon Sep 17 00:00:00 2001 From: Steven Landers Date: Mon, 29 Jan 2024 18:07:03 -0500 Subject: [PATCH 15/21] [EVM] Avoid returning nil transactions on ForEach (#197) * remove heapIndex to avoid nil scenario * avoid returning nil in loop (mimic Peek) --- internal/mempool/priority_queue.go | 15 +++++++++------ internal/mempool/priority_queue_test.go | 14 ++++++++++++++ 2 files changed, 23 insertions(+), 6 deletions(-) diff --git a/internal/mempool/priority_queue.go b/internal/mempool/priority_queue.go index 3de4e810f..abcbfe6a7 100644 --- a/internal/mempool/priority_queue.go +++ b/internal/mempool/priority_queue.go @@ -293,6 +293,11 @@ func (pq *TxPriorityQueue) popTxUnsafe() *WrappedTx { } tx := x.(*WrappedTx) + // this situation is primarily for a test case that inserts nils + if tx == nil { + return nil + } + // non-evm transactions do not have txs waiting on a nonce if !tx.isEVM { return tx @@ -338,6 +343,9 @@ func (pq *TxPriorityQueue) ForEachTx(handler func(wtx *WrappedTx) bool) { for i := 0; i < numTxs; i++ { popped := pq.popTxUnsafe() + if popped == nil { + break + } txs = append(txs, popped) if !handler(popped) { return @@ -377,9 +385,7 @@ func (pq *TxPriorityQueue) PeekTxs(max int) []*WrappedTx { // // NOTE: A caller should never call Push. Use PushTx instead. func (pq *TxPriorityQueue) Push(x interface{}) { - n := len(pq.txs) item := x.(*WrappedTx) - item.heapIndex = n pq.txs = append(pq.txs, item) } @@ -390,8 +396,7 @@ func (pq *TxPriorityQueue) Pop() interface{} { old := pq.txs n := len(old) item := old[n-1] - old[n-1] = nil // avoid memory leak - item.heapIndex = -1 // for safety + old[n-1] = nil // avoid memory leak pq.txs = old[0 : n-1] return item } @@ -420,6 +425,4 @@ func (pq *TxPriorityQueue) Less(i, j int) bool { // Swap implements the Heap interface. It swaps two transactions in the queue. func (pq *TxPriorityQueue) Swap(i, j int) { pq.txs[i], pq.txs[j] = pq.txs[j], pq.txs[i] - pq.txs[i].heapIndex = i - pq.txs[j].heapIndex = j } diff --git a/internal/mempool/priority_queue_test.go b/internal/mempool/priority_queue_test.go index 8cb6d2e1a..eb340b170 100644 --- a/internal/mempool/priority_queue_test.go +++ b/internal/mempool/priority_queue_test.go @@ -47,6 +47,20 @@ func TestTxPriorityQueue_ReapHalf(t *testing.T) { } } +func TestAvoidPanicIfTransactionIsNil(t *testing.T) { + pq := NewTxPriorityQueue() + pq.Push(&WrappedTx{sender: "1", isEVM: true, evmAddress: "0xabc", evmNonce: 1, priority: 10}) + pq.txs = append(pq.txs, nil) + + var count int + pq.ForEachTx(func(tx *WrappedTx) bool { + count++ + return true + }) + + require.Equal(t, 1, count) +} + func TestTxPriorityQueue_PriorityAndNonceOrdering(t *testing.T) { testCases := []TxTestCase{ { From 80279d48423d3cdc490cbcd31ec8332232da2436 Mon Sep 17 00:00:00 2001 From: Steven Landers Date: Mon, 5 Feb 2024 11:04:37 -0500 Subject: [PATCH 16/21] call callback from mempool (#200) --- internal/mempool/mempool.go | 45 ++++++++++++++++++++----------------- 1 file changed, 24 insertions(+), 21 deletions(-) diff --git a/internal/mempool/mempool.go b/internal/mempool/mempool.go index df30dc7c0..5e1e558ce 100644 --- a/internal/mempool/mempool.go +++ b/internal/mempool/mempool.go @@ -286,27 +286,32 @@ func (txmp *TxMempool) CheckTx( } res, err := txmp.proxyAppConn.CheckTx(ctx, &abci.RequestCheckTx{Tx: tx}) + + // when a transaction is removed/expired/rejected, this should be called + // The expire tx handler unreserves the pending nonce + removeHandler := func(removeFromCache bool) { + if removeFromCache { + txmp.cache.Remove(tx) + } + if res.ExpireTxHandler != nil { + res.ExpireTxHandler() + } + } + if err != nil { - txmp.cache.Remove(tx) + removeHandler(true) res.Log = txmp.AppendCheckTxErr(res.Log, err.Error()) } wtx := &WrappedTx{ - tx: tx, - hash: txHash, - timestamp: time.Now().UTC(), - height: txmp.height, - evmNonce: res.EVMNonce, - evmAddress: res.EVMSenderAddress, - isEVM: res.IsEVM, - removeHandler: func(removeFromCache bool) { - if removeFromCache { - txmp.cache.Remove(tx) - } - if res.ExpireTxHandler != nil { - res.ExpireTxHandler() - } - }, + tx: tx, + hash: txHash, + timestamp: time.Now().UTC(), + height: txmp.height, + evmNonce: res.EVMNonce, + evmAddress: res.EVMSenderAddress, + isEVM: res.IsEVM, + removeHandler: removeHandler, } if err == nil { @@ -561,9 +566,7 @@ func (txmp *TxMempool) addNewTransaction(wtx *WrappedTx, res *abci.ResponseCheck txmp.metrics.FailedTxs.Add(1) - if !txmp.config.KeepInvalidTxsInCache { - txmp.cache.Remove(wtx.tx) - } + wtx.removeHandler(!txmp.config.KeepInvalidTxsInCache) if res.Code != abci.CodeTypeOK { txmp.mtxFailedCheckTxCounts.Lock() defer txmp.mtxFailedCheckTxCounts.Unlock() @@ -601,7 +604,7 @@ func (txmp *TxMempool) addNewTransaction(wtx *WrappedTx, res *abci.ResponseCheck if len(evictTxs) == 0 { // No room for the new incoming transaction so we just remove it from // the cache. - txmp.cache.Remove(wtx.tx) + wtx.removeHandler(true) txmp.logger.Error( "rejected incoming good transaction; mempool full", "tx", fmt.Sprintf("%X", wtx.tx.Hash()), @@ -983,7 +986,7 @@ func (txmp *TxMempool) handlePendingTransactions() { } if !txmp.config.KeepInvalidTxsInCache { for _, tx := range rejected { - txmp.cache.Remove(tx.tx.tx) + tx.tx.removeHandler(true) } } } From e4415a8d5151a82a3684fae72eb96fd1f549aef4 Mon Sep 17 00:00:00 2001 From: codchen Date: Wed, 21 Feb 2024 10:13:48 +0800 Subject: [PATCH 17/21] separate limit for pending tx (#202) --- config/config.go | 14 +++++++++ config/toml.go | 8 +++++ internal/mempool/mempool.go | 63 ++++++++++++++++++++++++++++--------- types/mempool.go | 20 ++++++++++++ 4 files changed, 91 insertions(+), 14 deletions(-) diff --git a/config/config.go b/config/config.go index d5178869d..2d78698ab 100644 --- a/config/config.go +++ b/config/config.go @@ -799,6 +799,16 @@ type MempoolConfig struct { // blacklist the peer. CheckTxErrorBlacklistEnabled bool `mapstructure:"check-tx-error-blacklist-enabled"` CheckTxErrorThreshold int `mapstructure:"check-tx-error-threshold"` + + // Maximum number of transactions in the pending set + PendingSize int `mapstructure:"pending-size"` + + // Limit the total size of all txs in the pending set. + MaxPendingTxsBytes int64 `mapstructure:"max-pending-txs-bytes"` + + PendingTTLDuration time.Duration `mapstructure:"pending-ttl-duration"` + + PendingTTLNumBlocks int64 `mapstructure:"pending-ttl-num-blocks"` } // DefaultMempoolConfig returns a default configuration for the Tendermint mempool. @@ -816,6 +826,10 @@ func DefaultMempoolConfig() *MempoolConfig { TxNotifyThreshold: 0, CheckTxErrorBlacklistEnabled: false, CheckTxErrorThreshold: 0, + PendingSize: 5000, + MaxPendingTxsBytes: 1024 * 1024 * 1024, // 1GB + PendingTTLDuration: 0 * time.Second, + PendingTTLNumBlocks: 0, } } diff --git a/config/toml.go b/config/toml.go index 4e6c7577d..eee6fe008 100644 --- a/config/toml.go +++ b/config/toml.go @@ -405,6 +405,14 @@ check-tx-error-blacklist-enabled = {{ .Mempool.CheckTxErrorBlacklistEnabled }} check-tx-error-threshold = {{ .Mempool.CheckTxErrorThreshold }} +pending-size = {{ .Mempool.PendingSize }} + +max-pending-txs-bytes = {{ .Mempool.MaxPendingTxsBytes }} + +pending-ttl-duration = {{ .Mempool.PendingTTLDuration }} + +pending-ttl-num-blocks = {{ .Mempool.PendingTTLNumBlocks }} + ####################################################### ### State Sync Configuration Options ### ####################################################### diff --git a/internal/mempool/mempool.go b/internal/mempool/mempool.go index 5e1e558ce..8504c7cd7 100644 --- a/internal/mempool/mempool.go +++ b/internal/mempool/mempool.go @@ -43,6 +43,9 @@ type TxMempool struct { // sizeBytes defines the total size of the mempool (sum of all tx bytes) sizeBytes int64 + // pendingSizeBytes defines the total size of the pending set (sum of all tx bytes) + pendingSizeBytes int64 + // cache defines a fixed-size cache of already seen transactions as this // reduces pressure on the proxyApp. cache TxCache @@ -177,9 +180,11 @@ func (txmp *TxMempool) Unlock() { // Size returns the number of valid transactions in the mempool. It is // thread-safe. func (txmp *TxMempool) Size() int { - txSize := txmp.txStore.Size() - pendingSize := txmp.pendingTxs.Size() - return txSize + pendingSize + return txmp.SizeWithoutPending() + txmp.PendingSize() +} + +func (txmp *TxMempool) SizeWithoutPending() int { + return txmp.txStore.Size() } // PendingSize returns the number of pending transactions in the mempool. @@ -193,6 +198,10 @@ func (txmp *TxMempool) SizeBytes() int64 { return atomic.LoadInt64(&txmp.sizeBytes) } +func (txmp *TxMempool) PendingSizeBytes() int64 { + return atomic.LoadInt64(&txmp.pendingSizeBytes) +} + // FlushAppConn executes FlushSync on the mempool's proxyAppConn. // // NOTE: The caller must obtain a write-lock prior to execution. @@ -326,6 +335,11 @@ func (txmp *TxMempool) CheckTx( if res.Checker == nil { return errors.New("no checker available for pending transaction") } + if err := txmp.canAddPendingTx(wtx); err != nil { + // TODO: eviction strategy for pending transactions + return err + } + atomic.AddInt64(&txmp.pendingSizeBytes, int64(wtx.Size())) txmp.pendingTxs.Insert(wtx, res, txInfo) } } @@ -410,7 +424,7 @@ func (txmp *TxMempool) ReapMaxBytesMaxGas(maxBytes, maxGas int64) types.Txs { ) var txs []types.Tx - if uint64(txmp.Size()) < txmp.config.TxNotifyThreshold { + if uint64(txmp.SizeWithoutPending()) < txmp.config.TxNotifyThreshold { // do not reap anything if threshold is not met return txs } @@ -522,7 +536,7 @@ func (txmp *TxMempool) Update( } } - txmp.metrics.Size.Set(float64(txmp.Size())) + txmp.metrics.Size.Set(float64(txmp.SizeWithoutPending())) txmp.metrics.PendingSize.Set(float64(txmp.PendingSize())) return nil } @@ -640,7 +654,7 @@ func (txmp *TxMempool) addNewTransaction(wtx *WrappedTx, res *abci.ResponseCheck } txmp.metrics.TxSizeBytes.Observe(float64(wtx.Size())) - txmp.metrics.Size.Set(float64(txmp.Size())) + txmp.metrics.Size.Set(float64(txmp.SizeWithoutPending())) txmp.metrics.PendingSize.Set(float64(txmp.PendingSize())) if txmp.insertTx(wtx) { @@ -649,7 +663,7 @@ func (txmp *TxMempool) addNewTransaction(wtx *WrappedTx, res *abci.ResponseCheck "priority", wtx.priority, "tx", fmt.Sprintf("%X", wtx.tx.Hash()), "height", txmp.height, - "num_txs", txmp.Size(), + "num_txs", txmp.SizeWithoutPending(), ) txmp.notifyTxsAvailable() } @@ -745,12 +759,12 @@ func (txmp *TxMempool) handleRecheckResult(tx types.Tx, res *abci.ResponseCheckT if txmp.recheckCursor == nil { txmp.logger.Debug("finished rechecking transactions") - if txmp.Size() > 0 { + if txmp.SizeWithoutPending() > 0 { txmp.notifyTxsAvailable() } } - txmp.metrics.Size.Set(float64(txmp.Size())) + txmp.metrics.Size.Set(float64(txmp.SizeWithoutPending())) txmp.metrics.PendingSize.Set(float64(txmp.PendingSize())) } @@ -803,7 +817,7 @@ func (txmp *TxMempool) updateReCheckTxs(ctx context.Context) { // the transaction can be inserted into the mempool. func (txmp *TxMempool) canAddTx(wtx *WrappedTx) error { var ( - numTxs = txmp.Size() + numTxs = txmp.SizeWithoutPending() sizeBytes = txmp.SizeBytes() ) @@ -819,6 +833,24 @@ func (txmp *TxMempool) canAddTx(wtx *WrappedTx) error { return nil } +func (txmp *TxMempool) canAddPendingTx(wtx *WrappedTx) error { + var ( + numTxs = txmp.PendingSize() + sizeBytes = txmp.PendingSizeBytes() + ) + + if numTxs >= txmp.config.PendingSize || int64(wtx.Size())+sizeBytes > txmp.config.MaxPendingTxsBytes { + return types.ErrMempoolPendingIsFull{ + NumTxs: numTxs, + MaxTxs: txmp.config.PendingSize, + TxsBytes: sizeBytes, + MaxTxsBytes: txmp.config.MaxPendingTxsBytes, + } + } + + return nil +} + func (txmp *TxMempool) insertTx(wtx *WrappedTx) bool { if txmp.isInMempool(wtx.tx) { return false @@ -935,13 +967,14 @@ func (txmp *TxMempool) purgeExpiredTxs(blockHeight int64) { } // remove pending txs that have expired - txmp.pendingTxs.PurgeExpired(txmp.config.TTLNumBlocks, blockHeight, txmp.config.TTLDuration, now, func(wtx *WrappedTx) { + txmp.pendingTxs.PurgeExpired(txmp.config.PendingTTLNumBlocks, blockHeight, txmp.config.PendingTTLDuration, now, func(wtx *WrappedTx) { + atomic.AddInt64(&txmp.pendingSizeBytes, int64(-wtx.Size())) txmp.expire(blockHeight, wtx) }) } func (txmp *TxMempool) notifyTxsAvailable() { - if txmp.Size() == 0 { + if txmp.SizeWithoutPending() == 0 { return } @@ -980,12 +1013,14 @@ func (txmp *TxMempool) AppendCheckTxErr(existingLogs string, log string) string func (txmp *TxMempool) handlePendingTransactions() { accepted, rejected := txmp.pendingTxs.EvaluatePendingTransactions() for _, tx := range accepted { + atomic.AddInt64(&txmp.pendingSizeBytes, int64(-tx.tx.Size())) if err := txmp.addNewTransaction(tx.tx, tx.checkTxResponse.ResponseCheckTx, tx.txInfo); err != nil { txmp.logger.Error(fmt.Sprintf("error adding pending transaction: %s", err)) } } - if !txmp.config.KeepInvalidTxsInCache { - for _, tx := range rejected { + for _, tx := range rejected { + atomic.AddInt64(&txmp.pendingSizeBytes, int64(-tx.tx.Size())) + if !txmp.config.KeepInvalidTxsInCache { tx.tx.removeHandler(true) } } diff --git a/types/mempool.go b/types/mempool.go index 36e877508..a3a0c831e 100644 --- a/types/mempool.go +++ b/types/mempool.go @@ -4,6 +4,7 @@ import ( "crypto/sha256" "errors" "fmt" + tmproto "github.com/tendermint/tendermint/proto/tendermint/types" ) @@ -84,6 +85,25 @@ func (e ErrMempoolIsFull) Error() string { ) } +// ErrMempoolPendingIsFull defines an error where there are too many pending transactions +// not processed yet +type ErrMempoolPendingIsFull struct { + NumTxs int + MaxTxs int + TxsBytes int64 + MaxTxsBytes int64 +} + +func (e ErrMempoolPendingIsFull) Error() string { + return fmt.Sprintf( + "mempool pending set is full: number of txs %d (max: %d), total txs bytes %d (max: %d)", + e.NumTxs, + e.MaxTxs, + e.TxsBytes, + e.MaxTxsBytes, + ) +} + // ErrPreCheck defines an error where a transaction fails a pre-check. type ErrPreCheck struct { Reason error From c9e294440224ae2fceb86bc7dcee062c3d20f840 Mon Sep 17 00:00:00 2001 From: codchen Date: Mon, 26 Feb 2024 10:42:12 +0800 Subject: [PATCH 18/21] Add EVM txs eviction logic (#204) --- config/toml.go | 2 +- internal/mempool/mempool.go | 36 +++++-- internal/mempool/mempool_test.go | 127 ++++++++++++++++++++++-- internal/mempool/priority_queue.go | 26 +++-- internal/mempool/priority_queue_test.go | 8 +- internal/mempool/tx.go | 38 +++++-- internal/mempool/tx_test.go | 9 +- 7 files changed, 203 insertions(+), 43 deletions(-) diff --git a/config/toml.go b/config/toml.go index eee6fe008..ae0667010 100644 --- a/config/toml.go +++ b/config/toml.go @@ -409,7 +409,7 @@ pending-size = {{ .Mempool.PendingSize }} max-pending-txs-bytes = {{ .Mempool.MaxPendingTxsBytes }} -pending-ttl-duration = {{ .Mempool.PendingTTLDuration }} +pending-ttl-duration = "{{ .Mempool.PendingTTLDuration }}" pending-ttl-num-blocks = {{ .Mempool.PendingTTLNumBlocks }} diff --git a/internal/mempool/mempool.go b/internal/mempool/mempool.go index 8504c7cd7..96b214a33 100644 --- a/internal/mempool/mempool.go +++ b/internal/mempool/mempool.go @@ -127,7 +127,7 @@ func NewTxMempool( timestampIndex: NewWrappedTxList(func(wtx1, wtx2 *WrappedTx) bool { return wtx1.timestamp.After(wtx2.timestamp) || wtx1.timestamp.Equal(wtx2.timestamp) }), - pendingTxs: NewPendingTxs(), + pendingTxs: NewPendingTxs(cfg), failedCheckTxCounts: map[types.NodeID]uint64{}, peerManager: peerManager, } @@ -340,7 +340,9 @@ func (txmp *TxMempool) CheckTx( return err } atomic.AddInt64(&txmp.pendingSizeBytes, int64(wtx.Size())) - txmp.pendingTxs.Insert(wtx, res, txInfo) + if err := txmp.pendingTxs.Insert(wtx, res, txInfo); err != nil { + return err + } } } @@ -362,7 +364,7 @@ func (txmp *TxMempool) RemoveTxByKey(txKey types.TxKey) error { // remove the committed transaction from the transaction store and indexes if wtx := txmp.txStore.GetTxByHash(txKey); wtx != nil { - txmp.removeTx(wtx, false) + txmp.removeTx(wtx, false, true) return nil } @@ -401,7 +403,7 @@ func (txmp *TxMempool) Flush() { txmp.timestampIndex.Reset() for _, wtx := range txmp.txStore.GetAllTxs() { - txmp.removeTx(wtx, false) + txmp.removeTx(wtx, false, false) } atomic.SwapInt64(&txmp.sizeBytes, 0) @@ -513,7 +515,7 @@ func (txmp *TxMempool) Update( // remove the committed transaction from the transaction store and indexes if wtx := txmp.txStore.GetTxByHash(tx.Key()); wtx != nil { - txmp.removeTx(wtx, false) + txmp.removeTx(wtx, false, false) } } @@ -634,7 +636,7 @@ func (txmp *TxMempool) addNewTransaction(wtx *WrappedTx, res *abci.ResponseCheck // - The transaction, toEvict, can be removed while a concurrent // reCheckTx callback is being executed for the same transaction. for _, toEvict := range evictTxs { - txmp.removeTx(toEvict, true) + txmp.removeTx(toEvict, true, true) txmp.logger.Debug( "evicted existing good transaction; mempool full", "old_tx", fmt.Sprintf("%X", toEvict.tx.Hash()), @@ -745,7 +747,7 @@ func (txmp *TxMempool) handleRecheckResult(tx types.Tx, res *abci.ResponseCheckT panic("corrupted reCheckTx cursor") } - txmp.removeTx(wtx, !txmp.config.KeepInvalidTxsInCache) + txmp.removeTx(wtx, !txmp.config.KeepInvalidTxsInCache, true) } } @@ -871,13 +873,13 @@ func (txmp *TxMempool) insertTx(wtx *WrappedTx) bool { return true } -func (txmp *TxMempool) removeTx(wtx *WrappedTx, removeFromCache bool) { +func (txmp *TxMempool) removeTx(wtx *WrappedTx, removeFromCache bool, shouldReenqueue bool) { if txmp.txStore.IsTxRemoved(wtx.hash) { return } txmp.txStore.RemoveTx(wtx) - txmp.priorityIndex.RemoveTx(wtx) + toBeReenqueued := txmp.priorityIndex.RemoveTx(wtx, shouldReenqueue) txmp.heightIndex.Remove(wtx) txmp.timestampIndex.Remove(wtx) @@ -889,6 +891,20 @@ func (txmp *TxMempool) removeTx(wtx *WrappedTx, removeFromCache bool) { atomic.AddInt64(&txmp.sizeBytes, int64(-wtx.Size())) wtx.removeHandler(removeFromCache) + + if shouldReenqueue { + for _, reenqueue := range toBeReenqueued { + txmp.removeTx(reenqueue, removeFromCache, false) + } + for _, reenqueue := range toBeReenqueued { + rtx := reenqueue.tx + go func() { + if err := txmp.CheckTx(context.Background(), rtx, nil, TxInfo{}); err != nil { + txmp.logger.Error(fmt.Sprintf("failed to reenqueue transaction %X due to %s", rtx.Hash(), err)) + } + }() + } + } } func (txmp *TxMempool) expire(blockHeight int64, wtx *WrappedTx) { @@ -967,7 +983,7 @@ func (txmp *TxMempool) purgeExpiredTxs(blockHeight int64) { } // remove pending txs that have expired - txmp.pendingTxs.PurgeExpired(txmp.config.PendingTTLNumBlocks, blockHeight, txmp.config.PendingTTLDuration, now, func(wtx *WrappedTx) { + txmp.pendingTxs.PurgeExpired(blockHeight, now, func(wtx *WrappedTx) { atomic.AddInt64(&txmp.pendingSizeBytes, int64(-wtx.Size())) txmp.expire(blockHeight, wtx) }) diff --git a/internal/mempool/mempool_test.go b/internal/mempool/mempool_test.go index dd4874417..9b7d56888 100644 --- a/internal/mempool/mempool_test.go +++ b/internal/mempool/mempool_test.go @@ -30,6 +30,8 @@ import ( // transaction priority based on the value in the key/value pair. type application struct { *kvstore.Application + + occupiedNonces map[string][]uint64 } type testTx struct { @@ -38,6 +40,7 @@ type testTx struct { } func (app *application) CheckTx(_ context.Context, req *abci.RequestCheckTx) (*abci.ResponseCheckTxV2, error) { + var ( priority int64 sender string @@ -58,7 +61,7 @@ func (app *application) CheckTx(_ context.Context, req *abci.RequestCheckTx) (*a GasWanted: 1, }}, nil } - nonce, err := strconv.ParseInt(string(parts[3]), 10, 64) + nonce, err := strconv.ParseUint(string(parts[3]), 10, 64) if err != nil { // could not parse return &abci.ResponseCheckTxV2{ResponseCheckTx: &abci.ResponseCheckTx{ @@ -67,15 +70,50 @@ func (app *application) CheckTx(_ context.Context, req *abci.RequestCheckTx) (*a GasWanted: 1, }}, nil } + if app.occupiedNonces == nil { + app.occupiedNonces = make(map[string][]uint64) + } + if _, exists := app.occupiedNonces[account]; !exists { + app.occupiedNonces[account] = []uint64{} + } + active := true + for i := uint64(0); i < nonce; i++ { + found := false + for _, occ := range app.occupiedNonces[account] { + if occ == i { + found = true + break + } + } + if !found { + active = false + break + } + } + app.occupiedNonces[account] = append(app.occupiedNonces[account], nonce) return &abci.ResponseCheckTxV2{ ResponseCheckTx: &abci.ResponseCheckTx{ Priority: v, Code: code.CodeTypeOK, GasWanted: 1, }, - EVMNonce: uint64(nonce), - EVMSenderAddress: account, - IsEVM: true, + EVMNonce: nonce, + EVMSenderAddress: account, + IsEVM: true, + IsPendingTransaction: !active, + Checker: func() abci.PendingTxCheckerResponse { return abci.Pending }, + ExpireTxHandler: func() { + idx := -1 + for i, n := range app.occupiedNonces[account] { + if n == nonce { + idx = i + break + } + } + if idx >= 0 { + app.occupiedNonces[account] = append(app.occupiedNonces[account][:idx], app.occupiedNonces[account][idx+1:]...) + } + }, }, nil } @@ -470,12 +508,14 @@ func TestTxMempool_Prioritization(t *testing.T) { txs := [][]byte{ []byte(fmt.Sprintf("sender-0-1=peer=%d", 9)), []byte(fmt.Sprintf("sender-1-1=peer=%d", 8)), - []byte(fmt.Sprintf("evm-sender=%s=%d=%d", address1, 7, 0)), - []byte(fmt.Sprintf("evm-sender=%s=%d=%d", address1, 9, 1)), []byte(fmt.Sprintf("evm-sender=%s=%d=%d", address2, 6, 0)), []byte(fmt.Sprintf("sender-2-1=peer=%d", 5)), []byte(fmt.Sprintf("sender-3-1=peer=%d", 4)), } + evmTxs := [][]byte{ + []byte(fmt.Sprintf("evm-sender=%s=%d=%d", address1, 7, 0)), + []byte(fmt.Sprintf("evm-sender=%s=%d=%d", address1, 9, 1)), + } // copy the slice of txs and shuffle the order randomly txsCopy := make([][]byte, len(txs)) @@ -484,6 +524,16 @@ func TestTxMempool_Prioritization(t *testing.T) { rng.Shuffle(len(txsCopy), func(i, j int) { txsCopy[i], txsCopy[j] = txsCopy[j], txsCopy[i] }) + txs = [][]byte{ + []byte(fmt.Sprintf("sender-0-1=peer=%d", 9)), + []byte(fmt.Sprintf("sender-1-1=peer=%d", 8)), + []byte(fmt.Sprintf("evm-sender=%s=%d=%d", address1, 7, 0)), + []byte(fmt.Sprintf("evm-sender=%s=%d=%d", address1, 9, 1)), + []byte(fmt.Sprintf("evm-sender=%s=%d=%d", address2, 6, 0)), + []byte(fmt.Sprintf("sender-2-1=peer=%d", 5)), + []byte(fmt.Sprintf("sender-3-1=peer=%d", 4)), + } + txsCopy = append(txsCopy, evmTxs...) for i := range txsCopy { require.NoError(t, txmp.CheckTx(ctx, txsCopy[i], nil, TxInfo{SenderID: peerID})) @@ -504,6 +554,71 @@ func TestTxMempool_Prioritization(t *testing.T) { } } +func TestTxMempool_PendingStoreSize(t *testing.T) { + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + client := abciclient.NewLocalClient(log.NewNopLogger(), &application{Application: kvstore.NewApplication()}) + if err := client.Start(ctx); err != nil { + t.Fatal(err) + } + t.Cleanup(client.Wait) + + txmp := setup(t, client, 100) + txmp.config.PendingSize = 1 + peerID := uint16(1) + + address1 := "0xeD23B3A9DE15e92B9ef9540E587B3661E15A12fA" + + require.NoError(t, txmp.CheckTx(ctx, []byte(fmt.Sprintf("evm-sender=%s=%d=%d", address1, 1, 1)), nil, TxInfo{SenderID: peerID})) + err := txmp.CheckTx(ctx, []byte(fmt.Sprintf("evm-sender=%s=%d=%d", address1, 1, 2)), nil, TxInfo{SenderID: peerID}) + require.Error(t, err) + require.Contains(t, err.Error(), "mempool pending set is full") +} + +func TestTxMempool_EVMEviction(t *testing.T) { + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + client := abciclient.NewLocalClient(log.NewNopLogger(), &application{Application: kvstore.NewApplication()}) + if err := client.Start(ctx); err != nil { + t.Fatal(err) + } + t.Cleanup(client.Wait) + + txmp := setup(t, client, 100) + txmp.config.Size = 1 + peerID := uint16(1) + + address1 := "0xeD23B3A9DE15e92B9ef9540E587B3661E15A12fA" + address2 := "0xfD23B3A9DE15e92B9ef9540E587B3661E15A12fA" + + require.NoError(t, txmp.CheckTx(ctx, []byte(fmt.Sprintf("evm-sender=%s=%d=%d", address1, 1, 0)), nil, TxInfo{SenderID: peerID})) + // this should evict the previous tx + require.NoError(t, txmp.CheckTx(ctx, []byte(fmt.Sprintf("evm-sender=%s=%d=%d", address1, 2, 0)), nil, TxInfo{SenderID: peerID})) + require.Equal(t, 1, txmp.priorityIndex.NumTxs()) + require.Equal(t, int64(2), txmp.priorityIndex.txs[0].priority) + + txmp.config.Size = 2 + require.NoError(t, txmp.CheckTx(ctx, []byte(fmt.Sprintf("evm-sender=%s=%d=%d", address1, 3, 1)), nil, TxInfo{SenderID: peerID})) + require.Equal(t, 0, txmp.pendingTxs.Size()) + require.Equal(t, 2, txmp.priorityIndex.NumTxs()) + // this would evict the tx with priority 2 and cause the tx with priority 3 to go pending + require.NoError(t, txmp.CheckTx(ctx, []byte(fmt.Sprintf("evm-sender=%s=%d=%d", address2, 4, 0)), nil, TxInfo{SenderID: peerID})) + time.Sleep(1 * time.Second) // reenqueue is async + require.Equal(t, 1, txmp.priorityIndex.NumTxs()) + tx := txmp.priorityIndex.txs[0] + require.Equal(t, 1, txmp.pendingTxs.Size()) + + require.NoError(t, txmp.CheckTx(ctx, []byte(fmt.Sprintf("evm-sender=%s=%d=%d", address2, 5, 1)), nil, TxInfo{SenderID: peerID})) + require.Equal(t, 2, txmp.priorityIndex.NumTxs()) + txmp.removeTx(tx, true, false) + // should not reenqueue + require.Equal(t, 1, txmp.priorityIndex.NumTxs()) + time.Sleep(1 * time.Second) // pendingTxs should still be one even after sleeping for a sec + require.Equal(t, 1, txmp.pendingTxs.Size()) +} + func TestTxMempool_CheckTxSamePeer(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) defer cancel() diff --git a/internal/mempool/priority_queue.go b/internal/mempool/priority_queue.go index abcbfe6a7..e9a357d63 100644 --- a/internal/mempool/priority_queue.go +++ b/internal/mempool/priority_queue.go @@ -60,8 +60,11 @@ func (pq *TxPriorityQueue) GetEvictableTxs(priority, txSize, totalSize, cap int6 pq.mtx.RLock() defer pq.mtx.RUnlock() - txs := make([]*WrappedTx, len(pq.txs)) - copy(txs, pq.txs) + txs := []*WrappedTx{} + txs = append(txs, pq.txs...) + for _, queue := range pq.evmQueue { + txs = append(txs, queue[1:]...) + } sort.Slice(txs, func(i, j int) bool { return txs[i].priority < txs[j].priority @@ -111,7 +114,7 @@ func (pq *TxPriorityQueue) NumTxs() int { return len(pq.txs) + pq.numQueuedUnsafe() } -func (pq *TxPriorityQueue) removeQueuedEvmTxUnsafe(tx *WrappedTx) { +func (pq *TxPriorityQueue) removeQueuedEvmTxUnsafe(tx *WrappedTx) (removedIdx int) { if queue, ok := pq.evmQueue[tx.evmAddress]; ok { for i, t := range queue { if t.tx.Key() == tx.tx.Key() { @@ -119,10 +122,11 @@ func (pq *TxPriorityQueue) removeQueuedEvmTxUnsafe(tx *WrappedTx) { if len(pq.evmQueue[tx.evmAddress]) == 0 { delete(pq.evmQueue, tx.evmAddress) } - break + return i } } } + return -1 } func (pq *TxPriorityQueue) findTxIndexUnsafe(tx *WrappedTx) (int, bool) { @@ -135,21 +139,27 @@ func (pq *TxPriorityQueue) findTxIndexUnsafe(tx *WrappedTx) (int, bool) { } // RemoveTx removes a specific transaction from the priority queue. -func (pq *TxPriorityQueue) RemoveTx(tx *WrappedTx) { +func (pq *TxPriorityQueue) RemoveTx(tx *WrappedTx, shouldReenqueue bool) (toBeReenqueued []*WrappedTx) { pq.mtx.Lock() defer pq.mtx.Unlock() + var removedIdx int + if idx, ok := pq.findTxIndexUnsafe(tx); ok { heap.Remove(pq, idx) if tx.isEVM { - pq.removeQueuedEvmTxUnsafe(tx) - if len(pq.evmQueue[tx.evmAddress]) > 0 { + removedIdx = pq.removeQueuedEvmTxUnsafe(tx) + if !shouldReenqueue && len(pq.evmQueue[tx.evmAddress]) > 0 { heap.Push(pq, pq.evmQueue[tx.evmAddress][0]) } } } else if tx.isEVM { - pq.removeQueuedEvmTxUnsafe(tx) + removedIdx = pq.removeQueuedEvmTxUnsafe(tx) + } + if tx.isEVM && shouldReenqueue && len(pq.evmQueue[tx.evmAddress]) > 0 && removedIdx >= 0 { + toBeReenqueued = pq.evmQueue[tx.evmAddress][removedIdx:] } + return } func (pq *TxPriorityQueue) pushTxUnsafe(tx *WrappedTx) { diff --git a/internal/mempool/priority_queue_test.go b/internal/mempool/priority_queue_test.go index eb340b170..f680dc2fc 100644 --- a/internal/mempool/priority_queue_test.go +++ b/internal/mempool/priority_queue_test.go @@ -331,7 +331,7 @@ func TestTxPriorityQueue_RemoveTxEvm(t *testing.T) { pq.PushTx(tx1) pq.PushTx(tx2) - pq.RemoveTx(tx1) + pq.RemoveTx(tx1, false) result := pq.PopTx() require.Equal(t, tx2, result) @@ -360,14 +360,14 @@ func TestTxPriorityQueue_RemoveTx(t *testing.T) { max := values[len(values)-1] wtx := pq.txs[pq.NumTxs()/2] - pq.RemoveTx(wtx) + pq.RemoveTx(wtx, false) require.Equal(t, numTxs-1, pq.NumTxs()) require.Equal(t, int64(max), pq.PopTx().priority) require.Equal(t, numTxs-2, pq.NumTxs()) require.NotPanics(t, func() { - pq.RemoveTx(&WrappedTx{heapIndex: numTxs}) - pq.RemoveTx(&WrappedTx{heapIndex: numTxs + 1}) + pq.RemoveTx(&WrappedTx{heapIndex: numTxs}, false) + pq.RemoveTx(&WrappedTx{heapIndex: numTxs + 1}, false) }) require.Equal(t, numTxs-2, pq.NumTxs()) } diff --git a/internal/mempool/tx.go b/internal/mempool/tx.go index 13ddb0a12..79c91e039 100644 --- a/internal/mempool/tx.go +++ b/internal/mempool/tx.go @@ -1,11 +1,13 @@ package mempool import ( + "errors" "sort" "sync" "time" abci "github.com/tendermint/tendermint/abci/types" + "github.com/tendermint/tendermint/config" "github.com/tendermint/tendermint/internal/libs/clist" "github.com/tendermint/tendermint/types" ) @@ -308,8 +310,10 @@ func (wtl *WrappedTxList) Remove(wtx *WrappedTx) { } type PendingTxs struct { - mtx *sync.RWMutex - txs []TxWithResponse + mtx *sync.RWMutex + txs []TxWithResponse + config *config.MempoolConfig + sizeBytes uint64 } type TxWithResponse struct { @@ -318,10 +322,12 @@ type TxWithResponse struct { txInfo TxInfo } -func NewPendingTxs() *PendingTxs { +func NewPendingTxs(conf *config.MempoolConfig) *PendingTxs { return &PendingTxs{ - mtx: &sync.RWMutex{}, - txs: []TxWithResponse{}, + mtx: &sync.RWMutex{}, + txs: []TxWithResponse{}, + config: conf, + sizeBytes: 0, } } func (p *PendingTxs) EvaluatePendingTransactions() ( @@ -359,6 +365,7 @@ func (p *PendingTxs) popTxsAtIndices(indices []int) { if idx >= len(p.txs) { panic("indices popped from pending tx store out of range") } + p.sizeBytes -= uint64(p.txs[idx].tx.Size()) newTxs = append(newTxs, p.txs[start:idx]...) start = idx + 1 } @@ -366,14 +373,21 @@ func (p *PendingTxs) popTxsAtIndices(indices []int) { p.txs = newTxs } -func (p *PendingTxs) Insert(tx *WrappedTx, resCheckTx *abci.ResponseCheckTxV2, txInfo TxInfo) { +func (p *PendingTxs) Insert(tx *WrappedTx, resCheckTx *abci.ResponseCheckTxV2, txInfo TxInfo) error { p.mtx.Lock() defer p.mtx.Unlock() + + if len(p.txs) >= p.config.PendingSize && uint64(tx.Size())+p.sizeBytes > uint64(p.config.MaxPendingTxsBytes) { + return errors.New("pending store is full") + } + p.txs = append(p.txs, TxWithResponse{ tx: tx, checkTxResponse: resCheckTx, txInfo: txInfo, }) + p.sizeBytes += uint64(tx.Size()) + return nil } func (p *PendingTxs) Peek(max int) []TxWithResponse { @@ -392,7 +406,7 @@ func (p *PendingTxs) Size() int { return len(p.txs) } -func (p *PendingTxs) PurgeExpired(ttlNumBlock int64, blockHeight int64, ttlDuration time.Duration, now time.Time, cb func(wtx *WrappedTx)) { +func (p *PendingTxs) PurgeExpired(blockHeight int64, now time.Time, cb func(wtx *WrappedTx)) { p.mtx.Lock() defer p.mtx.Unlock() @@ -401,15 +415,16 @@ func (p *PendingTxs) PurgeExpired(ttlNumBlock int64, blockHeight int64, ttlDurat } // txs retains the ordering of insertion - if ttlNumBlock > 0 { + if p.config.TTLNumBlocks > 0 { idxFirstNotExpiredTx := len(p.txs) for i, ptx := range p.txs { // once found, we can break because these are ordered - if (blockHeight - ptx.tx.height) <= ttlNumBlock { + if (blockHeight - ptx.tx.height) <= p.config.TTLNumBlocks { idxFirstNotExpiredTx = i break } else { cb(ptx.tx) + p.sizeBytes -= uint64(ptx.tx.Size()) } } p.txs = p.txs[idxFirstNotExpiredTx:] @@ -419,15 +434,16 @@ func (p *PendingTxs) PurgeExpired(ttlNumBlock int64, blockHeight int64, ttlDurat return } - if ttlDuration > 0 { + if p.config.TTLDuration > 0 { idxFirstNotExpiredTx := len(p.txs) for i, ptx := range p.txs { // once found, we can break because these are ordered - if now.Sub(ptx.tx.timestamp) <= ttlDuration { + if now.Sub(ptx.tx.timestamp) <= p.config.TTLDuration { idxFirstNotExpiredTx = i break } else { cb(ptx.tx) + p.sizeBytes -= uint64(ptx.tx.Size()) } } p.txs = p.txs[idxFirstNotExpiredTx:] diff --git a/internal/mempool/tx_test.go b/internal/mempool/tx_test.go index 77a49c276..834beeda1 100644 --- a/internal/mempool/tx_test.go +++ b/internal/mempool/tx_test.go @@ -9,6 +9,7 @@ import ( "github.com/stretchr/testify/require" + "github.com/tendermint/tendermint/config" "github.com/tendermint/tendermint/types" ) @@ -231,7 +232,7 @@ func TestWrappedTxList_Remove(t *testing.T) { } func TestPendingTxsPopTxsGood(t *testing.T) { - pendingTxs := NewPendingTxs() + pendingTxs := NewPendingTxs(config.TestMempoolConfig()) for _, test := range []struct { origLen int popIndices []int @@ -281,7 +282,9 @@ func TestPendingTxsPopTxsGood(t *testing.T) { } { pendingTxs.txs = []TxWithResponse{} for i := 0; i < test.origLen; i++ { - pendingTxs.txs = append(pendingTxs.txs, TxWithResponse{txInfo: TxInfo{SenderID: uint16(i)}}) + pendingTxs.txs = append(pendingTxs.txs, TxWithResponse{ + tx: &WrappedTx{tx: []byte{}}, + txInfo: TxInfo{SenderID: uint16(i)}}) } pendingTxs.popTxsAtIndices(test.popIndices) require.Equal(t, len(test.expected), len(pendingTxs.txs)) @@ -292,7 +295,7 @@ func TestPendingTxsPopTxsGood(t *testing.T) { } func TestPendingTxsPopTxsBad(t *testing.T) { - pendingTxs := NewPendingTxs() + pendingTxs := NewPendingTxs(config.TestMempoolConfig()) // out of range require.Panics(t, func() { pendingTxs.popTxsAtIndices([]int{0}) }) // out of order From b4190aaad6259ec208ef872c2bda4a55eb7afb45 Mon Sep 17 00:00:00 2001 From: codchen Date: Tue, 27 Feb 2024 08:57:36 +0800 Subject: [PATCH 19/21] Fix debug log (#205) --- internal/consensus/reactor.go | 11 ++++++++++- internal/consensus/reactor_test.go | 2 ++ node/node.go | 1 + 3 files changed, 13 insertions(+), 1 deletion(-) diff --git a/internal/consensus/reactor.go b/internal/consensus/reactor.go index 3631fd1d8..3ceadae90 100644 --- a/internal/consensus/reactor.go +++ b/internal/consensus/reactor.go @@ -8,6 +8,7 @@ import ( "sync" "time" + "github.com/tendermint/tendermint/config" cstypes "github.com/tendermint/tendermint/internal/consensus/types" "github.com/tendermint/tendermint/internal/eventbus" "github.com/tendermint/tendermint/internal/p2p" @@ -121,6 +122,7 @@ type ConsSyncReactor interface { type Reactor struct { service.BaseService logger log.Logger + cfg *config.Config state *State eventBus *eventbus.EventBus @@ -148,6 +150,7 @@ func NewReactor( eventBus *eventbus.EventBus, waitSync bool, metrics *Metrics, + cfg *config.Config, ) *Reactor { r := &Reactor{ logger: logger, @@ -160,6 +163,7 @@ func NewReactor( peerEvents: peerEvents, readySignal: make(chan struct{}), channels: &channelBundle{}, + cfg: cfg, } r.BaseService = *service.NewBaseService(logger, "Consensus", r) @@ -646,7 +650,12 @@ func (r *Reactor) pickSendVote(ctx context.Context, ps *PeerState, votes types.V return false, nil } - r.logger.Debug("sending vote message", "ps", ps, "vote", vote) + if r.cfg.BaseConfig.LogLevel == log.LogLevelDebug { + psJson, err := ps.ToJSON() // expensive, so we only want to call if debug is on + if err != nil { + r.logger.Debug("sending vote message", "ps", string(psJson), "vote", vote) + } + } if err := voteCh.Send(ctx, p2p.Envelope{ To: ps.peerID, Message: &tmcons.Vote{ diff --git a/internal/consensus/reactor_test.go b/internal/consensus/reactor_test.go index 8d840a14f..97927b235 100644 --- a/internal/consensus/reactor_test.go +++ b/internal/consensus/reactor_test.go @@ -97,6 +97,7 @@ func setup( state.eventBus, true, NopMetrics(), + config.DefaultConfig(), ) reactor.SetStateChannel(rts.stateChannels[nodeID]) @@ -697,6 +698,7 @@ func TestSwitchToConsensusVoteExtensions(t *testing.T) { cs.eventBus, true, NopMetrics(), + config.DefaultConfig(), ) if testCase.shouldPanic { diff --git a/node/node.go b/node/node.go index 0cd8a62bf..d3075434e 100644 --- a/node/node.go +++ b/node/node.go @@ -349,6 +349,7 @@ func makeNode( eventBus, waitSync, nodeMetrics.consensus, + cfg, ) node.router.AddChDescToBeAdded(consensus.GetStateChannelDescriptor(), csReactor.SetStateChannel) From 77ad5bd5d124f0f8ff07f7cc9becd00bca9e3089 Mon Sep 17 00:00:00 2001 From: codchen Date: Thu, 7 Mar 2024 12:48:21 +0800 Subject: [PATCH 20/21] EVM transaction replacement (#206) (#208) --- abci/types/types.pb.go | 891 ++++++++++++++++-------- internal/mempool/mempool.go | 45 +- internal/mempool/mempool_test.go | 2 +- internal/mempool/priority_queue.go | 52 +- internal/mempool/priority_queue_test.go | 77 +- internal/mempool/reactor_test.go | 4 +- internal/mempool/tx.go | 2 +- proto/tendermint/abci/types.proto | 7 + proto/tendermint/crypto/keys.pb.go | 1 - proto/tendermint/mempool/types.pb.go | 1 - proto/tendermint/p2p/conn.pb.go | 1 - proto/tendermint/types/types.pb.go | 1 - 12 files changed, 780 insertions(+), 304 deletions(-) diff --git a/abci/types/types.pb.go b/abci/types/types.pb.go index e28ce2827..791ab6f57 100644 --- a/abci/types/types.pb.go +++ b/abci/types/types.pb.go @@ -217,6 +217,7 @@ func (ResponseVerifyVoteExtension_VerifyStatus) EnumDescriptor() ([]byte, []int) } // TxAction contains App-provided information on what to do with a transaction that is part of a raw proposal +// Deprecate entire usage: https://github.com/tendermint/tendermint/pull/9283 type TxRecord_TxAction int32 const ( @@ -2030,7 +2031,6 @@ var xxx_messageInfo_RequestLoadLatest proto.InternalMessageInfo type Response struct { // Types that are valid to be assigned to Value: - // // *Response_Exception // *Response_Echo // *Response_Flush @@ -2877,14 +2877,15 @@ func (m *ResponseCheckTx) GetPriority() int64 { } type ResponseDeliverTx struct { - Code uint32 `protobuf:"varint,1,opt,name=code,proto3" json:"code,omitempty"` - Data []byte `protobuf:"bytes,2,opt,name=data,proto3" json:"data,omitempty"` - Log string `protobuf:"bytes,3,opt,name=log,proto3" json:"log,omitempty"` - Info string `protobuf:"bytes,4,opt,name=info,proto3" json:"info,omitempty"` - GasWanted int64 `protobuf:"varint,5,opt,name=gas_wanted,proto3" json:"gas_wanted,omitempty"` - GasUsed int64 `protobuf:"varint,6,opt,name=gas_used,proto3" json:"gas_used,omitempty"` - Events []Event `protobuf:"bytes,7,rep,name=events,proto3" json:"events,omitempty"` - Codespace string `protobuf:"bytes,8,opt,name=codespace,proto3" json:"codespace,omitempty"` + Code uint32 `protobuf:"varint,1,opt,name=code,proto3" json:"code,omitempty"` + Data []byte `protobuf:"bytes,2,opt,name=data,proto3" json:"data,omitempty"` + Log string `protobuf:"bytes,3,opt,name=log,proto3" json:"log,omitempty"` + Info string `protobuf:"bytes,4,opt,name=info,proto3" json:"info,omitempty"` + GasWanted int64 `protobuf:"varint,5,opt,name=gas_wanted,proto3" json:"gas_wanted,omitempty"` + GasUsed int64 `protobuf:"varint,6,opt,name=gas_used,proto3" json:"gas_used,omitempty"` + Events []Event `protobuf:"bytes,7,rep,name=events,proto3" json:"events,omitempty"` + Codespace string `protobuf:"bytes,8,opt,name=codespace,proto3" json:"codespace,omitempty"` + EvmTxInfo *EvmTxInfo `protobuf:"bytes,9,opt,name=evm_tx_info,json=evmTxInfo,proto3" json:"evm_tx_info,omitempty"` } func (m *ResponseDeliverTx) Reset() { *m = ResponseDeliverTx{} } @@ -2976,6 +2977,13 @@ func (m *ResponseDeliverTx) GetCodespace() string { return "" } +func (m *ResponseDeliverTx) GetEvmTxInfo() *EvmTxInfo { + if m != nil { + return m.EvmTxInfo + } + return nil +} + type ResponseEndBlock struct { ValidatorUpdates []ValidatorUpdate `protobuf:"bytes,1,rep,name=validator_updates,json=validatorUpdates,proto3" json:"validator_updates"` ConsensusParamUpdates *ConsensusParams `protobuf:"bytes,2,opt,name=consensus_param_updates,json=consensusParamUpdates,proto3" json:"consensus_param_updates,omitempty"` @@ -3907,14 +3915,15 @@ func (m *EventAttribute) GetIndex() bool { // // * Its structure is equivalent to #ResponseDeliverTx which will be deprecated/deleted type ExecTxResult struct { - Code uint32 `protobuf:"varint,1,opt,name=code,proto3" json:"code,omitempty"` - Data []byte `protobuf:"bytes,2,opt,name=data,proto3" json:"data,omitempty"` - Log string `protobuf:"bytes,3,opt,name=log,proto3" json:"log,omitempty"` - Info string `protobuf:"bytes,4,opt,name=info,proto3" json:"info,omitempty"` - GasWanted int64 `protobuf:"varint,5,opt,name=gas_wanted,json=gasWanted,proto3" json:"gas_wanted,omitempty"` - GasUsed int64 `protobuf:"varint,6,opt,name=gas_used,json=gasUsed,proto3" json:"gas_used,omitempty"` - Events []Event `protobuf:"bytes,7,rep,name=events,proto3" json:"events,omitempty"` - Codespace string `protobuf:"bytes,8,opt,name=codespace,proto3" json:"codespace,omitempty"` + Code uint32 `protobuf:"varint,1,opt,name=code,proto3" json:"code,omitempty"` + Data []byte `protobuf:"bytes,2,opt,name=data,proto3" json:"data,omitempty"` + Log string `protobuf:"bytes,3,opt,name=log,proto3" json:"log,omitempty"` + Info string `protobuf:"bytes,4,opt,name=info,proto3" json:"info,omitempty"` + GasWanted int64 `protobuf:"varint,5,opt,name=gas_wanted,json=gasWanted,proto3" json:"gas_wanted,omitempty"` + GasUsed int64 `protobuf:"varint,6,opt,name=gas_used,json=gasUsed,proto3" json:"gas_used,omitempty"` + Events []Event `protobuf:"bytes,7,rep,name=events,proto3" json:"events,omitempty"` + Codespace string `protobuf:"bytes,8,opt,name=codespace,proto3" json:"codespace,omitempty"` + EvmTxInfo *EvmTxInfo `protobuf:"bytes,9,opt,name=evm_tx_info,json=evmTxInfo,proto3" json:"evm_tx_info,omitempty"` } func (m *ExecTxResult) Reset() { *m = ExecTxResult{} } @@ -4006,6 +4015,13 @@ func (m *ExecTxResult) GetCodespace() string { return "" } +func (m *ExecTxResult) GetEvmTxInfo() *EvmTxInfo { + if m != nil { + return m.EvmTxInfo + } + return nil +} + // TxResult contains results of executing the transaction. // // One usage is indexing transaction results. @@ -4639,6 +4655,58 @@ func (m *Evidence) GetTotalVotingPower() int64 { return 0 } +type EvmTxInfo struct { + SenderAddress string `protobuf:"bytes,1,opt,name=senderAddress,proto3" json:"senderAddress,omitempty"` + Nonce uint64 `protobuf:"varint,2,opt,name=nonce,proto3" json:"nonce,omitempty"` +} + +func (m *EvmTxInfo) Reset() { *m = EvmTxInfo{} } +func (m *EvmTxInfo) String() string { return proto.CompactTextString(m) } +func (*EvmTxInfo) ProtoMessage() {} +func (*EvmTxInfo) Descriptor() ([]byte, []int) { + return fileDescriptor_252557cfdd89a31a, []int{59} +} +func (m *EvmTxInfo) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *EvmTxInfo) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_EvmTxInfo.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 *EvmTxInfo) XXX_Merge(src proto.Message) { + xxx_messageInfo_EvmTxInfo.Merge(m, src) +} +func (m *EvmTxInfo) XXX_Size() int { + return m.Size() +} +func (m *EvmTxInfo) XXX_DiscardUnknown() { + xxx_messageInfo_EvmTxInfo.DiscardUnknown(m) +} + +var xxx_messageInfo_EvmTxInfo proto.InternalMessageInfo + +func (m *EvmTxInfo) GetSenderAddress() string { + if m != nil { + return m.SenderAddress + } + return "" +} + +func (m *EvmTxInfo) GetNonce() uint64 { + if m != nil { + return m.Nonce + } + return 0 +} + type Snapshot struct { Height uint64 `protobuf:"varint,1,opt,name=height,proto3" json:"height,omitempty"` Format uint32 `protobuf:"varint,2,opt,name=format,proto3" json:"format,omitempty"` @@ -4651,7 +4719,7 @@ func (m *Snapshot) Reset() { *m = Snapshot{} } func (m *Snapshot) String() string { return proto.CompactTextString(m) } func (*Snapshot) ProtoMessage() {} func (*Snapshot) Descriptor() ([]byte, []int) { - return fileDescriptor_252557cfdd89a31a, []int{59} + return fileDescriptor_252557cfdd89a31a, []int{60} } func (m *Snapshot) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4782,256 +4850,261 @@ func init() { proto.RegisterType((*ExtendedVoteInfo)(nil), "tendermint.abci.ExtendedVoteInfo") proto.RegisterType((*Misbehavior)(nil), "tendermint.abci.Misbehavior") proto.RegisterType((*Evidence)(nil), "tendermint.abci.Evidence") + proto.RegisterType((*EvmTxInfo)(nil), "tendermint.abci.EvmTxInfo") proto.RegisterType((*Snapshot)(nil), "tendermint.abci.Snapshot") } func init() { proto.RegisterFile("tendermint/abci/types.proto", fileDescriptor_252557cfdd89a31a) } var fileDescriptor_252557cfdd89a31a = []byte{ - // 3885 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xec, 0x5b, 0x4b, 0x73, 0x1b, 0xd9, - 0x75, 0x46, 0xe3, 0x8d, 0x83, 0x57, 0xf3, 0x92, 0xa2, 0x20, 0x68, 0x86, 0xe4, 0xf4, 0x94, 0x66, - 0x34, 0x1a, 0x99, 0x74, 0xa8, 0x48, 0x96, 0xa2, 0x71, 0x26, 0x24, 0x04, 0x0e, 0x28, 0x51, 0x24, - 0xa7, 0x09, 0x72, 0x32, 0x49, 0xac, 0x76, 0x13, 0xb8, 0x04, 0xda, 0x02, 0xd0, 0xed, 0xee, 0x06, - 0x07, 0x9c, 0x55, 0x2a, 0x89, 0x37, 0xce, 0x66, 0x96, 0x59, 0xc4, 0xbb, 0xf8, 0x0f, 0x64, 0x91, - 0x65, 0x56, 0xa9, 0x94, 0x17, 0x5e, 0x78, 0x91, 0x4a, 0x65, 0xe5, 0xa4, 0x66, 0xaa, 0xb2, 0xf0, - 0x1f, 0xc8, 0x2e, 0x4e, 0xdd, 0x47, 0xbf, 0x80, 0x6e, 0x3c, 0x24, 0x95, 0xab, 0x5c, 0x99, 0x5d, - 0xdf, 0xdb, 0xe7, 0x9c, 0xfb, 0x3a, 0xaf, 0xfb, 0x9d, 0x6e, 0xb8, 0x69, 0xe3, 0x41, 0x1b, 0x9b, - 0x7d, 0x6d, 0x60, 0x6f, 0xa9, 0xe7, 0x2d, 0x6d, 0xcb, 0xbe, 0x32, 0xb0, 0xb5, 0x69, 0x98, 0xba, - 0xad, 0xa3, 0xb2, 0xf7, 0x72, 0x93, 0xbc, 0xac, 0xbe, 0xed, 0xa3, 0x6e, 0x99, 0x57, 0x86, 0xad, - 0x6f, 0x19, 0xa6, 0xae, 0x5f, 0x30, 0xfa, 0xea, 0x5b, 0x93, 0xaf, 0x5f, 0xe2, 0x2b, 0x2e, 0x2d, - 0xc0, 0x4c, 0x47, 0xd9, 0x32, 0x54, 0x53, 0xed, 0x5b, 0x21, 0xcc, 0xec, 0xb5, 0x6f, 0x2a, 0xd5, - 0xf5, 0x8e, 0xae, 0x77, 0x7a, 0x78, 0x8b, 0xb6, 0xce, 0x87, 0x17, 0x5b, 0xb6, 0xd6, 0xc7, 0x96, - 0xad, 0xf6, 0x0d, 0x4e, 0xb0, 0xd2, 0xd1, 0x3b, 0x3a, 0x7d, 0xdc, 0x22, 0x4f, 0xac, 0x57, 0xfa, - 0x65, 0x1e, 0x32, 0x32, 0xfe, 0xf1, 0x10, 0x5b, 0x36, 0xda, 0x86, 0x24, 0x6e, 0x75, 0xf5, 0x8a, - 0xb0, 0x21, 0xdc, 0xce, 0x6f, 0xbf, 0xb5, 0x39, 0xb6, 0xb8, 0x4d, 0x4e, 0x57, 0x6f, 0x75, 0xf5, - 0x46, 0x4c, 0xa6, 0xb4, 0xe8, 0x3e, 0xa4, 0x2e, 0x7a, 0x43, 0xab, 0x5b, 0x89, 0x53, 0xa6, 0xb7, - 0xa3, 0x98, 0xf6, 0x08, 0x51, 0x23, 0x26, 0x33, 0x6a, 0x32, 0x94, 0x36, 0xb8, 0xd0, 0x2b, 0x89, - 0xe9, 0x43, 0xed, 0x0f, 0x2e, 0xe8, 0x50, 0x84, 0x16, 0xed, 0x02, 0x68, 0x03, 0xcd, 0x56, 0x5a, - 0x5d, 0x55, 0x1b, 0x54, 0x92, 0x94, 0xf3, 0x9d, 0x68, 0x4e, 0xcd, 0xae, 0x11, 0xc2, 0x46, 0x4c, - 0xce, 0x69, 0x4e, 0x83, 0x4c, 0xf7, 0xc7, 0x43, 0x6c, 0x5e, 0x55, 0x52, 0xd3, 0xa7, 0xfb, 0x29, - 0x21, 0x22, 0xd3, 0xa5, 0xd4, 0xe8, 0x23, 0xc8, 0xb6, 0xba, 0xb8, 0xf5, 0x52, 0xb1, 0x47, 0x95, - 0x0c, 0xe5, 0x5c, 0x8f, 0xe2, 0xac, 0x11, 0xba, 0xe6, 0xa8, 0x11, 0x93, 0x33, 0x2d, 0xf6, 0x88, - 0x1e, 0x42, 0xba, 0xa5, 0xf7, 0xfb, 0x9a, 0x5d, 0x01, 0xca, 0xbb, 0x16, 0xc9, 0x4b, 0xa9, 0x1a, - 0x31, 0x99, 0xd3, 0xa3, 0x43, 0x28, 0xf5, 0x34, 0xcb, 0x56, 0xac, 0x81, 0x6a, 0x58, 0x5d, 0xdd, - 0xb6, 0x2a, 0x79, 0x2a, 0xe1, 0x56, 0x94, 0x84, 0x03, 0xcd, 0xb2, 0x4f, 0x1c, 0xe2, 0x46, 0x4c, - 0x2e, 0xf6, 0xfc, 0x1d, 0x44, 0x9e, 0x7e, 0x71, 0x81, 0x4d, 0x57, 0x60, 0xa5, 0x30, 0x5d, 0xde, - 0x11, 0xa1, 0x76, 0xf8, 0x89, 0x3c, 0xdd, 0xdf, 0x81, 0xfe, 0x1c, 0x96, 0x7b, 0xba, 0xda, 0x76, - 0xc5, 0x29, 0xad, 0xee, 0x70, 0xf0, 0xb2, 0x52, 0xa4, 0x42, 0x3f, 0x88, 0x9c, 0xa4, 0xae, 0xb6, - 0x1d, 0x11, 0x35, 0xc2, 0xd0, 0x88, 0xc9, 0x4b, 0xbd, 0xf1, 0x4e, 0xf4, 0x02, 0x56, 0x54, 0xc3, - 0xe8, 0x5d, 0x8d, 0x4b, 0x2f, 0x51, 0xe9, 0x77, 0xa2, 0xa4, 0xef, 0x10, 0x9e, 0x71, 0xf1, 0x48, - 0x9d, 0xe8, 0x45, 0x4d, 0x10, 0x0d, 0x13, 0x1b, 0xaa, 0x89, 0x15, 0xc3, 0xd4, 0x0d, 0xdd, 0x52, - 0x7b, 0x95, 0x32, 0x95, 0xfd, 0x7e, 0x94, 0xec, 0x63, 0x46, 0x7f, 0xcc, 0xc9, 0x1b, 0x31, 0xb9, - 0x6c, 0x04, 0xbb, 0x98, 0x54, 0xbd, 0x85, 0x2d, 0xcb, 0x93, 0x2a, 0xce, 0x92, 0x4a, 0xe9, 0x83, - 0x52, 0x03, 0x5d, 0xa8, 0x0e, 0x79, 0x3c, 0x22, 0xec, 0xca, 0xa5, 0x6e, 0xe3, 0xca, 0x12, 0x15, - 0x28, 0x45, 0x5a, 0x28, 0x25, 0x3d, 0xd3, 0x6d, 0xdc, 0x88, 0xc9, 0x80, 0xdd, 0x16, 0x52, 0xe1, - 0xda, 0x25, 0x36, 0xb5, 0x8b, 0x2b, 0x2a, 0x46, 0xa1, 0x6f, 0x2c, 0x4d, 0x1f, 0x54, 0x10, 0x15, - 0xf8, 0x61, 0x94, 0xc0, 0x33, 0xca, 0x44, 0x44, 0xd4, 0x1d, 0x96, 0x46, 0x4c, 0x5e, 0xbe, 0x9c, - 0xec, 0x26, 0x2a, 0x76, 0xa1, 0x0d, 0xd4, 0x9e, 0xf6, 0x25, 0x56, 0xce, 0x7b, 0x7a, 0xeb, 0x65, - 0x65, 0x79, 0xba, 0x8a, 0xed, 0x71, 0xea, 0x5d, 0x42, 0x4c, 0x54, 0xec, 0xc2, 0xdf, 0x41, 0x56, - 0x7e, 0x8e, 0x3b, 0xda, 0x80, 0x0b, 0x5b, 0x99, 0xbe, 0xf2, 0x5d, 0x42, 0xea, 0x48, 0x82, 0x73, - 0xb7, 0x45, 0x9c, 0x47, 0x1b, 0xf7, 0xb4, 0x4b, 0x6c, 0x12, 0x1b, 0xbe, 0x36, 0xdd, 0x79, 0x3c, - 0x61, 0x94, 0xd4, 0x8a, 0x73, 0x6d, 0xa7, 0x81, 0x3e, 0x86, 0x1c, 0x39, 0x01, 0x36, 0x91, 0x55, - 0x2a, 0x62, 0x23, 0xf2, 0x08, 0x06, 0x6d, 0x67, 0x1a, 0x59, 0xcc, 0x9f, 0xc9, 0x5a, 0xa8, 0xb9, - 0xf4, 0x54, 0x1b, 0x5b, 0x76, 0xe5, 0xfa, 0xf4, 0xb5, 0x10, 0x33, 0x39, 0xa0, 0x94, 0x64, 0x2d, - 0x3d, 0xb7, 0xb5, 0x9b, 0x81, 0xd4, 0xa5, 0xda, 0x1b, 0xe2, 0xa7, 0xc9, 0x6c, 0x5a, 0xcc, 0x3c, - 0x4d, 0x66, 0xb3, 0x62, 0xee, 0x69, 0x32, 0x9b, 0x13, 0x41, 0x7a, 0x1f, 0xf2, 0x3e, 0x2f, 0x8d, - 0x2a, 0x90, 0xe9, 0x63, 0xcb, 0x52, 0x3b, 0x98, 0x3a, 0xf5, 0x9c, 0xec, 0x34, 0xa5, 0x12, 0x14, - 0xfc, 0x9e, 0x59, 0xfa, 0x4a, 0x70, 0x39, 0x89, 0xd3, 0x25, 0x9c, 0x97, 0xd8, 0xa4, 0xba, 0xc1, - 0x39, 0x79, 0x13, 0xbd, 0x0b, 0x45, 0xba, 0x03, 0x8a, 0xf3, 0x9e, 0x78, 0xfe, 0xa4, 0x5c, 0xa0, - 0x9d, 0x67, 0x9c, 0x68, 0x1d, 0xf2, 0xc6, 0xb6, 0xe1, 0x92, 0x24, 0x28, 0x09, 0x18, 0xdb, 0x86, - 0x43, 0xf0, 0x0e, 0x14, 0xc8, 0x5a, 0x5d, 0x8a, 0x24, 0x1d, 0x24, 0x4f, 0xfa, 0x38, 0x89, 0xf4, - 0xcb, 0x38, 0x88, 0xe3, 0xde, 0x1c, 0x3d, 0x84, 0x24, 0x09, 0x6c, 0x3c, 0x46, 0x55, 0x37, 0x59, - 0xd4, 0xdb, 0x74, 0xa2, 0xde, 0x66, 0xd3, 0x89, 0x7a, 0xbb, 0xd9, 0x5f, 0xfc, 0x7a, 0x3d, 0xf6, - 0xd5, 0x7f, 0xae, 0x0b, 0x32, 0xe5, 0x40, 0x37, 0x88, 0x0f, 0x57, 0xb5, 0x81, 0xa2, 0xb5, 0xe9, - 0x94, 0x73, 0xc4, 0x41, 0xab, 0xda, 0x60, 0xbf, 0x8d, 0x0e, 0x40, 0x6c, 0xe9, 0x03, 0x0b, 0x0f, - 0xac, 0xa1, 0xa5, 0xb0, 0x98, 0xcb, 0x23, 0x53, 0x40, 0x45, 0x58, 0xb8, 0xad, 0x39, 0x94, 0xc7, - 0x94, 0x50, 0x2e, 0xb7, 0x82, 0x1d, 0x68, 0x0f, 0xe0, 0x52, 0xed, 0x69, 0x6d, 0xd5, 0xd6, 0x4d, - 0xab, 0x92, 0xdc, 0x48, 0x84, 0xea, 0xc9, 0x99, 0x43, 0x72, 0x6a, 0xb4, 0x55, 0x1b, 0xef, 0x26, - 0xc9, 0x74, 0x65, 0x1f, 0x27, 0x7a, 0x0f, 0xca, 0xaa, 0x61, 0x28, 0x96, 0xad, 0xda, 0x58, 0x39, - 0xbf, 0xb2, 0xb1, 0x45, 0xa3, 0x56, 0x41, 0x2e, 0xaa, 0x86, 0x71, 0x42, 0x7a, 0x77, 0x49, 0x27, - 0xba, 0x05, 0x25, 0x12, 0xe0, 0x34, 0xb5, 0xa7, 0x74, 0xb1, 0xd6, 0xe9, 0xda, 0x95, 0xf4, 0x86, - 0x70, 0x3b, 0x21, 0x17, 0x79, 0x6f, 0x83, 0x76, 0x4a, 0x6d, 0xf7, 0xc4, 0x69, 0x70, 0x43, 0x08, - 0x92, 0x6d, 0xd5, 0x56, 0xe9, 0x4e, 0x16, 0x64, 0xfa, 0x4c, 0xfa, 0x0c, 0xd5, 0xee, 0xf2, 0xfd, - 0xa1, 0xcf, 0x68, 0x15, 0xd2, 0x5c, 0x6c, 0x82, 0x8a, 0xe5, 0x2d, 0xb4, 0x02, 0x29, 0xc3, 0xd4, - 0x2f, 0x31, 0x3d, 0xba, 0xac, 0xcc, 0x1a, 0x92, 0x0c, 0xa5, 0x60, 0x20, 0x44, 0x25, 0x88, 0xdb, - 0x23, 0x3e, 0x4a, 0xdc, 0x1e, 0xa1, 0xef, 0x42, 0x92, 0x6c, 0x24, 0x1d, 0xa3, 0x14, 0x12, 0xfa, - 0x39, 0x5f, 0xf3, 0xca, 0xc0, 0x32, 0xa5, 0x94, 0xca, 0x50, 0x0c, 0x04, 0x48, 0x69, 0x15, 0x56, - 0xc2, 0xe2, 0x9d, 0xd4, 0x75, 0xfb, 0x03, 0x71, 0x0b, 0xdd, 0x87, 0xac, 0x1b, 0xf0, 0x98, 0xe2, - 0xdc, 0x98, 0x18, 0xd6, 0x21, 0x96, 0x5d, 0x52, 0xa2, 0x31, 0xe4, 0x00, 0xba, 0x2a, 0x4f, 0x6f, - 0x0a, 0x72, 0x46, 0x35, 0x8c, 0x86, 0x6a, 0x75, 0xa5, 0x1f, 0x42, 0x25, 0x2a, 0x98, 0xf9, 0x36, - 0x4c, 0xa0, 0x6a, 0xef, 0x6c, 0xd8, 0x2a, 0xa4, 0x2f, 0x74, 0xb3, 0xaf, 0xda, 0x54, 0x58, 0x51, - 0xe6, 0x2d, 0xb2, 0x91, 0x2c, 0xb0, 0x25, 0x68, 0x37, 0x6b, 0x48, 0x0a, 0xdc, 0x88, 0x0c, 0x68, - 0x84, 0x45, 0x1b, 0xb4, 0x31, 0xdb, 0xd6, 0xa2, 0xcc, 0x1a, 0x9e, 0x20, 0x36, 0x59, 0xd6, 0x20, - 0xc3, 0x5a, 0x74, 0xad, 0x54, 0x7e, 0x4e, 0xe6, 0x2d, 0xe9, 0x5f, 0xd3, 0xb0, 0x1a, 0x1e, 0xd6, - 0xd0, 0x06, 0x14, 0xfa, 0xea, 0x48, 0xb1, 0x47, 0x5c, 0xed, 0x04, 0x7a, 0xf0, 0xd0, 0x57, 0x47, - 0xcd, 0x11, 0xd3, 0x39, 0x11, 0x12, 0xf6, 0xc8, 0xaa, 0xc4, 0x37, 0x12, 0xb7, 0x0b, 0x32, 0x79, - 0x44, 0xa7, 0xb0, 0xd4, 0xd3, 0x5b, 0x6a, 0x4f, 0xe9, 0xa9, 0x96, 0xad, 0xf0, 0x7c, 0x87, 0x19, - 0xd1, 0xbb, 0x13, 0x9b, 0xcd, 0x02, 0x14, 0x6e, 0xb3, 0xf3, 0x24, 0x0e, 0x87, 0xeb, 0x7f, 0x99, - 0xca, 0x38, 0x50, 0x9d, 0xa3, 0x46, 0xa7, 0xb0, 0x72, 0x7e, 0xf5, 0xa5, 0x3a, 0xb0, 0xb5, 0x01, - 0x56, 0x26, 0xcc, 0x6a, 0x52, 0x7b, 0x9e, 0x6b, 0xd6, 0x39, 0xee, 0xaa, 0x97, 0x9a, 0x6e, 0x72, - 0x91, 0xcb, 0x2e, 0xff, 0x99, 0x67, 0x5b, 0xde, 0x19, 0xa5, 0x02, 0x4a, 0xed, 0xb8, 0x97, 0xf4, - 0xc2, 0xee, 0xe5, 0xbb, 0xb0, 0x32, 0xc0, 0x23, 0xdb, 0x37, 0x47, 0xa6, 0x38, 0x19, 0x7a, 0x16, - 0x88, 0xbc, 0xf3, 0xc6, 0x27, 0x3a, 0x84, 0x3e, 0xa0, 0x99, 0x82, 0xa1, 0x5b, 0xd8, 0x54, 0xd4, - 0x76, 0xdb, 0xc4, 0x96, 0x55, 0xc9, 0x52, 0xea, 0xb2, 0xd3, 0xbf, 0xc3, 0xba, 0x03, 0x9a, 0x98, - 0x0b, 0x68, 0x22, 0x7a, 0x1f, 0xca, 0xe3, 0x43, 0x02, 0xa5, 0x28, 0x5d, 0x06, 0x87, 0xbb, 0x05, - 0x25, 0xcf, 0xc9, 0x51, 0xba, 0x3c, 0xf3, 0x26, 0x6e, 0x2f, 0x25, 0xbb, 0x09, 0x39, 0xe2, 0x0a, - 0x18, 0x45, 0x81, 0x52, 0x64, 0x49, 0x07, 0x7d, 0xf9, 0x2e, 0x14, 0xf1, 0xa5, 0xd6, 0xc6, 0x83, - 0x16, 0x66, 0x04, 0x45, 0x4a, 0x50, 0x70, 0x3a, 0x29, 0xd1, 0x7b, 0x50, 0xa6, 0x3a, 0xc0, 0xa2, - 0x04, 0x25, 0x2b, 0xb1, 0x91, 0x48, 0x37, 0x8b, 0x8a, 0x84, 0xee, 0x21, 0xdc, 0xf0, 0xd1, 0x19, - 0xaa, 0x69, 0x2b, 0x16, 0xb6, 0x15, 0x5b, 0xb7, 0x79, 0x22, 0x96, 0x90, 0xaf, 0xb9, 0x1c, 0xc7, - 0xaa, 0x69, 0x9f, 0x60, 0xbb, 0x49, 0x5e, 0xa2, 0x07, 0x50, 0x09, 0xe3, 0xa4, 0x43, 0x89, 0x74, - 0xa8, 0x95, 0x71, 0x46, 0x3a, 0xe2, 0x6d, 0x10, 0x7d, 0xda, 0xc9, 0xe8, 0x97, 0xd8, 0x66, 0xf5, - 0x5c, 0x95, 0xa3, 0x94, 0x77, 0x60, 0x89, 0x52, 0x9a, 0xd8, 0x1a, 0xf6, 0x6c, 0xbe, 0x5f, 0x88, - 0x1d, 0x0e, 0x79, 0x21, 0xb3, 0x7e, 0xea, 0x0b, 0xfe, 0xc9, 0x6f, 0x48, 0xc1, 0xb4, 0x8d, 0x9b, - 0x89, 0xe0, 0x99, 0xc9, 0x09, 0xac, 0xf0, 0xc3, 0x6d, 0x07, 0x2c, 0x85, 0x5d, 0x9f, 0x6e, 0x4e, - 0x7a, 0xc3, 0x71, 0x0b, 0x41, 0x0e, 0xfb, 0x1c, 0x46, 0x92, 0x78, 0x3d, 0x23, 0x41, 0x90, 0xa4, - 0xeb, 0x4e, 0xb2, 0x08, 0x41, 0x9e, 0x7f, 0x9f, 0x0d, 0x07, 0x66, 0x1a, 0x4e, 0x7e, 0x4e, 0xc3, - 0x29, 0xcc, 0x34, 0x9c, 0xe2, 0x2c, 0xc3, 0x29, 0xcd, 0x67, 0x38, 0xe5, 0x85, 0x0d, 0x47, 0x7c, - 0x55, 0xc3, 0x59, 0x5a, 0xd0, 0x70, 0xd0, 0xfc, 0x86, 0xb3, 0x1c, 0x6e, 0x38, 0x1f, 0xc3, 0xd2, - 0xc4, 0x85, 0xc5, 0x55, 0x3a, 0x21, 0x54, 0xe9, 0xe2, 0x7e, 0xa5, 0x93, 0xfe, 0x5e, 0x80, 0x6a, - 0xf4, 0x0d, 0x25, 0x54, 0xd4, 0x87, 0xb0, 0xe4, 0x1e, 0xaf, 0xab, 0x3c, 0x2c, 0x5e, 0x8a, 0xee, - 0x0b, 0x47, 0x7b, 0xa2, 0x52, 0x9f, 0x5b, 0x50, 0x1a, 0xbb, 0x3f, 0x31, 0x13, 0x29, 0x5e, 0xfa, - 0xc7, 0x97, 0xfe, 0x31, 0xed, 0xe6, 0x23, 0x81, 0x4b, 0x4e, 0x88, 0x5b, 0xf8, 0x14, 0x96, 0xdb, - 0xb8, 0xa5, 0xb5, 0x5f, 0xd5, 0x2b, 0x2c, 0x71, 0xee, 0x6f, 0x9d, 0xc2, 0xb7, 0x4e, 0xe1, 0xf7, - 0xdb, 0x29, 0xfc, 0x4d, 0xdc, 0xf5, 0x0a, 0xde, 0x65, 0x3e, 0xd4, 0x94, 0x1f, 0x10, 0xad, 0x53, - 0x49, 0x62, 0xcb, 0xcc, 0xa4, 0x32, 0x79, 0x57, 0x6b, 0xd0, 0xf7, 0x5c, 0x9d, 0x39, 0x35, 0x3a, - 0x0a, 0xce, 0xdb, 0x87, 0x43, 0x4e, 0x82, 0x7a, 0x9e, 0x3d, 0xf9, 0x8c, 0xcd, 0xb7, 0x3c, 0x7a, - 0x57, 0x96, 0xa7, 0xe6, 0xa8, 0x93, 0x57, 0x8d, 0x3a, 0x3f, 0xdf, 0x29, 0x66, 0x26, 0xd5, 0xdd, - 0xbb, 0xaf, 0x0b, 0x46, 0x4c, 0xdc, 0xa4, 0xde, 0x81, 0x82, 0xa5, 0x75, 0x14, 0x8a, 0xc2, 0x68, - 0x98, 0xdd, 0x6a, 0xb3, 0x72, 0xde, 0xd2, 0x3a, 0x67, 0xbc, 0x4b, 0xfa, 0x00, 0xca, 0x63, 0x80, - 0xc4, 0xd8, 0xf5, 0xc4, 0x73, 0xa6, 0xcb, 0xee, 0xbe, 0x7b, 0xc0, 0x83, 0xf4, 0xf3, 0x02, 0x64, - 0x65, 0x6c, 0x19, 0x44, 0xa9, 0xd1, 0x2e, 0xe4, 0xf0, 0xa8, 0x85, 0x0d, 0xdb, 0x41, 0x05, 0xc2, - 0xc1, 0x0b, 0x46, 0x5d, 0x77, 0x28, 0x1b, 0x31, 0xd9, 0x63, 0x43, 0xf7, 0x38, 0xc6, 0x1c, 0x0d, - 0x17, 0x73, 0x76, 0x3f, 0xc8, 0xfc, 0xc0, 0x01, 0x99, 0x13, 0x91, 0xf8, 0x29, 0xe3, 0x1a, 0x43, - 0x99, 0xef, 0x71, 0x94, 0x39, 0x39, 0x63, 0xb0, 0x00, 0xcc, 0x5c, 0x0b, 0xc0, 0xcc, 0xa9, 0x19, - 0xcb, 0x8c, 0xc0, 0x99, 0x1f, 0x38, 0x38, 0x73, 0x7a, 0xc6, 0x8c, 0xc7, 0x80, 0xe6, 0xef, 0xfb, - 0x80, 0xe6, 0x6c, 0x24, 0xc2, 0xc4, 0x58, 0x43, 0x90, 0xe6, 0x47, 0x2e, 0xd2, 0x9c, 0x8f, 0x44, - 0xa9, 0x39, 0xf3, 0x38, 0xd4, 0x7c, 0x34, 0x01, 0x35, 0x33, 0x68, 0xf8, 0xbd, 0x48, 0x11, 0x33, - 0xb0, 0xe6, 0xa3, 0x09, 0xac, 0xb9, 0x38, 0x43, 0xe0, 0x0c, 0xb0, 0xf9, 0x2f, 0xc2, 0xc1, 0xe6, - 0x68, 0x38, 0x98, 0x4f, 0x73, 0x3e, 0xb4, 0x59, 0x89, 0x40, 0x9b, 0xcb, 0x91, 0xc8, 0x28, 0x13, - 0x3f, 0x37, 0xdc, 0x7c, 0x1a, 0x02, 0x37, 0x33, 0x60, 0xf8, 0x76, 0xa4, 0xf0, 0x39, 0xf0, 0xe6, - 0xd3, 0x10, 0xbc, 0x79, 0x69, 0xa6, 0xd8, 0x99, 0x80, 0xf3, 0x5e, 0x10, 0x70, 0x46, 0x11, 0x17, - 0x79, 0xcf, 0xda, 0x23, 0x10, 0xe7, 0xf3, 0x28, 0xc4, 0x99, 0xa1, 0xc2, 0x77, 0x23, 0x25, 0x2e, - 0x00, 0x39, 0x1f, 0x4d, 0x40, 0xce, 0x2b, 0x33, 0x34, 0x6d, 0x06, 0xe6, 0xbc, 0x17, 0xc4, 0x9c, - 0xaf, 0xcd, 0x58, 0x7c, 0x24, 0xe8, 0x5c, 0x0b, 0x80, 0xce, 0xab, 0x33, 0x5c, 0x49, 0x04, 0xea, - 0xfc, 0x27, 0x7e, 0xd4, 0xf9, 0x7a, 0x24, 0x70, 0xcd, 0xcf, 0x21, 0x0c, 0x76, 0xde, 0x0b, 0xc2, - 0xce, 0x95, 0x19, 0xcb, 0x99, 0x07, 0x77, 0xce, 0x88, 0x59, 0x86, 0x38, 0x3f, 0x4d, 0x66, 0x41, - 0xcc, 0x4b, 0x1f, 0x90, 0xe0, 0x31, 0xe6, 0xf8, 0xd1, 0x0a, 0xa4, 0xb0, 0x69, 0xea, 0x26, 0x47, - 0x90, 0x59, 0x43, 0xba, 0x0d, 0x05, 0xbf, 0x93, 0x9f, 0x82, 0x51, 0x53, 0xdc, 0xcf, 0xe7, 0xd8, - 0xa5, 0xff, 0x16, 0x3c, 0x5e, 0x1a, 0x79, 0xfd, 0x18, 0x66, 0x8e, 0x63, 0x98, 0x3e, 0xe4, 0x3a, - 0x1e, 0x44, 0xae, 0xd7, 0x21, 0x4f, 0xf2, 0xbe, 0x31, 0x50, 0x5a, 0x35, 0x5c, 0x50, 0xda, 0xc9, - 0x53, 0x78, 0xae, 0xc5, 0xa2, 0x64, 0x92, 0x46, 0xc9, 0xb2, 0x97, 0x6d, 0xb1, 0xdc, 0xf6, 0x3b, - 0xb0, 0xec, 0xa3, 0x75, 0xf3, 0x49, 0x86, 0xd0, 0x8a, 0x2e, 0xf5, 0x0e, 0x4f, 0x2c, 0xef, 0x02, - 0xea, 0x6b, 0x03, 0xad, 0x3f, 0xec, 0x2b, 0x1d, 0x95, 0x98, 0xaa, 0xd6, 0xc2, 0x16, 0x8d, 0x0e, - 0x39, 0x59, 0xe4, 0x6f, 0x3e, 0x51, 0xad, 0x63, 0xda, 0x2f, 0xfd, 0x8b, 0xe0, 0xed, 0xa7, 0x87, - 0x7d, 0x87, 0xc1, 0xd4, 0xc2, 0x1b, 0x82, 0xa9, 0xe3, 0xaf, 0x0c, 0x53, 0xfb, 0xb3, 0xe9, 0x44, - 0x10, 0x25, 0xfd, 0x1f, 0xc1, 0x3b, 0x41, 0x17, 0x74, 0x6e, 0xe9, 0x6d, 0xcc, 0x71, 0x4b, 0xfa, - 0x4c, 0x6e, 0x43, 0x3d, 0xbd, 0xc3, 0xd1, 0x49, 0xf2, 0x48, 0xa8, 0xdc, 0xb8, 0x9d, 0xe3, 0x61, - 0xd9, 0x85, 0x3c, 0xd9, 0x15, 0x83, 0x43, 0x9e, 0x22, 0x24, 0x5e, 0x62, 0x16, 0x65, 0x0b, 0x32, - 0x79, 0x24, 0x74, 0x54, 0x49, 0xf9, 0x55, 0x81, 0x35, 0xd0, 0x43, 0xc8, 0xd1, 0x3a, 0xbc, 0xa2, - 0x1b, 0x16, 0x0f, 0xac, 0x81, 0x5b, 0x15, 0x2b, 0xc6, 0x6f, 0x1e, 0x13, 0x9a, 0x23, 0xc3, 0x92, - 0xb3, 0x06, 0x7f, 0xf2, 0xa5, 0x4b, 0xb9, 0xc0, 0xdd, 0xe6, 0x2d, 0xc8, 0x91, 0xd9, 0x5b, 0x86, - 0xda, 0xc2, 0xf4, 0x16, 0x91, 0x93, 0xbd, 0x0e, 0xe9, 0x05, 0xa0, 0x49, 0xef, 0x80, 0x1a, 0x90, - 0xc6, 0x97, 0x78, 0x60, 0xb3, 0xab, 0x5f, 0x7e, 0x7b, 0x35, 0x24, 0x35, 0xc4, 0x03, 0x7b, 0xb7, - 0x42, 0x36, 0xf9, 0x37, 0xbf, 0x5e, 0x17, 0x19, 0xf5, 0x5d, 0xbd, 0xaf, 0xd9, 0xb8, 0x6f, 0xd8, - 0x57, 0x32, 0xe7, 0x97, 0xfe, 0x4d, 0x20, 0x89, 0x5d, 0x20, 0x0f, 0x08, 0xdd, 0x5b, 0xc7, 0x40, - 0xe2, 0x3e, 0x90, 0x7f, 0x72, 0xbf, 0xdf, 0x06, 0x20, 0x4a, 0xf9, 0x85, 0x3a, 0xb0, 0x71, 0x9b, - 0x6f, 0x70, 0xae, 0xa3, 0x5a, 0x9f, 0xd1, 0x8e, 0xe0, 0x52, 0xb3, 0x63, 0x4b, 0xf5, 0xe1, 0xcb, - 0x39, 0x3f, 0xbe, 0x8c, 0xaa, 0x90, 0x35, 0x4c, 0x4d, 0x37, 0x35, 0xfb, 0x8a, 0xee, 0x4f, 0x42, - 0x76, 0xdb, 0x4f, 0x93, 0xd9, 0xa4, 0x98, 0x72, 0xcb, 0x57, 0xcc, 0x99, 0xe4, 0xc5, 0x82, 0xf4, - 0x93, 0xb8, 0xa7, 0xf7, 0x5e, 0xde, 0xfb, 0xea, 0x0b, 0x0b, 0x53, 0xa4, 0xb5, 0x90, 0xc5, 0xfa, - 0x7a, 0xc8, 0xbc, 0x49, 0x6b, 0x68, 0xe1, 0x36, 0x2f, 0xa4, 0xb8, 0x6d, 0xdf, 0x01, 0x66, 0x5e, - 0xef, 0x00, 0xa7, 0xef, 0xa9, 0xf4, 0xb7, 0xb4, 0xf4, 0x15, 0x74, 0xe9, 0xe8, 0xc4, 0x0f, 0x5d, - 0x0c, 0xa9, 0x39, 0x3a, 0x8a, 0x34, 0xaf, 0xdd, 0x7a, 0x10, 0x07, 0xeb, 0xb6, 0xd0, 0x9f, 0xc2, - 0xf5, 0x31, 0x9f, 0xe2, 0x8a, 0x8e, 0x47, 0xe4, 0x9f, 0xe3, 0x9e, 0xe5, 0x5a, 0xd0, 0xb3, 0x38, - 0x92, 0xbd, 0xbd, 0x4a, 0xbc, 0xa6, 0xb2, 0xdf, 0x87, 0x52, 0x30, 0x6d, 0x25, 0x37, 0x67, 0x13, - 0xdb, 0xaa, 0x36, 0x50, 0x02, 0xf8, 0x4c, 0x81, 0x75, 0xf2, 0x82, 0xd7, 0x31, 0x5c, 0x0b, 0x4d, - 0x55, 0xd1, 0xf7, 0x20, 0xe7, 0x65, 0xb9, 0x42, 0xc4, 0x25, 0xcd, 0xad, 0x07, 0x79, 0xb4, 0xd2, - 0x3f, 0x0b, 0x9e, 0xc8, 0x60, 0x85, 0xa9, 0x0e, 0x69, 0x76, 0xb9, 0xa5, 0x4a, 0x5a, 0xda, 0xfe, - 0xce, 0x7c, 0x49, 0xee, 0x26, 0xbb, 0xf9, 0xca, 0x9c, 0x59, 0x7a, 0x01, 0x69, 0xd6, 0x83, 0xf2, - 0x90, 0x39, 0x3d, 0x7c, 0x76, 0x78, 0xf4, 0xd9, 0xa1, 0x18, 0x43, 0x00, 0xe9, 0x9d, 0x5a, 0xad, - 0x7e, 0xdc, 0x14, 0x05, 0x94, 0x83, 0xd4, 0xce, 0xee, 0x91, 0xdc, 0x14, 0xe3, 0xa4, 0x5b, 0xae, - 0x3f, 0xad, 0xd7, 0x9a, 0x62, 0x02, 0x2d, 0x41, 0x91, 0x3d, 0x2b, 0x7b, 0x47, 0xf2, 0xf3, 0x9d, - 0xa6, 0x98, 0xf4, 0x75, 0x9d, 0xd4, 0x0f, 0x9f, 0xd4, 0x65, 0x31, 0x25, 0xfd, 0x01, 0xdc, 0x88, - 0x4c, 0x8b, 0xbd, 0xf2, 0x91, 0xe0, 0x2b, 0x1f, 0x49, 0x7f, 0x17, 0x87, 0x6a, 0x74, 0xae, 0x8b, - 0x9e, 0x8e, 0x2d, 0x7c, 0x7b, 0x81, 0x44, 0x79, 0x6c, 0xf5, 0xe8, 0x16, 0x94, 0x4c, 0x7c, 0x81, - 0xed, 0x56, 0x97, 0xe5, 0xde, 0x2c, 0x2a, 0x15, 0xe5, 0x22, 0xef, 0xa5, 0x4c, 0x16, 0x23, 0xfb, - 0x11, 0x6e, 0xd9, 0x0a, 0xf3, 0x34, 0x4c, 0xc1, 0x72, 0x84, 0x8c, 0xf4, 0x9e, 0xb0, 0x4e, 0xe9, - 0x87, 0x0b, 0xed, 0x65, 0x0e, 0x52, 0x72, 0xbd, 0x29, 0x7f, 0x2e, 0x26, 0x10, 0x82, 0x12, 0x7d, - 0x54, 0x4e, 0x0e, 0x77, 0x8e, 0x4f, 0x1a, 0x47, 0x64, 0x2f, 0x97, 0xa1, 0xec, 0xec, 0xa5, 0xd3, - 0x99, 0x92, 0xfe, 0x3d, 0x0e, 0xd7, 0x23, 0x32, 0x75, 0xf4, 0x10, 0xc0, 0x1e, 0x29, 0x26, 0x6e, - 0xe9, 0x66, 0x3b, 0x5a, 0xc9, 0x9a, 0x23, 0x99, 0x52, 0xc8, 0x39, 0x9b, 0x3f, 0x59, 0x53, 0xaa, - 0x8e, 0xe8, 0x23, 0x2e, 0x94, 0xa2, 0x25, 0xdc, 0xac, 0xde, 0x0e, 0x29, 0xae, 0xe1, 0x16, 0x11, - 0x4c, 0xf7, 0x96, 0x0a, 0xa6, 0xf4, 0xe8, 0x79, 0x98, 0xff, 0x98, 0xb3, 0x3c, 0x1d, 0xe2, 0x39, - 0x3e, 0x8f, 0xf6, 0x1c, 0xa9, 0x79, 0x93, 0x92, 0x70, 0xd7, 0x21, 0xfd, 0x43, 0xc2, 0xbf, 0xb1, - 0xc1, 0x8b, 0xc9, 0x11, 0xa4, 0x2d, 0x5b, 0xb5, 0x87, 0x16, 0x57, 0xb8, 0xef, 0xcd, 0x7b, 0xcb, - 0xd9, 0x74, 0x1e, 0x4e, 0x28, 0xbb, 0xcc, 0xc5, 0x7c, 0xbb, 0xdf, 0xd4, 0xc1, 0x06, 0x37, 0x27, - 0xda, 0x64, 0x3c, 0x9f, 0x13, 0x97, 0x1e, 0x7b, 0x49, 0x8e, 0x0f, 0xc0, 0x9f, 0x04, 0xc7, 0x85, - 0x30, 0x70, 0xfc, 0xe7, 0x02, 0xdc, 0x9c, 0x72, 0xd7, 0x43, 0x9f, 0x8e, 0x9d, 0xf3, 0xa3, 0x45, - 0x6e, 0x8a, 0x9b, 0xac, 0x2f, 0x78, 0xd2, 0xd2, 0x3d, 0x28, 0xf8, 0xfb, 0xe7, 0x5b, 0xe4, 0x6f, - 0xe2, 0x9e, 0xcf, 0x0f, 0xa2, 0xf8, 0x6f, 0x2c, 0x9b, 0x1b, 0xd3, 0xb3, 0xf8, 0x82, 0x7a, 0x16, - 0x9a, 0x17, 0x24, 0x5e, 0x33, 0x2f, 0x98, 0xa2, 0x6d, 0xc9, 0xd7, 0xd3, 0xb6, 0x80, 0xc1, 0xa5, - 0x82, 0x17, 0x86, 0x15, 0x4f, 0xa3, 0x7c, 0x20, 0xe4, 0xe7, 0x00, 0x3e, 0xb4, 0x75, 0x05, 0x52, - 0xa6, 0x3e, 0x1c, 0xb4, 0xa9, 0x5e, 0xa4, 0x64, 0xd6, 0x40, 0xf7, 0x21, 0x45, 0xf4, 0xcb, 0xd9, - 0xbd, 0x49, 0x57, 0x4b, 0xf4, 0xc3, 0x87, 0xe1, 0x32, 0x6a, 0xe9, 0x07, 0x50, 0x0a, 0x42, 0xbc, - 0x6f, 0x56, 0xbc, 0x06, 0x68, 0xf2, 0x53, 0x87, 0x88, 0x21, 0xbe, 0x1f, 0x1c, 0xe2, 0x9d, 0xc8, - 0x8f, 0x26, 0xc2, 0x87, 0xfa, 0x12, 0x52, 0x54, 0xdd, 0x48, 0xce, 0x4b, 0xbf, 0xaf, 0xe1, 0x77, - 0x62, 0xf2, 0x8c, 0x7e, 0x00, 0xa0, 0xda, 0xb6, 0xa9, 0x9d, 0x0f, 0xbd, 0x01, 0xd6, 0xc3, 0xd5, - 0x75, 0xc7, 0xa1, 0xdb, 0x7d, 0x8b, 0xeb, 0xed, 0x8a, 0xc7, 0xea, 0xd3, 0x5d, 0x9f, 0x40, 0xe9, - 0x10, 0x4a, 0x41, 0x5e, 0xe7, 0x5e, 0x26, 0x84, 0xdc, 0xcb, 0xe2, 0xfe, 0x7b, 0x99, 0x7b, 0xab, - 0x4b, 0xb0, 0x8f, 0x88, 0x68, 0x43, 0xfa, 0x5f, 0x01, 0x0a, 0x7e, 0x6d, 0x7f, 0xc3, 0x37, 0x80, - 0x19, 0xd7, 0x9d, 0x1b, 0x13, 0x17, 0x80, 0x4c, 0x47, 0xb5, 0x4e, 0x7f, 0x97, 0xf9, 0xff, 0x4f, - 0x04, 0xc8, 0xba, 0x8b, 0x8f, 0x00, 0xec, 0xbd, 0xbd, 0x8b, 0xfb, 0x3f, 0x02, 0x62, 0x45, 0x82, - 0x84, 0x5b, 0x24, 0x78, 0xec, 0x26, 0x68, 0x51, 0x28, 0xb8, 0x7f, 0xa7, 0x9d, 0x52, 0x09, 0xcf, - 0x47, 0x4d, 0x36, 0x0d, 0x92, 0x98, 0xa0, 0x3f, 0x82, 0xb4, 0xda, 0x72, 0xa1, 0xff, 0x52, 0x08, - 0x90, 0xe5, 0x90, 0x6e, 0x36, 0x47, 0x3b, 0x94, 0x52, 0xe6, 0x1c, 0x7c, 0x52, 0x71, 0x67, 0x52, - 0x52, 0x95, 0xc8, 0xdd, 0x71, 0xde, 0xc1, 0xe9, 0xe1, 0xf3, 0xa3, 0x27, 0xfb, 0x7b, 0xfb, 0xf5, - 0x27, 0x62, 0x4c, 0xaa, 0x41, 0xde, 0x29, 0x35, 0xa9, 0x7d, 0x0b, 0xdd, 0x84, 0x5c, 0x5f, 0x0d, - 0x7e, 0x88, 0x94, 0xed, 0xab, 0xfc, 0x33, 0xa4, 0xeb, 0x90, 0x21, 0x2f, 0x3b, 0xaa, 0xe5, 0x54, - 0x86, 0xfb, 0xea, 0xe8, 0x13, 0xd5, 0x92, 0x7e, 0x2b, 0x40, 0x79, 0xcc, 0x1d, 0xa1, 0x6d, 0x48, - 0x31, 0x10, 0x2d, 0xea, 0xfb, 0x76, 0xdf, 0xb0, 0x32, 0x23, 0x45, 0x1f, 0x41, 0xd6, 0xa9, 0xc6, - 0x85, 0xdd, 0x87, 0x98, 0xdf, 0x73, 0xea, 0x39, 0x9c, 0xd5, 0xe5, 0x40, 0x1f, 0x43, 0xce, 0x75, - 0xac, 0xd1, 0x1f, 0x14, 0xba, 0x2e, 0x99, 0xf3, 0x7b, 0x3c, 0xe8, 0x91, 0x87, 0x65, 0x25, 0x27, - 0x01, 0x7d, 0xce, 0xce, 0x08, 0x38, 0xb3, 0x43, 0x2f, 0x3d, 0x86, 0x9c, 0x2b, 0x18, 0x55, 0x20, - 0xe3, 0xd4, 0x44, 0x05, 0xee, 0x71, 0x79, 0x2d, 0x74, 0x05, 0x52, 0x86, 0xfe, 0x05, 0xff, 0x38, - 0x2c, 0x21, 0xb3, 0x86, 0xd4, 0x86, 0xf2, 0x58, 0xa0, 0x40, 0x8f, 0x21, 0x63, 0x0c, 0xcf, 0x15, - 0xc7, 0xaa, 0xc7, 0xf6, 0xcf, 0xc1, 0x4f, 0x86, 0xe7, 0x3d, 0xad, 0xf5, 0x0c, 0x5f, 0x39, 0x7a, - 0x64, 0x0c, 0xcf, 0x9f, 0x31, 0xe3, 0x67, 0xa3, 0xc4, 0xfd, 0xa3, 0x5c, 0x42, 0xd6, 0xf1, 0x65, - 0xe8, 0x8f, 0xfd, 0x5b, 0xe5, 0x7c, 0xdc, 0x19, 0x19, 0xbc, 0xb8, 0x78, 0xdf, 0x4e, 0xdd, 0x81, - 0x25, 0x4b, 0xeb, 0x0c, 0x9c, 0xfa, 0x39, 0x3b, 0x68, 0x56, 0x10, 0x2b, 0xb3, 0x17, 0x07, 0x0e, - 0x24, 0x47, 0x52, 0x0f, 0x71, 0xdc, 0x99, 0xfe, 0x2e, 0x27, 0x10, 0x92, 0x22, 0x25, 0xc2, 0x52, - 0xa4, 0xbf, 0x8e, 0x43, 0xde, 0x57, 0x95, 0x47, 0x7f, 0xe8, 0xf3, 0xec, 0xa5, 0x90, 0xd8, 0xee, - 0xa3, 0xf5, 0xbe, 0x9e, 0x0c, 0x2e, 0x2c, 0xbe, 0xf8, 0xc2, 0xa2, 0x3e, 0x82, 0x70, 0x8a, 0xfb, - 0xc9, 0x85, 0x8b, 0xfb, 0x77, 0x01, 0xd1, 0xb2, 0xb4, 0x72, 0xa9, 0xdb, 0xda, 0xa0, 0xa3, 0x30, - 0xd5, 0x60, 0x7e, 0x58, 0xa4, 0x6f, 0xce, 0xe8, 0x8b, 0x63, 0xaa, 0x25, 0x7f, 0x19, 0x87, 0xac, - 0x63, 0x61, 0xff, 0x4f, 0xb7, 0xe0, 0xaf, 0x04, 0xc8, 0xba, 0x50, 0xc3, 0xa2, 0x9f, 0x97, 0xae, - 0x42, 0x9a, 0xdf, 0xa6, 0xd9, 0xf7, 0xa5, 0xbc, 0x15, 0xfa, 0x21, 0x47, 0x15, 0xb2, 0x7d, 0x6c, - 0xab, 0x34, 0xae, 0xb2, 0xd4, 0xcc, 0x6d, 0xdf, 0x79, 0x04, 0x79, 0xdf, 0xa7, 0xb9, 0x24, 0xd4, - 0x1e, 0xd6, 0x3f, 0x13, 0x63, 0xd5, 0xcc, 0x4f, 0x7f, 0xb6, 0x91, 0x38, 0xc4, 0x5f, 0x10, 0x27, - 0x23, 0xd7, 0x6b, 0x8d, 0x7a, 0xed, 0x99, 0x28, 0x54, 0xf3, 0x3f, 0xfd, 0xd9, 0x46, 0x46, 0xc6, - 0xb4, 0x30, 0x79, 0xe7, 0x19, 0x94, 0xc7, 0x0e, 0x26, 0x98, 0x7b, 0x23, 0x28, 0x3d, 0x39, 0x3d, - 0x3e, 0xd8, 0xaf, 0xed, 0x34, 0xeb, 0xca, 0xd9, 0x51, 0xb3, 0x2e, 0x0a, 0xe8, 0x3a, 0x2c, 0x1f, - 0xec, 0x7f, 0xd2, 0x68, 0x2a, 0xb5, 0x83, 0xfd, 0xfa, 0x61, 0x53, 0xd9, 0x69, 0x36, 0x77, 0x6a, - 0xcf, 0xc4, 0xf8, 0xf6, 0x6f, 0xf3, 0x50, 0xde, 0xd9, 0xad, 0xed, 0xef, 0x18, 0x46, 0x4f, 0x6b, - 0xa9, 0x34, 0x86, 0xd4, 0x20, 0x49, 0x6b, 0x09, 0x53, 0xff, 0x59, 0xaa, 0x4e, 0xaf, 0x36, 0xa3, - 0x3d, 0x48, 0xd1, 0x32, 0x03, 0x9a, 0xfe, 0x13, 0x53, 0x75, 0x46, 0xf9, 0x99, 0x4c, 0x86, 0x7a, - 0x94, 0xa9, 0x7f, 0x35, 0x55, 0xa7, 0x57, 0xa3, 0xd1, 0x01, 0x64, 0x1c, 0x5c, 0x77, 0xd6, 0xaf, - 0x46, 0xd5, 0x99, 0x25, 0x62, 0xb2, 0x34, 0x86, 0xbf, 0x4f, 0xff, 0xe1, 0xa9, 0x3a, 0xa3, 0x4e, - 0x8d, 0xf6, 0x21, 0xcd, 0x11, 0xb8, 0x19, 0xff, 0x30, 0x55, 0x67, 0x55, 0x9e, 0x91, 0x0c, 0x39, - 0xaf, 0xb2, 0x31, 0xfb, 0x37, 0xae, 0xea, 0x1c, 0x25, 0x78, 0xf4, 0x02, 0x8a, 0x41, 0xa4, 0x6f, - 0xbe, 0xff, 0xa4, 0xaa, 0x73, 0xd6, 0xb8, 0x89, 0xfc, 0x20, 0xec, 0x37, 0xdf, 0x7f, 0x53, 0xd5, - 0x39, 0x4b, 0xde, 0xe8, 0x47, 0xb0, 0x34, 0x09, 0xcb, 0xcd, 0xff, 0x1b, 0x55, 0x75, 0x81, 0x22, - 0x38, 0xea, 0x03, 0x0a, 0x81, 0xf3, 0x16, 0xf8, 0xab, 0xaa, 0xba, 0x48, 0x4d, 0x1c, 0xb5, 0xa1, - 0x3c, 0x0e, 0x91, 0xcd, 0xfb, 0x97, 0x55, 0x75, 0xee, 0xfa, 0x38, 0x1b, 0x25, 0x88, 0x17, 0xcd, - 0xfb, 0xd7, 0x55, 0x75, 0xee, 0x72, 0x39, 0x3a, 0x05, 0xf0, 0xe1, 0x1d, 0x73, 0xfc, 0x85, 0x55, - 0x9d, 0xa7, 0x70, 0x8e, 0x0c, 0x58, 0x0e, 0x03, 0x42, 0x16, 0xf9, 0x29, 0xab, 0xba, 0x50, 0x3d, - 0x9d, 0xe8, 0x73, 0x10, 0xd2, 0x98, 0xef, 0x27, 0xad, 0xea, 0x9c, 0x85, 0x75, 0xb2, 0x51, 0xde, - 0x35, 0x1e, 0xcd, 0xf1, 0xa3, 0x53, 0x75, 0x9e, 0xaa, 0xf4, 0x6e, 0xfd, 0x17, 0x5f, 0xaf, 0x09, - 0xbf, 0xfa, 0x7a, 0x4d, 0xf8, 0xaf, 0xaf, 0xd7, 0x84, 0xaf, 0xbe, 0x59, 0x8b, 0xfd, 0xea, 0x9b, - 0xb5, 0xd8, 0x7f, 0x7c, 0xb3, 0x16, 0xfb, 0xb3, 0x0f, 0x3b, 0x9a, 0xdd, 0x1d, 0x9e, 0x6f, 0xb6, - 0xf4, 0xfe, 0x96, 0xff, 0x6f, 0xd9, 0xb0, 0x5f, 0x78, 0xcf, 0xd3, 0x34, 0x4e, 0xdf, 0xfb, 0xbf, - 0x00, 0x00, 0x00, 0xff, 0xff, 0xb0, 0x2f, 0x03, 0x74, 0xe2, 0x3b, 0x00, 0x00, + // 3945 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xec, 0x5b, 0xcd, 0x93, 0xe3, 0xc6, + 0x75, 0x27, 0xf8, 0x8d, 0xc7, 0x2f, 0x4c, 0xcf, 0xec, 0x2e, 0x97, 0x2b, 0xed, 0xae, 0xa0, 0xac, + 0xb4, 0x5a, 0xc9, 0x33, 0xce, 0x28, 0x92, 0x57, 0x96, 0x1c, 0x85, 0xc3, 0xe5, 0x88, 0xb3, 0x3b, + 0x9a, 0x19, 0x61, 0x38, 0xa3, 0x28, 0x89, 0x05, 0x63, 0xc8, 0x1e, 0x12, 0x5e, 0x92, 0x80, 0x01, + 0x90, 0xe2, 0xe8, 0x94, 0xca, 0xc7, 0xc5, 0xb9, 0xe8, 0x98, 0x43, 0x7c, 0x8b, 0xff, 0x81, 0x1c, + 0x72, 0xc8, 0x21, 0x55, 0xa9, 0x4a, 0xa5, 0x7c, 0xf0, 0xc1, 0x87, 0x54, 0x2a, 0x27, 0x27, 0x25, + 0x55, 0xe5, 0xe0, 0x7f, 0x20, 0x47, 0xa7, 0xfa, 0x03, 0x5f, 0x24, 0xc0, 0x0f, 0xed, 0x56, 0xaa, + 0x54, 0xd6, 0x0d, 0xdd, 0x78, 0xef, 0x35, 0xba, 0xfb, 0xbd, 0xd7, 0xef, 0xfd, 0x5e, 0x03, 0x6e, + 0x39, 0x78, 0xd4, 0xc5, 0xd6, 0x50, 0x1f, 0x39, 0x3b, 0xda, 0x45, 0x47, 0xdf, 0x71, 0xae, 0x4c, + 0x6c, 0x6f, 0x9b, 0x96, 0xe1, 0x18, 0xa8, 0xe2, 0xbf, 0xdc, 0x26, 0x2f, 0x6b, 0x2f, 0x06, 0xa8, + 0x3b, 0xd6, 0x95, 0xe9, 0x18, 0x3b, 0xa6, 0x65, 0x18, 0x97, 0x8c, 0xbe, 0xf6, 0xc2, 0xfc, 0xeb, + 0xa7, 0xf8, 0x8a, 0x4b, 0x0b, 0x31, 0xd3, 0x51, 0x76, 0x4c, 0xcd, 0xd2, 0x86, 0x76, 0x04, 0x33, + 0x7b, 0x1d, 0xf8, 0x94, 0xda, 0x9d, 0x9e, 0x61, 0xf4, 0x06, 0x78, 0x87, 0xb6, 0x2e, 0xc6, 0x97, + 0x3b, 0x8e, 0x3e, 0xc4, 0xb6, 0xa3, 0x0d, 0x4d, 0x4e, 0xb0, 0xd5, 0x33, 0x7a, 0x06, 0x7d, 0xdc, + 0x21, 0x4f, 0xac, 0x57, 0xfe, 0x65, 0x01, 0x72, 0x0a, 0xfe, 0xc9, 0x18, 0xdb, 0x0e, 0xda, 0x85, + 0x34, 0xee, 0xf4, 0x8d, 0xaa, 0x70, 0x57, 0xb8, 0x5f, 0xd8, 0x7d, 0x61, 0x7b, 0x66, 0x72, 0xdb, + 0x9c, 0xae, 0xd9, 0xe9, 0x1b, 0xad, 0x84, 0x42, 0x69, 0xd1, 0x5b, 0x90, 0xb9, 0x1c, 0x8c, 0xed, + 0x7e, 0x35, 0x49, 0x99, 0x5e, 0x8c, 0x63, 0xda, 0x27, 0x44, 0xad, 0x84, 0xc2, 0xa8, 0xc9, 0x50, + 0xfa, 0xe8, 0xd2, 0xa8, 0xa6, 0x16, 0x0f, 0x75, 0x30, 0xba, 0xa4, 0x43, 0x11, 0x5a, 0xb4, 0x07, + 0xa0, 0x8f, 0x74, 0x47, 0xed, 0xf4, 0x35, 0x7d, 0x54, 0x4d, 0x53, 0xce, 0x97, 0xe2, 0x39, 0x75, + 0xa7, 0x41, 0x08, 0x5b, 0x09, 0x45, 0xd4, 0xdd, 0x06, 0xf9, 0xdc, 0x9f, 0x8c, 0xb1, 0x75, 0x55, + 0xcd, 0x2c, 0xfe, 0xdc, 0x8f, 0x08, 0x11, 0xf9, 0x5c, 0x4a, 0x8d, 0xde, 0x83, 0x7c, 0xa7, 0x8f, + 0x3b, 0x4f, 0x55, 0x67, 0x5a, 0xcd, 0x51, 0xce, 0x3b, 0x71, 0x9c, 0x0d, 0x42, 0xd7, 0x9e, 0xb6, + 0x12, 0x4a, 0xae, 0xc3, 0x1e, 0xd1, 0x43, 0xc8, 0x76, 0x8c, 0xe1, 0x50, 0x77, 0xaa, 0x40, 0x79, + 0x6f, 0xc7, 0xf2, 0x52, 0xaa, 0x56, 0x42, 0xe1, 0xf4, 0xe8, 0x08, 0xca, 0x03, 0xdd, 0x76, 0x54, + 0x7b, 0xa4, 0x99, 0x76, 0xdf, 0x70, 0xec, 0x6a, 0x81, 0x4a, 0xb8, 0x17, 0x27, 0xe1, 0x50, 0xb7, + 0x9d, 0x53, 0x97, 0xb8, 0x95, 0x50, 0x4a, 0x83, 0x60, 0x07, 0x91, 0x67, 0x5c, 0x5e, 0x62, 0xcb, + 0x13, 0x58, 0x2d, 0x2e, 0x96, 0x77, 0x4c, 0xa8, 0x5d, 0x7e, 0x22, 0xcf, 0x08, 0x76, 0xa0, 0x3f, + 0x85, 0xcd, 0x81, 0xa1, 0x75, 0x3d, 0x71, 0x6a, 0xa7, 0x3f, 0x1e, 0x3d, 0xad, 0x96, 0xa8, 0xd0, + 0xd7, 0x62, 0x3f, 0xd2, 0xd0, 0xba, 0xae, 0x88, 0x06, 0x61, 0x68, 0x25, 0x94, 0x8d, 0xc1, 0x6c, + 0x27, 0xfa, 0x14, 0xb6, 0x34, 0xd3, 0x1c, 0x5c, 0xcd, 0x4a, 0x2f, 0x53, 0xe9, 0x0f, 0xe2, 0xa4, + 0xd7, 0x09, 0xcf, 0xac, 0x78, 0xa4, 0xcd, 0xf5, 0xa2, 0x36, 0x48, 0xa6, 0x85, 0x4d, 0xcd, 0xc2, + 0xaa, 0x69, 0x19, 0xa6, 0x61, 0x6b, 0x83, 0x6a, 0x85, 0xca, 0x7e, 0x35, 0x4e, 0xf6, 0x09, 0xa3, + 0x3f, 0xe1, 0xe4, 0xad, 0x84, 0x52, 0x31, 0xc3, 0x5d, 0x4c, 0xaa, 0xd1, 0xc1, 0xb6, 0xed, 0x4b, + 0x95, 0x96, 0x49, 0xa5, 0xf4, 0x61, 0xa9, 0xa1, 0x2e, 0xd4, 0x84, 0x02, 0x9e, 0x12, 0x76, 0x75, + 0x62, 0x38, 0xb8, 0xba, 0x41, 0x05, 0xca, 0xb1, 0x16, 0x4a, 0x49, 0xcf, 0x0d, 0x07, 0xb7, 0x12, + 0x0a, 0x60, 0xaf, 0x85, 0x34, 0xb8, 0x36, 0xc1, 0x96, 0x7e, 0x79, 0x45, 0xc5, 0xa8, 0xf4, 0x8d, + 0xad, 0x1b, 0xa3, 0x2a, 0xa2, 0x02, 0x5f, 0x8f, 0x13, 0x78, 0x4e, 0x99, 0x88, 0x88, 0xa6, 0xcb, + 0xd2, 0x4a, 0x28, 0x9b, 0x93, 0xf9, 0x6e, 0xa2, 0x62, 0x97, 0xfa, 0x48, 0x1b, 0xe8, 0x9f, 0x63, + 0xf5, 0x62, 0x60, 0x74, 0x9e, 0x56, 0x37, 0x17, 0xab, 0xd8, 0x3e, 0xa7, 0xde, 0x23, 0xc4, 0x44, + 0xc5, 0x2e, 0x83, 0x1d, 0x64, 0xe6, 0x17, 0xb8, 0xa7, 0x8f, 0xb8, 0xb0, 0xad, 0xc5, 0x33, 0xdf, + 0x23, 0xa4, 0xae, 0x24, 0xb8, 0xf0, 0x5a, 0xc4, 0x79, 0x74, 0xf1, 0x40, 0x9f, 0x60, 0x8b, 0xd8, + 0xf0, 0xb5, 0xc5, 0xce, 0xe3, 0x11, 0xa3, 0xa4, 0x56, 0x2c, 0x76, 0xdd, 0x06, 0x7a, 0x1f, 0x44, + 0xb2, 0x03, 0xec, 0x43, 0xae, 0x53, 0x11, 0x77, 0x63, 0xb7, 0x60, 0xd4, 0x75, 0x3f, 0x23, 0x8f, + 0xf9, 0x33, 0x99, 0x0b, 0x35, 0x97, 0x81, 0xe6, 0x60, 0xdb, 0xa9, 0xde, 0x58, 0x3c, 0x17, 0x62, + 0x26, 0x87, 0x94, 0x92, 0xcc, 0x65, 0xe0, 0xb5, 0xf6, 0x72, 0x90, 0x99, 0x68, 0x83, 0x31, 0x7e, + 0x9c, 0xce, 0x67, 0xa5, 0xdc, 0xe3, 0x74, 0x3e, 0x2f, 0x89, 0x8f, 0xd3, 0x79, 0x51, 0x02, 0xf9, + 0x55, 0x28, 0x04, 0xbc, 0x34, 0xaa, 0x42, 0x6e, 0x88, 0x6d, 0x5b, 0xeb, 0x61, 0xea, 0xd4, 0x45, + 0xc5, 0x6d, 0xca, 0x65, 0x28, 0x06, 0x3d, 0xb3, 0xfc, 0x85, 0xe0, 0x71, 0x12, 0xa7, 0x4b, 0x38, + 0x27, 0xd8, 0xa2, 0xba, 0xc1, 0x39, 0x79, 0x13, 0xbd, 0x0c, 0x25, 0xba, 0x02, 0xaa, 0xfb, 0x9e, + 0x78, 0xfe, 0xb4, 0x52, 0xa4, 0x9d, 0xe7, 0x9c, 0xe8, 0x0e, 0x14, 0xcc, 0x5d, 0xd3, 0x23, 0x49, + 0x51, 0x12, 0x30, 0x77, 0x4d, 0x97, 0xe0, 0x25, 0x28, 0x92, 0xb9, 0x7a, 0x14, 0x69, 0x3a, 0x48, + 0x81, 0xf4, 0x71, 0x12, 0xf9, 0x97, 0x49, 0x90, 0x66, 0xbd, 0x39, 0x7a, 0x08, 0x69, 0x72, 0xb0, + 0xf1, 0x33, 0xaa, 0xb6, 0xcd, 0x4e, 0xbd, 0x6d, 0xf7, 0xd4, 0xdb, 0x6e, 0xbb, 0xa7, 0xde, 0x5e, + 0xfe, 0x17, 0xbf, 0xbe, 0x93, 0xf8, 0xe2, 0xbf, 0xee, 0x08, 0x0a, 0xe5, 0x40, 0x37, 0x89, 0x0f, + 0xd7, 0xf4, 0x91, 0xaa, 0x77, 0xe9, 0x27, 0x8b, 0xc4, 0x41, 0x6b, 0xfa, 0xe8, 0xa0, 0x8b, 0x0e, + 0x41, 0xea, 0x18, 0x23, 0x1b, 0x8f, 0xec, 0xb1, 0xad, 0xb2, 0x33, 0x97, 0x9f, 0x4c, 0x21, 0x15, + 0x61, 0xc7, 0x6d, 0xc3, 0xa5, 0x3c, 0xa1, 0x84, 0x4a, 0xa5, 0x13, 0xee, 0x40, 0xfb, 0x00, 0x13, + 0x6d, 0xa0, 0x77, 0x35, 0xc7, 0xb0, 0xec, 0x6a, 0xfa, 0x6e, 0x2a, 0x52, 0x4f, 0xce, 0x5d, 0x92, + 0x33, 0xb3, 0xab, 0x39, 0x78, 0x2f, 0x4d, 0x3e, 0x57, 0x09, 0x70, 0xa2, 0x57, 0xa0, 0xa2, 0x99, + 0xa6, 0x6a, 0x3b, 0x9a, 0x83, 0xd5, 0x8b, 0x2b, 0x07, 0xdb, 0xf4, 0xd4, 0x2a, 0x2a, 0x25, 0xcd, + 0x34, 0x4f, 0x49, 0xef, 0x1e, 0xe9, 0x44, 0xf7, 0xa0, 0x4c, 0x0e, 0x38, 0x5d, 0x1b, 0xa8, 0x7d, + 0xac, 0xf7, 0xfa, 0x4e, 0x35, 0x7b, 0x57, 0xb8, 0x9f, 0x52, 0x4a, 0xbc, 0xb7, 0x45, 0x3b, 0xe5, + 0xae, 0xb7, 0xe3, 0xf4, 0x70, 0x43, 0x08, 0xd2, 0x5d, 0xcd, 0xd1, 0xe8, 0x4a, 0x16, 0x15, 0xfa, + 0x4c, 0xfa, 0x4c, 0xcd, 0xe9, 0xf3, 0xf5, 0xa1, 0xcf, 0xe8, 0x3a, 0x64, 0xb9, 0xd8, 0x14, 0x15, + 0xcb, 0x5b, 0x68, 0x0b, 0x32, 0xa6, 0x65, 0x4c, 0x30, 0xdd, 0xba, 0xbc, 0xc2, 0x1a, 0xb2, 0x02, + 0xe5, 0xf0, 0x41, 0x88, 0xca, 0x90, 0x74, 0xa6, 0x7c, 0x94, 0xa4, 0x33, 0x45, 0xdf, 0x85, 0x34, + 0x59, 0x48, 0x3a, 0x46, 0x39, 0xe2, 0xe8, 0xe7, 0x7c, 0xed, 0x2b, 0x13, 0x2b, 0x94, 0x52, 0xae, + 0x40, 0x29, 0x74, 0x40, 0xca, 0xd7, 0x61, 0x2b, 0xea, 0xbc, 0x93, 0xfb, 0x5e, 0x7f, 0xe8, 0xdc, + 0x42, 0x6f, 0x41, 0xde, 0x3b, 0xf0, 0x98, 0xe2, 0xdc, 0x9c, 0x1b, 0xd6, 0x25, 0x56, 0x3c, 0x52, + 0xa2, 0x31, 0x64, 0x03, 0xfa, 0x1a, 0x0f, 0x6f, 0x8a, 0x4a, 0x4e, 0x33, 0xcd, 0x96, 0x66, 0xf7, + 0xe5, 0x1f, 0x41, 0x35, 0xee, 0x30, 0x0b, 0x2c, 0x98, 0x40, 0xd5, 0xde, 0x5d, 0xb0, 0xeb, 0x90, + 0xbd, 0x34, 0xac, 0xa1, 0xe6, 0x50, 0x61, 0x25, 0x85, 0xb7, 0xc8, 0x42, 0xb2, 0x83, 0x2d, 0x45, + 0xbb, 0x59, 0x43, 0x56, 0xe1, 0x66, 0xec, 0x81, 0x46, 0x58, 0xf4, 0x51, 0x17, 0xb3, 0x65, 0x2d, + 0x29, 0xac, 0xe1, 0x0b, 0x62, 0x1f, 0xcb, 0x1a, 0x64, 0x58, 0x9b, 0xce, 0x95, 0xca, 0x17, 0x15, + 0xde, 0x92, 0xff, 0x2d, 0x0b, 0xd7, 0xa3, 0x8f, 0x35, 0x74, 0x17, 0x8a, 0x43, 0x6d, 0xaa, 0x3a, + 0x53, 0xae, 0x76, 0x02, 0xdd, 0x78, 0x18, 0x6a, 0xd3, 0xf6, 0x94, 0xe9, 0x9c, 0x04, 0x29, 0x67, + 0x6a, 0x57, 0x93, 0x77, 0x53, 0xf7, 0x8b, 0x0a, 0x79, 0x44, 0x67, 0xb0, 0x31, 0x30, 0x3a, 0xda, + 0x40, 0x1d, 0x68, 0xb6, 0xa3, 0xf2, 0x78, 0x87, 0x19, 0xd1, 0xcb, 0x73, 0x8b, 0xcd, 0x0e, 0x28, + 0xdc, 0x65, 0xfb, 0x49, 0x1c, 0x0e, 0xd7, 0xff, 0x0a, 0x95, 0x71, 0xa8, 0xb9, 0x5b, 0x8d, 0xce, + 0x60, 0xeb, 0xe2, 0xea, 0x73, 0x6d, 0xe4, 0xe8, 0x23, 0xac, 0xce, 0x99, 0xd5, 0xbc, 0xf6, 0x7c, + 0xa8, 0xdb, 0x17, 0xb8, 0xaf, 0x4d, 0x74, 0xc3, 0xe2, 0x22, 0x37, 0x3d, 0xfe, 0x73, 0xdf, 0xb6, + 0xfc, 0x3d, 0xca, 0x84, 0x94, 0xda, 0x75, 0x2f, 0xd9, 0xb5, 0xdd, 0xcb, 0x77, 0x61, 0x6b, 0x84, + 0xa7, 0x4e, 0xe0, 0x1b, 0x99, 0xe2, 0xe4, 0xe8, 0x5e, 0x20, 0xf2, 0xce, 0x1f, 0x9f, 0xe8, 0x10, + 0x7a, 0x8d, 0x46, 0x0a, 0xa6, 0x61, 0x63, 0x4b, 0xd5, 0xba, 0x5d, 0x0b, 0xdb, 0x76, 0x35, 0x4f, + 0xa9, 0x2b, 0x6e, 0x7f, 0x9d, 0x75, 0x87, 0x34, 0x51, 0x0c, 0x69, 0x22, 0x7a, 0x15, 0x2a, 0xb3, + 0x43, 0x02, 0xa5, 0x28, 0x4f, 0xc2, 0xc3, 0xdd, 0x83, 0xb2, 0xef, 0xe4, 0x28, 0x5d, 0x81, 0x79, + 0x13, 0xaf, 0x97, 0x92, 0xdd, 0x02, 0x91, 0xb8, 0x02, 0x46, 0x51, 0xa4, 0x14, 0x79, 0xd2, 0x41, + 0x5f, 0xbe, 0x0c, 0x25, 0x3c, 0xd1, 0xbb, 0x78, 0xd4, 0xc1, 0x8c, 0xa0, 0x44, 0x09, 0x8a, 0x6e, + 0x27, 0x25, 0x7a, 0x05, 0x2a, 0x54, 0x07, 0xd8, 0x29, 0x41, 0xc9, 0xca, 0x6c, 0x24, 0xd2, 0xcd, + 0x4e, 0x45, 0x42, 0xf7, 0x10, 0x6e, 0x06, 0xe8, 0x4c, 0xcd, 0x72, 0x54, 0x1b, 0x3b, 0xaa, 0x63, + 0x38, 0x3c, 0x10, 0x4b, 0x29, 0xd7, 0x3c, 0x8e, 0x13, 0xcd, 0x72, 0x4e, 0xb1, 0xd3, 0x26, 0x2f, + 0xd1, 0xdb, 0x50, 0x8d, 0xe2, 0xa4, 0x43, 0x49, 0x74, 0xa8, 0xad, 0x59, 0x46, 0x3a, 0xe2, 0x7d, + 0x90, 0x02, 0xda, 0xc9, 0xe8, 0x37, 0xd8, 0x62, 0x0d, 0x3c, 0x95, 0xa3, 0x94, 0x0f, 0x60, 0x83, + 0x52, 0x5a, 0xd8, 0x1e, 0x0f, 0x1c, 0xbe, 0x5e, 0x88, 0x6d, 0x0e, 0x79, 0xa1, 0xb0, 0x7e, 0xea, + 0x0b, 0xfe, 0x31, 0x68, 0x48, 0xe1, 0xb0, 0x8d, 0x9b, 0x89, 0xe0, 0x9b, 0xc9, 0x29, 0x6c, 0xf1, + 0xcd, 0xed, 0x86, 0x2c, 0x85, 0xa5, 0x4f, 0xb7, 0xe6, 0xbd, 0xe1, 0xac, 0x85, 0x20, 0x97, 0x7d, + 0x05, 0x23, 0x49, 0x3d, 0x9b, 0x91, 0x20, 0x48, 0xd3, 0x79, 0xa7, 0xd9, 0x09, 0x41, 0x9e, 0xbf, + 0xc9, 0x86, 0x03, 0x4b, 0x0d, 0xa7, 0xb0, 0xa2, 0xe1, 0x14, 0x97, 0x1a, 0x4e, 0x69, 0x99, 0xe1, + 0x94, 0x57, 0x33, 0x9c, 0xca, 0xda, 0x86, 0x23, 0x7d, 0x5d, 0xc3, 0xd9, 0x58, 0xd3, 0x70, 0xd0, + 0xea, 0x86, 0xb3, 0x19, 0x6d, 0x38, 0xef, 0xc3, 0xc6, 0x5c, 0xc2, 0xe2, 0x29, 0x9d, 0x10, 0xa9, + 0x74, 0xc9, 0xa0, 0xd2, 0xc9, 0x7f, 0x27, 0x40, 0x2d, 0x3e, 0x43, 0x89, 0x14, 0xf5, 0x3a, 0x6c, + 0x78, 0xdb, 0xeb, 0x29, 0x0f, 0x3b, 0x2f, 0x25, 0xef, 0x85, 0xab, 0x3d, 0x71, 0xa1, 0xcf, 0x3d, + 0x28, 0xcf, 0xe4, 0x4f, 0xcc, 0x44, 0x4a, 0x93, 0xe0, 0xf8, 0xf2, 0x3f, 0x64, 0xbd, 0x78, 0x24, + 0x94, 0xe4, 0x44, 0xb8, 0x85, 0x8f, 0x60, 0xb3, 0x8b, 0x3b, 0x7a, 0xf7, 0xeb, 0x7a, 0x85, 0x0d, + 0xce, 0xfd, 0xad, 0x53, 0xf8, 0xd6, 0x29, 0x7c, 0xb3, 0x9d, 0xc2, 0x5f, 0x25, 0x3d, 0xaf, 0xe0, + 0x27, 0xf3, 0x91, 0xa6, 0xfc, 0x36, 0xd1, 0x3a, 0x8d, 0x04, 0xb6, 0xcc, 0x4c, 0xaa, 0xf3, 0xb9, + 0x5a, 0x8b, 0xbe, 0xe7, 0xea, 0xcc, 0xa9, 0xd1, 0x71, 0xf8, 0xbb, 0x03, 0x38, 0xe4, 0x3c, 0xa8, + 0xe7, 0xdb, 0x53, 0xc0, 0xd8, 0x02, 0xd3, 0xa3, 0xb9, 0xb2, 0xb2, 0x30, 0x46, 0x9d, 0x4f, 0x35, + 0x9a, 0x7c, 0x7f, 0x17, 0x98, 0x99, 0xdc, 0xf4, 0x72, 0x5f, 0x0f, 0x8c, 0x98, 0xcb, 0xa4, 0x5e, + 0x82, 0xa2, 0xad, 0xf7, 0x54, 0x8a, 0xc2, 0xe8, 0x98, 0x65, 0xb5, 0x79, 0xa5, 0x60, 0xeb, 0xbd, + 0x73, 0xde, 0x25, 0xbf, 0x06, 0x95, 0x19, 0x40, 0x62, 0x26, 0x3d, 0xf1, 0x9d, 0xe9, 0xa6, 0xb7, + 0xee, 0x3e, 0xf0, 0x20, 0xff, 0xbc, 0x08, 0x79, 0x05, 0xdb, 0x26, 0x51, 0x6a, 0xb4, 0x07, 0x22, + 0x9e, 0x76, 0xb0, 0xe9, 0xb8, 0xa8, 0x40, 0x34, 0x78, 0xc1, 0xa8, 0x9b, 0x2e, 0x65, 0x2b, 0xa1, + 0xf8, 0x6c, 0xe8, 0x4d, 0x8e, 0x31, 0xc7, 0xc3, 0xc5, 0x9c, 0x3d, 0x08, 0x32, 0xbf, 0xed, 0x82, + 0xcc, 0xa9, 0x58, 0xfc, 0x94, 0x71, 0xcd, 0xa0, 0xcc, 0x6f, 0x72, 0x94, 0x39, 0xbd, 0x64, 0xb0, + 0x10, 0xcc, 0xdc, 0x08, 0xc1, 0xcc, 0x99, 0x25, 0xd3, 0x8c, 0xc1, 0x99, 0xdf, 0x76, 0x71, 0xe6, + 0xec, 0x92, 0x2f, 0x9e, 0x01, 0x9a, 0x7f, 0x10, 0x00, 0x9a, 0xf3, 0xb1, 0x08, 0x13, 0x63, 0x8d, + 0x40, 0x9a, 0xdf, 0xf1, 0x90, 0xe6, 0x42, 0x2c, 0x4a, 0xcd, 0x99, 0x67, 0xa1, 0xe6, 0xe3, 0x39, + 0xa8, 0x99, 0x41, 0xc3, 0xaf, 0xc4, 0x8a, 0x58, 0x82, 0x35, 0x1f, 0xcf, 0x61, 0xcd, 0xa5, 0x25, + 0x02, 0x97, 0x80, 0xcd, 0x7f, 0x16, 0x0d, 0x36, 0xc7, 0xc3, 0xc1, 0xfc, 0x33, 0x57, 0x43, 0x9b, + 0xd5, 0x18, 0xb4, 0xb9, 0x12, 0x8b, 0x8c, 0x32, 0xf1, 0x2b, 0xc3, 0xcd, 0x67, 0x11, 0x70, 0x33, + 0x03, 0x86, 0xef, 0xc7, 0x0a, 0x5f, 0x01, 0x6f, 0x3e, 0x8b, 0xc0, 0x9b, 0x37, 0x96, 0x8a, 0x5d, + 0x0a, 0x38, 0xef, 0x87, 0x01, 0x67, 0x14, 0x93, 0xc8, 0xfb, 0xd6, 0x1e, 0x83, 0x38, 0x5f, 0xc4, + 0x21, 0xce, 0x0c, 0x15, 0x7e, 0x23, 0x56, 0xe2, 0x1a, 0x90, 0xf3, 0xf1, 0x1c, 0xe4, 0xbc, 0xb5, + 0x44, 0xd3, 0x96, 0x60, 0xce, 0xfb, 0x61, 0xcc, 0xf9, 0xda, 0x92, 0xc9, 0xc7, 0x82, 0xce, 0x8d, + 0x10, 0xe8, 0x7c, 0x7d, 0x89, 0x2b, 0x89, 0x41, 0x9d, 0xff, 0x28, 0x88, 0x3a, 0xdf, 0x88, 0x05, + 0xae, 0xf9, 0x3e, 0x44, 0xc1, 0xce, 0xfb, 0x61, 0xd8, 0xb9, 0xba, 0x64, 0x3a, 0xab, 0xe0, 0xce, + 0x39, 0x29, 0xcf, 0x10, 0xe7, 0xc7, 0xe9, 0x3c, 0x48, 0x05, 0xf9, 0x35, 0x72, 0x78, 0xcc, 0x38, + 0x7e, 0xb4, 0x05, 0x19, 0x6c, 0x59, 0x86, 0xc5, 0x11, 0x64, 0xd6, 0x90, 0xef, 0x43, 0x31, 0xe8, + 0xe4, 0x17, 0x60, 0xd4, 0x14, 0xf7, 0x0b, 0x38, 0x76, 0xf9, 0x7f, 0x04, 0x9f, 0x97, 0x9e, 0xbc, + 0x41, 0x0c, 0x53, 0xe4, 0x18, 0x66, 0x00, 0xb9, 0x4e, 0x86, 0x91, 0xeb, 0x3b, 0x50, 0x20, 0x71, + 0xdf, 0x0c, 0x28, 0xad, 0x99, 0x1e, 0x28, 0xed, 0xc6, 0x29, 0x3c, 0xd6, 0x62, 0xa7, 0x64, 0x9a, + 0x9e, 0x92, 0x15, 0x3f, 0xda, 0x62, 0xb1, 0xed, 0x77, 0x60, 0x33, 0x40, 0xeb, 0xc5, 0x93, 0x0c, + 0xa1, 0x95, 0x3c, 0xea, 0x3a, 0x0f, 0x2c, 0xdf, 0x00, 0x34, 0xd4, 0x47, 0xfa, 0x70, 0x3c, 0x54, + 0x7b, 0x1a, 0x31, 0x55, 0xbd, 0x83, 0x6d, 0x7a, 0x3a, 0x88, 0x8a, 0xc4, 0xdf, 0x7c, 0xa0, 0xd9, + 0x27, 0xb4, 0x5f, 0xfe, 0x57, 0xc1, 0x5f, 0x4f, 0x1f, 0xfb, 0x8e, 0x82, 0xa9, 0x85, 0xe7, 0x04, + 0x53, 0x27, 0xbf, 0x36, 0x4c, 0x1d, 0x8c, 0xa6, 0x53, 0x61, 0x94, 0xf4, 0x7f, 0x05, 0x7f, 0x07, + 0x3d, 0xd0, 0xb9, 0x63, 0x74, 0x31, 0xc7, 0x2d, 0xe9, 0x33, 0xc9, 0x86, 0x06, 0x46, 0x8f, 0xa3, + 0x93, 0xe4, 0x91, 0x50, 0x79, 0xe7, 0xb6, 0xc8, 0x8f, 0x65, 0x0f, 0xf2, 0x64, 0x29, 0x06, 0x87, + 0x3c, 0x25, 0x48, 0x3d, 0xc5, 0xec, 0x94, 0x2d, 0x2a, 0xe4, 0x91, 0xd0, 0x51, 0x25, 0xe5, 0xa9, + 0x02, 0x6b, 0xa0, 0x87, 0x20, 0xd2, 0x3a, 0xbc, 0x6a, 0x98, 0x36, 0x3f, 0x58, 0x43, 0x59, 0x15, + 0x2b, 0xc6, 0x6f, 0x9f, 0x10, 0x9a, 0x63, 0xd3, 0x56, 0xf2, 0x26, 0x7f, 0x0a, 0x84, 0x4b, 0x62, + 0x28, 0xb7, 0x79, 0x01, 0x44, 0xf2, 0xf5, 0xb6, 0xa9, 0x75, 0x30, 0xcd, 0x22, 0x44, 0xc5, 0xef, + 0x90, 0x3f, 0x05, 0x34, 0xef, 0x1d, 0x50, 0x0b, 0xb2, 0x78, 0x82, 0x47, 0x0e, 0x4b, 0xfd, 0x0a, + 0xbb, 0xd7, 0x23, 0x42, 0x43, 0x3c, 0x72, 0xf6, 0xaa, 0x64, 0x91, 0x7f, 0xf3, 0xeb, 0x3b, 0x12, + 0xa3, 0x7e, 0xc3, 0x18, 0xea, 0x0e, 0x1e, 0x9a, 0xce, 0x95, 0xc2, 0xf9, 0xe5, 0x7f, 0x17, 0x48, + 0x60, 0x17, 0x8a, 0x03, 0x22, 0xd7, 0xd6, 0x35, 0x90, 0x64, 0x00, 0xe4, 0x9f, 0x5f, 0xef, 0x17, + 0x01, 0x88, 0x52, 0x7e, 0xa6, 0x8d, 0x1c, 0xdc, 0xe5, 0x0b, 0x2c, 0xf6, 0x34, 0xfb, 0x63, 0xda, + 0x11, 0x9e, 0x6a, 0x7e, 0x66, 0xaa, 0x01, 0x7c, 0x59, 0x0c, 0xe2, 0xcb, 0xa8, 0x06, 0x79, 0xd3, + 0xd2, 0x0d, 0x4b, 0x77, 0xae, 0xe8, 0xfa, 0xa4, 0x14, 0xaf, 0xfd, 0x38, 0x9d, 0x4f, 0x4b, 0x19, + 0xaf, 0x7c, 0xc5, 0x9c, 0x49, 0x41, 0x2a, 0xca, 0xff, 0x92, 0xf4, 0xf5, 0xde, 0x8f, 0x7b, 0xbf, + 0xfe, 0xc4, 0xa2, 0x14, 0xe9, 0x76, 0xc4, 0x64, 0x03, 0x3d, 0xe4, 0xbb, 0x49, 0x6b, 0x6c, 0xe3, + 0x2e, 0x2f, 0xa4, 0x78, 0xed, 0xc0, 0x06, 0xe6, 0x9e, 0x6d, 0x03, 0x97, 0xac, 0xe9, 0xf7, 0xa1, + 0x80, 0x27, 0x43, 0xd5, 0x99, 0xb2, 0xec, 0x44, 0xe4, 0xf9, 0xf3, 0xfc, 0x60, 0xc3, 0xf6, 0x94, + 0x38, 0x42, 0x45, 0xc4, 0xee, 0xa3, 0xfc, 0x37, 0xb4, 0x6c, 0x16, 0x3e, 0x0e, 0xd0, 0x69, 0x10, + 0xf6, 0x18, 0x53, 0x53, 0x76, 0x95, 0x70, 0x55, 0x9b, 0xf7, 0xe1, 0x11, 0xd6, 0x6d, 0xa3, 0x3f, + 0x86, 0x1b, 0x33, 0xfe, 0xc8, 0x13, 0x9d, 0x8c, 0x89, 0x5d, 0x67, 0xbd, 0xd2, 0xb5, 0xb0, 0x57, + 0x72, 0x25, 0xfb, 0xeb, 0x9c, 0x7a, 0x46, 0x43, 0x79, 0x0b, 0xca, 0xe1, 0x90, 0x97, 0x64, 0xdd, + 0x16, 0x76, 0x34, 0x7d, 0xa4, 0x86, 0xb0, 0x9d, 0x22, 0xeb, 0xe4, 0xc5, 0xb2, 0x13, 0xb8, 0x16, + 0x19, 0xe6, 0xa2, 0xef, 0x81, 0xe8, 0x47, 0xc8, 0x42, 0x4c, 0x82, 0xe7, 0xd5, 0x92, 0x7c, 0x5a, + 0xf9, 0x9f, 0x05, 0x5f, 0x64, 0xb8, 0x3a, 0xd5, 0x84, 0x2c, 0x4b, 0x8c, 0xa9, 0x82, 0x97, 0x77, + 0xbf, 0xb3, 0x5a, 0x80, 0xbc, 0xcd, 0xb2, 0x66, 0x85, 0x33, 0xcb, 0x9f, 0x42, 0x96, 0xf5, 0xa0, + 0x02, 0xe4, 0xce, 0x8e, 0x9e, 0x1c, 0x1d, 0x7f, 0x7c, 0x24, 0x25, 0x10, 0x40, 0xb6, 0xde, 0x68, + 0x34, 0x4f, 0xda, 0x92, 0x80, 0x44, 0xc8, 0xd4, 0xf7, 0x8e, 0x95, 0xb6, 0x94, 0x24, 0xdd, 0x4a, + 0xf3, 0x71, 0xb3, 0xd1, 0x96, 0x52, 0x68, 0x03, 0x4a, 0xec, 0x59, 0xdd, 0x3f, 0x56, 0x3e, 0xac, + 0xb7, 0xa5, 0x74, 0xa0, 0xeb, 0xb4, 0x79, 0xf4, 0xa8, 0xa9, 0x48, 0x19, 0xf9, 0xf7, 0xe1, 0x66, + 0x6c, 0x48, 0xed, 0x97, 0x9e, 0x84, 0x40, 0xe9, 0x49, 0xfe, 0xdb, 0x24, 0xd4, 0xe2, 0xe3, 0x64, + 0xf4, 0x78, 0x66, 0xe2, 0xbb, 0x6b, 0x04, 0xd9, 0x33, 0xb3, 0x47, 0xf7, 0xa0, 0x6c, 0xe1, 0x4b, + 0xec, 0x74, 0xfa, 0x2c, 0x6e, 0x67, 0x27, 0x5a, 0x49, 0x29, 0xf1, 0x5e, 0xca, 0x64, 0x33, 0xb2, + 0x1f, 0xe3, 0x8e, 0xa3, 0x32, 0x2f, 0xc5, 0x14, 0x4c, 0x24, 0x64, 0xa4, 0xf7, 0x94, 0x75, 0xca, + 0x3f, 0x5a, 0x6b, 0x2d, 0x45, 0xc8, 0x28, 0xcd, 0xb6, 0xf2, 0x89, 0x94, 0x42, 0x08, 0xca, 0xf4, + 0x51, 0x3d, 0x3d, 0xaa, 0x9f, 0x9c, 0xb6, 0x8e, 0xc9, 0x5a, 0x6e, 0x42, 0xc5, 0x5d, 0x4b, 0xb7, + 0x33, 0x23, 0xff, 0x47, 0x12, 0x6e, 0xc4, 0x44, 0xf9, 0xe8, 0x21, 0x80, 0x33, 0x55, 0x2d, 0xdc, + 0x31, 0xac, 0x6e, 0xbc, 0x92, 0xb5, 0xa7, 0x0a, 0xa5, 0x50, 0x44, 0x87, 0x3f, 0xd9, 0x0b, 0x2a, + 0x96, 0xe8, 0x3d, 0x2e, 0x94, 0x22, 0x2d, 0xdc, 0xac, 0x5e, 0x8c, 0x28, 0xcc, 0xe1, 0x0e, 0x11, + 0x4c, 0xd7, 0x96, 0x0a, 0xa6, 0xf4, 0xe8, 0xc3, 0x28, 0xff, 0xb1, 0x62, 0x69, 0x3b, 0xc2, 0x73, + 0x7c, 0x12, 0xef, 0x39, 0x32, 0xab, 0x06, 0x34, 0xd1, 0xae, 0x43, 0xfe, 0xfb, 0x54, 0x70, 0x61, + 0xc3, 0x49, 0xcd, 0x31, 0x64, 0x6d, 0x47, 0x73, 0xc6, 0x36, 0x57, 0xb8, 0xef, 0xad, 0x9a, 0x21, + 0x6d, 0xbb, 0x0f, 0xa7, 0x94, 0x5d, 0xe1, 0x62, 0xbe, 0x5d, 0x6f, 0xea, 0x60, 0xc3, 0x8b, 0x13, + 0x6f, 0x32, 0xbe, 0xcf, 0x49, 0xca, 0xef, 0xfa, 0x01, 0x52, 0x00, 0xfc, 0x9f, 0x07, 0xd6, 0x85, + 0x28, 0x60, 0xfd, 0xe7, 0x02, 0xdc, 0x5a, 0x90, 0x27, 0xa2, 0x8f, 0x66, 0xf6, 0xf9, 0x9d, 0x75, + 0xb2, 0xcc, 0x6d, 0xd6, 0x17, 0xde, 0x69, 0xf9, 0x4d, 0x28, 0x06, 0xfb, 0x57, 0x9b, 0xe4, 0x6f, + 0x92, 0xbe, 0xcf, 0x0f, 0x57, 0x00, 0x9e, 0x5b, 0x24, 0x38, 0xa3, 0x67, 0xc9, 0x35, 0xf5, 0x2c, + 0x32, 0x2e, 0x48, 0x3d, 0x63, 0x5c, 0xb0, 0x40, 0xdb, 0xd2, 0xcf, 0xa6, 0x6d, 0x21, 0x83, 0xcb, + 0x84, 0x93, 0x8d, 0x2d, 0x5f, 0xa3, 0x02, 0x00, 0xe6, 0x27, 0x00, 0x01, 0xa4, 0x76, 0x0b, 0x32, + 0x96, 0x31, 0x1e, 0x75, 0xa9, 0x5e, 0x64, 0x14, 0xd6, 0x40, 0x6f, 0x41, 0x86, 0xe8, 0x97, 0xbb, + 0x7a, 0xf3, 0xae, 0x96, 0xe8, 0x47, 0x00, 0xff, 0x65, 0xd4, 0xf2, 0x0f, 0xa1, 0x1c, 0x86, 0x87, + 0x9f, 0xaf, 0x78, 0x1d, 0xd0, 0xfc, 0x35, 0x89, 0x98, 0x21, 0x7e, 0x10, 0x1e, 0xe2, 0xa5, 0xd8, + 0x0b, 0x17, 0xd1, 0x43, 0x7d, 0x0e, 0x19, 0xaa, 0x6e, 0x24, 0x5e, 0xa6, 0x77, 0x73, 0x78, 0x3e, + 0x4d, 0x9e, 0xd1, 0x0f, 0x01, 0x34, 0xc7, 0xb1, 0xf4, 0x8b, 0xb1, 0x3f, 0xc0, 0x9d, 0x68, 0x75, + 0xad, 0xbb, 0x74, 0x7b, 0x2f, 0x70, 0xbd, 0xdd, 0xf2, 0x59, 0x03, 0xba, 0x1b, 0x10, 0x28, 0x1f, + 0x41, 0x39, 0xcc, 0xeb, 0xe6, 0x74, 0x42, 0x44, 0x4e, 0x97, 0x0c, 0xe6, 0x74, 0x5e, 0x46, 0x98, + 0x62, 0x17, 0x90, 0x68, 0x43, 0xfe, 0xa7, 0x24, 0x14, 0x83, 0xda, 0xfe, 0x9c, 0xb3, 0x87, 0x25, + 0xa9, 0xd2, 0xcd, 0xb9, 0xe4, 0x21, 0xd7, 0xd3, 0xec, 0xb3, 0x6f, 0x4a, 0xee, 0xf0, 0xd7, 0x02, + 0xe4, 0xbd, 0x85, 0x8b, 0x29, 0x14, 0xf8, 0xeb, 0x9e, 0x0c, 0x5e, 0x3e, 0x62, 0xc5, 0x89, 0x94, + 0x57, 0x9c, 0x78, 0xd7, 0x0b, 0xee, 0xe2, 0xd0, 0xf7, 0xe0, 0x2e, 0xb9, 0x25, 0x1a, 0x1e, 0xcb, + 0x5a, 0xec, 0x33, 0x48, 0x50, 0x83, 0xbe, 0x0f, 0x59, 0xad, 0xe3, 0x95, 0x1c, 0xca, 0x11, 0x00, + 0x9a, 0x4b, 0xba, 0xdd, 0x9e, 0xd6, 0x29, 0xa5, 0xc2, 0x39, 0xf8, 0x47, 0x25, 0xdd, 0x8f, 0x92, + 0x6b, 0x44, 0x6e, 0xdd, 0x7d, 0x07, 0x67, 0x47, 0x1f, 0x1e, 0x3f, 0x3a, 0xd8, 0x3f, 0x68, 0x3e, + 0x92, 0x12, 0x72, 0x03, 0x0a, 0x6e, 0x89, 0x4b, 0x1b, 0xda, 0xe8, 0x16, 0x88, 0x43, 0x2d, 0x7c, + 0x01, 0x2a, 0x3f, 0xd4, 0xf8, 0xf5, 0xa7, 0x1b, 0x90, 0x23, 0x2f, 0x7b, 0x9a, 0xed, 0x56, 0xa4, + 0x87, 0xda, 0xf4, 0x03, 0xcd, 0x96, 0x7f, 0x2b, 0x40, 0x65, 0xc6, 0x95, 0xa1, 0x5d, 0xc8, 0x30, + 0xf0, 0x2e, 0xee, 0x5e, 0x7d, 0x60, 0x58, 0x85, 0x91, 0xa2, 0xf7, 0x20, 0xef, 0x56, 0x01, 0xa3, + 0x72, 0x29, 0xe6, 0x33, 0xdd, 0x3a, 0x12, 0x67, 0xf5, 0x38, 0xd0, 0xfb, 0x20, 0x7a, 0x4e, 0x39, + 0xfe, 0x22, 0xa3, 0xe7, 0xce, 0x39, 0xbf, 0xcf, 0x83, 0xde, 0xf1, 0x31, 0xb4, 0xf4, 0x7c, 0x21, + 0x81, 0xb3, 0x33, 0x02, 0xce, 0xec, 0xd2, 0xcb, 0xef, 0x82, 0xe8, 0x09, 0x46, 0x55, 0xc8, 0xb9, + 0xb5, 0x58, 0x81, 0x7b, 0x6b, 0x5e, 0x83, 0xdd, 0x82, 0x8c, 0x69, 0x7c, 0xc6, 0x2f, 0xa5, 0xa5, + 0x14, 0xd6, 0x90, 0xbb, 0x50, 0x99, 0x39, 0x64, 0xd0, 0xbb, 0x90, 0x33, 0xc7, 0x17, 0xaa, 0xeb, + 0x11, 0x66, 0xd6, 0xcf, 0xc5, 0x6d, 0xc6, 0x17, 0x03, 0xbd, 0xf3, 0x04, 0x5f, 0xb9, 0x7a, 0x64, + 0x8e, 0x2f, 0x9e, 0x30, 0xc7, 0xc1, 0x46, 0x49, 0x06, 0x47, 0x99, 0x40, 0xde, 0xf5, 0x83, 0xe8, + 0x0f, 0x83, 0x4b, 0x25, 0xc4, 0xd8, 0x8a, 0xf7, 0x4d, 0x5c, 0x7c, 0x60, 0xa5, 0x1e, 0xc0, 0x86, + 0xad, 0xf7, 0x46, 0x6e, 0xdd, 0x9e, 0x6d, 0x34, 0x2b, 0xc4, 0x55, 0xd8, 0x8b, 0x43, 0x17, 0x0a, + 0x24, 0x61, 0x8b, 0x34, 0xeb, 0x88, 0xff, 0x3f, 0x3f, 0x20, 0x22, 0xbc, 0x4a, 0x45, 0x85, 0x57, + 0x7f, 0x99, 0x84, 0x42, 0xe0, 0x36, 0x00, 0xfa, 0x83, 0xc0, 0xa9, 0x50, 0x8e, 0x88, 0x0b, 0x02, + 0xb4, 0xfe, 0xad, 0xcd, 0xf0, 0xc4, 0x92, 0xeb, 0x4f, 0x2c, 0xee, 0xf2, 0x85, 0x7b, 0xa9, 0x20, + 0xbd, 0xf6, 0xa5, 0x82, 0x37, 0x00, 0xd1, 0x72, 0xb8, 0x3a, 0x31, 0x1c, 0x7d, 0xd4, 0x53, 0x99, + 0x6a, 0x30, 0x1f, 0x2e, 0xd1, 0x37, 0xe7, 0xf4, 0xc5, 0x09, 0xd5, 0x92, 0x3f, 0x4f, 0x42, 0xde, + 0xb5, 0xb0, 0xdf, 0xd1, 0x25, 0xf8, 0x00, 0x44, 0xef, 0x98, 0x40, 0xbf, 0x07, 0x25, 0x96, 0x33, + 0xd7, 0x03, 0x16, 0x2d, 0x2a, 0xe1, 0x4e, 0x62, 0x71, 0x23, 0xc3, 0xf5, 0x5a, 0x69, 0x85, 0x35, + 0xe4, 0xbf, 0x10, 0x20, 0xef, 0xe1, 0x1d, 0xeb, 0xde, 0x8f, 0xbd, 0x0e, 0x59, 0x9e, 0xd2, 0xb3, + 0x0b, 0xb2, 0xbc, 0x15, 0x79, 0x13, 0xa5, 0x06, 0xf9, 0x21, 0x76, 0x34, 0x7a, 0xb8, 0xb3, 0xf8, + 0xd0, 0x6b, 0x3f, 0x78, 0x07, 0x0a, 0x81, 0xbb, 0xc5, 0xe4, 0xbc, 0x3f, 0x6a, 0x7e, 0x2c, 0x25, + 0x6a, 0xb9, 0x9f, 0xfe, 0xec, 0x6e, 0xea, 0x08, 0x7f, 0x46, 0xbc, 0x95, 0xd2, 0x6c, 0xb4, 0x9a, + 0x8d, 0x27, 0x92, 0x50, 0x2b, 0xfc, 0xf4, 0x67, 0x77, 0x73, 0x0a, 0xa6, 0x95, 0xd5, 0x07, 0x4f, + 0xa0, 0x32, 0xb3, 0xc3, 0xe1, 0x04, 0x00, 0x41, 0xf9, 0xd1, 0xd9, 0xc9, 0xe1, 0x41, 0xa3, 0xde, + 0x6e, 0xaa, 0xe7, 0xc7, 0xed, 0xa6, 0x24, 0xa0, 0x1b, 0xb0, 0x79, 0x78, 0xf0, 0x41, 0xab, 0xad, + 0x36, 0x0e, 0x0f, 0x9a, 0x47, 0x6d, 0xb5, 0xde, 0x6e, 0xd7, 0x1b, 0x4f, 0xa4, 0xe4, 0xee, 0x6f, + 0x0b, 0x50, 0xa9, 0xef, 0x35, 0x0e, 0xea, 0xa6, 0x39, 0xd0, 0x3b, 0x1a, 0x3d, 0x8c, 0x1a, 0x90, + 0xa6, 0xc5, 0x90, 0x85, 0x3f, 0x5d, 0xd5, 0x16, 0x97, 0xcb, 0xd1, 0x3e, 0x64, 0x68, 0x9d, 0x04, + 0x2d, 0xfe, 0x0b, 0xab, 0xb6, 0xa4, 0x7e, 0x4e, 0x3e, 0x86, 0xee, 0xf8, 0xc2, 0xdf, 0xb2, 0x6a, + 0x8b, 0xcb, 0xe9, 0xe8, 0x10, 0x72, 0x2e, 0x30, 0xbd, 0xec, 0x5f, 0xa9, 0xda, 0xd2, 0x1a, 0x37, + 0x99, 0x1a, 0x2b, 0x20, 0x2c, 0xfe, 0x63, 0xab, 0xb6, 0xa4, 0xd0, 0x8e, 0x0e, 0x20, 0xcb, 0x61, + 0xc0, 0x25, 0x3f, 0x61, 0xd5, 0x96, 0x95, 0xce, 0x91, 0x02, 0xa2, 0x5f, 0x9a, 0x59, 0xfe, 0x1f, + 0x5a, 0x6d, 0x85, 0x3b, 0x04, 0xe8, 0x53, 0x28, 0x85, 0xe1, 0xc6, 0xd5, 0x7e, 0xf4, 0xaa, 0xad, + 0x58, 0xa4, 0x27, 0xf2, 0xc3, 0xd8, 0xe3, 0x6a, 0x3f, 0x7e, 0xd5, 0x56, 0xac, 0xd9, 0xa3, 0x1f, + 0xc3, 0xc6, 0x3c, 0x36, 0xb8, 0xfa, 0x7f, 0x60, 0xb5, 0x35, 0xaa, 0xf8, 0x68, 0x08, 0x28, 0x02, + 0x53, 0x5c, 0xe3, 0xb7, 0xb0, 0xda, 0x3a, 0x45, 0x7d, 0xd4, 0x85, 0xca, 0x2c, 0x4e, 0xb7, 0xea, + 0x6f, 0x62, 0xb5, 0x95, 0x0b, 0xfc, 0x6c, 0x94, 0x30, 0x68, 0xb5, 0xea, 0x6f, 0x63, 0xb5, 0x95, + 0xeb, 0xfd, 0xe8, 0x0c, 0x20, 0x00, 0xba, 0xac, 0xf0, 0x1b, 0x59, 0x6d, 0x95, 0xca, 0x3f, 0x32, + 0x61, 0x33, 0x0a, 0x8d, 0x59, 0xe7, 0xaf, 0xb2, 0xda, 0x5a, 0x17, 0x02, 0x88, 0x3e, 0x87, 0x71, + 0x95, 0xd5, 0xfe, 0x32, 0xab, 0xad, 0x78, 0x33, 0x80, 0x2c, 0x94, 0x8f, 0x25, 0xa0, 0x15, 0xfe, + 0xd4, 0xaa, 0xad, 0x52, 0x56, 0xdf, 0x6b, 0xfe, 0xe2, 0xcb, 0xdb, 0xc2, 0xaf, 0xbe, 0xbc, 0x2d, + 0xfc, 0xf7, 0x97, 0xb7, 0x85, 0x2f, 0xbe, 0xba, 0x9d, 0xf8, 0xd5, 0x57, 0xb7, 0x13, 0xff, 0xf9, + 0xd5, 0xed, 0xc4, 0x9f, 0xbc, 0xde, 0xd3, 0x9d, 0xfe, 0xf8, 0x62, 0xbb, 0x63, 0x0c, 0x77, 0x82, + 0xbf, 0xfb, 0x46, 0xfd, 0x83, 0x7c, 0x91, 0xa5, 0x07, 0xfe, 0x9b, 0xff, 0x17, 0x00, 0x00, 0xff, + 0xff, 0xbe, 0x7e, 0xe8, 0x5b, 0xa3, 0x3c, 0x00, 0x00, } // Reference imports to suppress errors if they are not otherwise used. @@ -8221,6 +8294,18 @@ func (m *ResponseDeliverTx) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = i var l int _ = l + if m.EvmTxInfo != nil { + { + size, err := m.EvmTxInfo.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintTypes(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x4a + } if len(m.Codespace) > 0 { i -= len(m.Codespace) copy(dAtA[i:], m.Codespace) @@ -8497,20 +8582,20 @@ func (m *ResponseApplySnapshotChunk) MarshalToSizedBuffer(dAtA []byte) (int, err } } if len(m.RefetchChunks) > 0 { - dAtA57 := make([]byte, len(m.RefetchChunks)*10) - var j56 int + dAtA58 := make([]byte, len(m.RefetchChunks)*10) + var j57 int for _, num := range m.RefetchChunks { for num >= 1<<7 { - dAtA57[j56] = uint8(uint64(num)&0x7f | 0x80) + dAtA58[j57] = uint8(uint64(num)&0x7f | 0x80) num >>= 7 - j56++ + j57++ } - dAtA57[j56] = uint8(num) - j56++ + dAtA58[j57] = uint8(num) + j57++ } - i -= j56 - copy(dAtA[i:], dAtA57[:j56]) - i = encodeVarintTypes(dAtA, i, uint64(j56)) + i -= j57 + copy(dAtA[i:], dAtA58[:j57]) + i = encodeVarintTypes(dAtA, i, uint64(j57)) i-- dAtA[i] = 0x12 } @@ -9083,6 +9168,18 @@ func (m *ExecTxResult) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = i var l int _ = l + if m.EvmTxInfo != nil { + { + size, err := m.EvmTxInfo.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintTypes(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x4a + } if len(m.Codespace) > 0 { i -= len(m.Codespace) copy(dAtA[i:], m.Codespace) @@ -9523,12 +9620,12 @@ func (m *Misbehavior) MarshalToSizedBuffer(dAtA []byte) (int, error) { i-- dAtA[i] = 0x28 } - n69, err69 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.Time, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.Time):]) - if err69 != nil { - return 0, err69 + n71, err71 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.Time, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.Time):]) + if err71 != nil { + return 0, err71 } - i -= n69 - i = encodeVarintTypes(dAtA, i, uint64(n69)) + i -= n71 + i = encodeVarintTypes(dAtA, i, uint64(n71)) i-- dAtA[i] = 0x22 if m.Height != 0 { @@ -9579,12 +9676,12 @@ func (m *Evidence) MarshalToSizedBuffer(dAtA []byte) (int, error) { i-- dAtA[i] = 0x28 } - n71, err71 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.Time, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.Time):]) - if err71 != nil { - return 0, err71 + n73, err73 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.Time, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.Time):]) + if err73 != nil { + return 0, err73 } - i -= n71 - i = encodeVarintTypes(dAtA, i, uint64(n71)) + i -= n73 + i = encodeVarintTypes(dAtA, i, uint64(n73)) i-- dAtA[i] = 0x22 if m.Height != 0 { @@ -9610,6 +9707,41 @@ func (m *Evidence) MarshalToSizedBuffer(dAtA []byte) (int, error) { return len(dAtA) - i, nil } +func (m *EvmTxInfo) 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 *EvmTxInfo) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *EvmTxInfo) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if m.Nonce != 0 { + i = encodeVarintTypes(dAtA, i, uint64(m.Nonce)) + i-- + dAtA[i] = 0x10 + } + if len(m.SenderAddress) > 0 { + i -= len(m.SenderAddress) + copy(dAtA[i:], m.SenderAddress) + i = encodeVarintTypes(dAtA, i, uint64(len(m.SenderAddress))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + func (m *Snapshot) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) @@ -10931,6 +11063,10 @@ func (m *ResponseDeliverTx) Size() (n int) { if l > 0 { n += 1 + l + sovTypes(uint64(l)) } + if m.EvmTxInfo != nil { + l = m.EvmTxInfo.Size() + n += 1 + l + sovTypes(uint64(l)) + } return n } @@ -11302,6 +11438,10 @@ func (m *ExecTxResult) Size() (n int) { if l > 0 { n += 1 + l + sovTypes(uint64(l)) } + if m.EvmTxInfo != nil { + l = m.EvmTxInfo.Size() + n += 1 + l + sovTypes(uint64(l)) + } return n } @@ -11488,6 +11628,22 @@ func (m *Evidence) Size() (n int) { return n } +func (m *EvmTxInfo) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.SenderAddress) + if l > 0 { + n += 1 + l + sovTypes(uint64(l)) + } + if m.Nonce != 0 { + n += 1 + sovTypes(uint64(m.Nonce)) + } + return n +} + func (m *Snapshot) Size() (n int) { if m == nil { return 0 @@ -18311,6 +18467,42 @@ func (m *ResponseDeliverTx) Unmarshal(dAtA []byte) error { } m.Codespace = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex + case 9: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field EvmTxInfo", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthTypes + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthTypes + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.EvmTxInfo == nil { + m.EvmTxInfo = &EvmTxInfo{} + } + if err := m.EvmTxInfo.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipTypes(dAtA[iNdEx:]) @@ -20636,6 +20828,42 @@ func (m *ExecTxResult) Unmarshal(dAtA []byte) error { } m.Codespace = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex + case 9: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field EvmTxInfo", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthTypes + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthTypes + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.EvmTxInfo == nil { + m.EvmTxInfo = &EvmTxInfo{} + } + if err := m.EvmTxInfo.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipTypes(dAtA[iNdEx:]) @@ -21988,6 +22216,107 @@ func (m *Evidence) Unmarshal(dAtA []byte) error { } return nil } +func (m *EvmTxInfo) 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 ErrIntOverflowTypes + } + 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: EvmTxInfo: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: EvmTxInfo: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field SenderAddress", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthTypes + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthTypes + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.SenderAddress = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Nonce", wireType) + } + m.Nonce = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Nonce |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + default: + iNdEx = preIndex + skippy, err := skipTypes(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthTypes + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} func (m *Snapshot) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 diff --git a/internal/mempool/mempool.go b/internal/mempool/mempool.go index 96b214a33..80201ef3f 100644 --- a/internal/mempool/mempool.go +++ b/internal/mempool/mempool.go @@ -364,7 +364,7 @@ func (txmp *TxMempool) RemoveTxByKey(txKey types.TxKey) error { // remove the committed transaction from the transaction store and indexes if wtx := txmp.txStore.GetTxByHash(txKey); wtx != nil { - txmp.removeTx(wtx, false, true) + txmp.removeTx(wtx, false, true, true) return nil } @@ -403,7 +403,7 @@ func (txmp *TxMempool) Flush() { txmp.timestampIndex.Reset() for _, wtx := range txmp.txStore.GetAllTxs() { - txmp.removeTx(wtx, false, false) + txmp.removeTx(wtx, false, false, true) } atomic.SwapInt64(&txmp.sizeBytes, 0) @@ -515,7 +515,17 @@ func (txmp *TxMempool) Update( // remove the committed transaction from the transaction store and indexes if wtx := txmp.txStore.GetTxByHash(tx.Key()); wtx != nil { - txmp.removeTx(wtx, false, false) + txmp.removeTx(wtx, false, false, true) + } + if execTxResult[i].EvmTxInfo != nil { + // remove any tx that has the same nonce (because the committed tx + // may be from block proposal and is never in the local mempool) + if wtx, _ := txmp.priorityIndex.GetTxWithSameNonce(&WrappedTx{ + evmAddress: execTxResult[i].EvmTxInfo.SenderAddress, + evmNonce: execTxResult[i].EvmTxInfo.Nonce, + }); wtx != nil { + txmp.removeTx(wtx, false, false, true) + } } } @@ -636,7 +646,7 @@ func (txmp *TxMempool) addNewTransaction(wtx *WrappedTx, res *abci.ResponseCheck // - The transaction, toEvict, can be removed while a concurrent // reCheckTx callback is being executed for the same transaction. for _, toEvict := range evictTxs { - txmp.removeTx(toEvict, true, true) + txmp.removeTx(toEvict, true, true, true) txmp.logger.Debug( "evicted existing good transaction; mempool full", "old_tx", fmt.Sprintf("%X", toEvict.tx.Hash()), @@ -655,11 +665,19 @@ func (txmp *TxMempool) addNewTransaction(wtx *WrappedTx, res *abci.ResponseCheck txInfo.SenderID: {}, } + replaced, shouldDrop := txmp.priorityIndex.TryReplacement(wtx) + if shouldDrop { + return nil + } + txmp.metrics.TxSizeBytes.Observe(float64(wtx.Size())) txmp.metrics.Size.Set(float64(txmp.SizeWithoutPending())) txmp.metrics.PendingSize.Set(float64(txmp.PendingSize())) - if txmp.insertTx(wtx) { + if replaced != nil { + txmp.removeTx(replaced, true, false, false) + } + if txmp.insertTx(wtx, replaced == nil) { txmp.logger.Debug( "inserted good transaction", "priority", wtx.priority, @@ -747,7 +765,7 @@ func (txmp *TxMempool) handleRecheckResult(tx types.Tx, res *abci.ResponseCheckT panic("corrupted reCheckTx cursor") } - txmp.removeTx(wtx, !txmp.config.KeepInvalidTxsInCache, true) + txmp.removeTx(wtx, !txmp.config.KeepInvalidTxsInCache, true, true) } } @@ -853,13 +871,15 @@ func (txmp *TxMempool) canAddPendingTx(wtx *WrappedTx) error { return nil } -func (txmp *TxMempool) insertTx(wtx *WrappedTx) bool { +func (txmp *TxMempool) insertTx(wtx *WrappedTx, updatePriorityIndex bool) bool { if txmp.isInMempool(wtx.tx) { return false } txmp.txStore.SetTx(wtx) - txmp.priorityIndex.PushTx(wtx) + if updatePriorityIndex { + txmp.priorityIndex.PushTx(wtx) + } txmp.heightIndex.Insert(wtx) txmp.timestampIndex.Insert(wtx) @@ -873,13 +893,16 @@ func (txmp *TxMempool) insertTx(wtx *WrappedTx) bool { return true } -func (txmp *TxMempool) removeTx(wtx *WrappedTx, removeFromCache bool, shouldReenqueue bool) { +func (txmp *TxMempool) removeTx(wtx *WrappedTx, removeFromCache bool, shouldReenqueue bool, updatePriorityIndex bool) { if txmp.txStore.IsTxRemoved(wtx.hash) { return } txmp.txStore.RemoveTx(wtx) - toBeReenqueued := txmp.priorityIndex.RemoveTx(wtx, shouldReenqueue) + toBeReenqueued := []*WrappedTx{} + if updatePriorityIndex { + toBeReenqueued = txmp.priorityIndex.RemoveTx(wtx, shouldReenqueue) + } txmp.heightIndex.Remove(wtx) txmp.timestampIndex.Remove(wtx) @@ -894,7 +917,7 @@ func (txmp *TxMempool) removeTx(wtx *WrappedTx, removeFromCache bool, shouldReen if shouldReenqueue { for _, reenqueue := range toBeReenqueued { - txmp.removeTx(reenqueue, removeFromCache, false) + txmp.removeTx(reenqueue, removeFromCache, false, true) } for _, reenqueue := range toBeReenqueued { rtx := reenqueue.tx diff --git a/internal/mempool/mempool_test.go b/internal/mempool/mempool_test.go index 9b7d56888..4c590bf57 100644 --- a/internal/mempool/mempool_test.go +++ b/internal/mempool/mempool_test.go @@ -612,7 +612,7 @@ func TestTxMempool_EVMEviction(t *testing.T) { require.NoError(t, txmp.CheckTx(ctx, []byte(fmt.Sprintf("evm-sender=%s=%d=%d", address2, 5, 1)), nil, TxInfo{SenderID: peerID})) require.Equal(t, 2, txmp.priorityIndex.NumTxs()) - txmp.removeTx(tx, true, false) + txmp.removeTx(tx, true, false, true) // should not reenqueue require.Equal(t, 1, txmp.priorityIndex.NumTxs()) time.Sleep(1 * time.Second) // pendingTxs should still be one even after sleeping for a sec diff --git a/internal/mempool/priority_queue.go b/internal/mempool/priority_queue.go index e9a357d63..0dcc6edd0 100644 --- a/internal/mempool/priority_queue.go +++ b/internal/mempool/priority_queue.go @@ -12,8 +12,11 @@ var _ heap.Interface = (*TxPriorityQueue)(nil) // TxPriorityQueue defines a thread-safe priority queue for valid transactions. type TxPriorityQueue struct { - mtx sync.RWMutex - txs []*WrappedTx // priority heap + mtx sync.RWMutex + txs []*WrappedTx // priority heap + // invariant 1: no duplicate nonce in the same queue + // invariant 2: no nonce gap in the same queue + // invariant 3: head of the queue must be in heap evmQueue map[string][]*WrappedTx // sorted by nonce } @@ -50,6 +53,51 @@ func NewTxPriorityQueue() *TxPriorityQueue { return pq } +func (pq *TxPriorityQueue) GetTxWithSameNonce(tx *WrappedTx) (*WrappedTx, int) { + pq.mtx.RLock() + defer pq.mtx.RUnlock() + return pq.getTxWithSameNonceUnsafe(tx) +} + +func (pq *TxPriorityQueue) getTxWithSameNonceUnsafe(tx *WrappedTx) (*WrappedTx, int) { + queue, ok := pq.evmQueue[tx.evmAddress] + if !ok { + return nil, -1 + } + idx := binarySearch(queue, tx) + if idx < len(queue) && queue[idx].evmNonce == tx.evmNonce { + return queue[idx], idx + } + return nil, -1 +} + +func (pq *TxPriorityQueue) TryReplacement(tx *WrappedTx) (replaced *WrappedTx, shouldDrop bool) { + if !tx.isEVM { + return nil, false + } + pq.mtx.Lock() + defer pq.mtx.Unlock() + queue, ok := pq.evmQueue[tx.evmAddress] + if ok && len(queue) > 0 { + existing, idx := pq.getTxWithSameNonceUnsafe(tx) + if existing != nil { + if tx.priority > existing.priority { + // should replace + // replace heap if applicable + if hi, ok := pq.findTxIndexUnsafe(existing); ok { + heap.Remove(pq, hi) + heap.Push(pq, tx) // need to be in the heap since it has the same nonce + } + pq.evmQueue[tx.evmAddress][idx] = tx // replace queue item in-place + return existing, false + } + // tx should be dropped since it's dominated by an existing tx + return nil, true + } + } + return nil, false +} + // GetEvictableTxs attempts to find and return a list of *WrappedTx than can be // evicted to make room for another *WrappedTx with higher priority. If no such // list of *WrappedTx exists, nil will be returned. The returned list of *WrappedTx diff --git a/internal/mempool/priority_queue_test.go b/internal/mempool/priority_queue_test.go index f680dc2fc..a319e9888 100644 --- a/internal/mempool/priority_queue_test.go +++ b/internal/mempool/priority_queue_test.go @@ -79,12 +79,11 @@ func TestTxPriorityQueue_PriorityAndNonceOrdering(t *testing.T) { {sender: "2", isEVM: false, priority: 9}, {sender: "4", isEVM: true, evmAddress: "0xabc", evmNonce: 0, priority: 9}, // Same EVM address as first, lower nonce {sender: "5", isEVM: true, evmAddress: "0xdef", evmNonce: 1, priority: 7}, - {sender: "5", isEVM: true, evmAddress: "0xdef", evmNonce: 1, priority: 7}, {sender: "3", isEVM: true, evmAddress: "0xdef", evmNonce: 0, priority: 8}, {sender: "6", isEVM: false, priority: 6}, {sender: "7", isEVM: true, evmAddress: "0xghi", evmNonce: 2, priority: 5}, }, - expectedOutput: []int64{2, 4, 1, 3, 5, 5, 6, 7}, + expectedOutput: []int64{2, 4, 1, 3, 5, 6, 7}, }, { name: "PriorityWithEVMAndNonEVM", @@ -371,3 +370,77 @@ func TestTxPriorityQueue_RemoveTx(t *testing.T) { }) require.Equal(t, numTxs-2, pq.NumTxs()) } + +func TestTxPriorityQueue_TryReplacement(t *testing.T) { + for _, test := range []struct { + tx *WrappedTx + existing []*WrappedTx + expectedReplaced bool + expectedDropped bool + expectedQueue []*WrappedTx + expectedHeap []*WrappedTx + }{ + {&WrappedTx{isEVM: false}, []*WrappedTx{}, false, false, []*WrappedTx{}, []*WrappedTx{}}, + {&WrappedTx{isEVM: true, evmAddress: "addr1"}, []*WrappedTx{}, false, false, []*WrappedTx{}, []*WrappedTx{}}, + { + &WrappedTx{isEVM: true, evmAddress: "addr1", evmNonce: 1, priority: 100, tx: []byte("abc")}, []*WrappedTx{ + {isEVM: true, evmAddress: "addr1", evmNonce: 0, priority: 100, tx: []byte("def")}, + }, false, false, []*WrappedTx{ + {isEVM: true, evmAddress: "addr1", evmNonce: 0, priority: 100, tx: []byte("def")}, + }, []*WrappedTx{ + {isEVM: true, evmAddress: "addr1", evmNonce: 0, priority: 100, tx: []byte("def")}, + }, + }, + { + &WrappedTx{isEVM: true, evmAddress: "addr1", evmNonce: 0, priority: 100, tx: []byte("abc")}, []*WrappedTx{ + {isEVM: true, evmAddress: "addr1", evmNonce: 0, priority: 100, tx: []byte("def")}, + }, false, true, []*WrappedTx{ + {isEVM: true, evmAddress: "addr1", evmNonce: 0, priority: 100, tx: []byte("def")}, + }, []*WrappedTx{ + {isEVM: true, evmAddress: "addr1", evmNonce: 0, priority: 100, tx: []byte("def")}, + }, + }, + { + &WrappedTx{isEVM: true, evmAddress: "addr1", evmNonce: 0, priority: 101, tx: []byte("abc")}, []*WrappedTx{ + {isEVM: true, evmAddress: "addr1", evmNonce: 0, priority: 100, tx: []byte("def")}, + }, true, false, []*WrappedTx{ + {isEVM: true, evmAddress: "addr1", evmNonce: 0, priority: 101, tx: []byte("abc")}, + }, []*WrappedTx{ + {isEVM: true, evmAddress: "addr1", evmNonce: 0, priority: 101, tx: []byte("abc")}, + }, + }, + { + &WrappedTx{isEVM: true, evmAddress: "addr1", evmNonce: 1, priority: 100, tx: []byte("abc")}, []*WrappedTx{ + {isEVM: true, evmAddress: "addr1", evmNonce: 0, priority: 100, tx: []byte("def")}, + {isEVM: true, evmAddress: "addr1", evmNonce: 1, priority: 99, tx: []byte("ghi")}, + }, true, false, []*WrappedTx{ + {isEVM: true, evmAddress: "addr1", evmNonce: 0, priority: 100, tx: []byte("def")}, + {isEVM: true, evmAddress: "addr1", evmNonce: 1, priority: 100, tx: []byte("abc")}, + }, []*WrappedTx{ + {isEVM: true, evmAddress: "addr1", evmNonce: 0, priority: 100, tx: []byte("def")}, + }, + }, + } { + pq := NewTxPriorityQueue() + for _, e := range test.existing { + pq.PushTx(e) + } + replaced, dropped := pq.TryReplacement(test.tx) + if test.expectedReplaced { + require.NotNil(t, replaced) + } else { + require.Nil(t, replaced) + } + require.Equal(t, test.expectedDropped, dropped) + for i, q := range pq.evmQueue[test.tx.evmAddress] { + require.Equal(t, test.expectedQueue[i].tx.Key(), q.tx.Key()) + require.Equal(t, test.expectedQueue[i].priority, q.priority) + require.Equal(t, test.expectedQueue[i].evmNonce, q.evmNonce) + } + for i, q := range pq.txs { + require.Equal(t, test.expectedHeap[i].tx.Key(), q.tx.Key()) + require.Equal(t, test.expectedHeap[i].priority, q.priority) + require.Equal(t, test.expectedHeap[i].evmNonce, q.evmNonce) + } + } +} diff --git a/internal/mempool/reactor_test.go b/internal/mempool/reactor_test.go index 9ab4fbc70..342910d56 100644 --- a/internal/mempool/reactor_test.go +++ b/internal/mempool/reactor_test.go @@ -167,7 +167,7 @@ func TestReactorBroadcastDoesNotPanic(t *testing.T) { secondaryReactor.observePanic = observePanic firstTx := &WrappedTx{} - primaryMempool.insertTx(firstTx) + primaryMempool.insertTx(firstTx, true) // run the router rts.start(ctx, t) @@ -180,7 +180,7 @@ func TestReactorBroadcastDoesNotPanic(t *testing.T) { wg.Add(1) go func() { defer wg.Done() - primaryMempool.insertTx(next) + primaryMempool.insertTx(next, true) }() } diff --git a/internal/mempool/tx.go b/internal/mempool/tx.go index 79c91e039..17846d5bc 100644 --- a/internal/mempool/tx.go +++ b/internal/mempool/tx.go @@ -79,7 +79,7 @@ type WrappedTx struct { // IsBefore returns true if the WrappedTx is before the given WrappedTx // this applies to EVM transactions only func (wtx *WrappedTx) IsBefore(tx *WrappedTx) bool { - return wtx.evmNonce < tx.evmNonce || (wtx.evmNonce == tx.evmNonce && wtx.timestamp.Before(tx.timestamp)) + return wtx.evmNonce < tx.evmNonce } func (wtx *WrappedTx) Size() int { diff --git a/proto/tendermint/abci/types.proto b/proto/tendermint/abci/types.proto index c814e80b2..3bdafdd7b 100644 --- a/proto/tendermint/abci/types.proto +++ b/proto/tendermint/abci/types.proto @@ -309,6 +309,7 @@ message ResponseDeliverTx { repeated Event events = 7 [(gogoproto.nullable) = false, (gogoproto.jsontag) = "events,omitempty"]; // nondeterministic string codespace = 8; + EvmTxInfo evm_tx_info = 9; } message ResponseEndBlock { @@ -459,6 +460,7 @@ message ExecTxResult { repeated Event events = 7 [(gogoproto.nullable) = false, (gogoproto.jsontag) = "events,omitempty"]; // nondeterministic string codespace = 8; + EvmTxInfo evm_tx_info = 9; } // TxResult contains results of executing the transaction. @@ -565,6 +567,11 @@ message Evidence { int64 total_voting_power = 5; } +message EvmTxInfo { + string senderAddress = 1; + uint64 nonce = 2; +} + //---------------------------------------- // State Sync Types diff --git a/proto/tendermint/crypto/keys.pb.go b/proto/tendermint/crypto/keys.pb.go index 62a109cbd..24c6c1b1b 100644 --- a/proto/tendermint/crypto/keys.pb.go +++ b/proto/tendermint/crypto/keys.pb.go @@ -27,7 +27,6 @@ const _ = proto.GoGoProtoPackageIsVersion3 // please upgrade the proto package // PublicKey defines the keys available for use with Tendermint Validators type PublicKey struct { // Types that are valid to be assigned to Sum: - // // *PublicKey_Ed25519 // *PublicKey_Secp256K1 // *PublicKey_Sr25519 diff --git a/proto/tendermint/mempool/types.pb.go b/proto/tendermint/mempool/types.pb.go index ffe5ab359..11e259551 100644 --- a/proto/tendermint/mempool/types.pb.go +++ b/proto/tendermint/mempool/types.pb.go @@ -68,7 +68,6 @@ func (m *Txs) GetTxs() [][]byte { type Message struct { // Types that are valid to be assigned to Sum: - // // *Message_Txs Sum isMessage_Sum `protobuf_oneof:"sum"` } diff --git a/proto/tendermint/p2p/conn.pb.go b/proto/tendermint/p2p/conn.pb.go index 16ee463a6..47a3bb0cd 100644 --- a/proto/tendermint/p2p/conn.pb.go +++ b/proto/tendermint/p2p/conn.pb.go @@ -158,7 +158,6 @@ func (m *PacketMsg) GetData() []byte { type Packet struct { // Types that are valid to be assigned to Sum: - // // *Packet_PacketPing // *Packet_PacketPong // *Packet_PacketMsg diff --git a/proto/tendermint/types/types.pb.go b/proto/tendermint/types/types.pb.go index 1fdefc056..3671bf7a1 100644 --- a/proto/tendermint/types/types.pb.go +++ b/proto/tendermint/types/types.pb.go @@ -1294,7 +1294,6 @@ func (m *TxProof) GetProof() *crypto.Proof { type Evidence struct { // Types that are valid to be assigned to Sum: - // // *Evidence_DuplicateVoteEvidence // *Evidence_LightClientAttackEvidence Sum isEvidence_Sum `protobuf_oneof:"sum"` From 1d2e74cec7c6e4d9acb2e3bee62511b1e85901a0 Mon Sep 17 00:00:00 2001 From: Steven Landers Date: Thu, 21 Mar 2024 10:11:17 -0400 Subject: [PATCH 21/21] hack in old mempool --- internal/mempool/mempool.go | 336 ++++++++------------------ internal/mempool/priority_queue.go | 366 ++--------------------------- internal/mempool/tx.go | 158 ------------- 3 files changed, 114 insertions(+), 746 deletions(-) diff --git a/internal/mempool/mempool.go b/internal/mempool/mempool.go index 80201ef3f..5c6cc4f77 100644 --- a/internal/mempool/mempool.go +++ b/internal/mempool/mempool.go @@ -3,9 +3,9 @@ package mempool import ( "bytes" "context" - "encoding/json" "errors" "fmt" + "strings" "sync" "sync/atomic" "time" @@ -15,6 +15,7 @@ import ( "github.com/tendermint/tendermint/config" "github.com/tendermint/tendermint/internal/libs/clist" "github.com/tendermint/tendermint/libs/log" + tmmath "github.com/tendermint/tendermint/libs/math" "github.com/tendermint/tendermint/types" ) @@ -43,9 +44,6 @@ type TxMempool struct { // sizeBytes defines the total size of the mempool (sum of all tx bytes) sizeBytes int64 - // pendingSizeBytes defines the total size of the pending set (sum of all tx bytes) - pendingSizeBytes int64 - // cache defines a fixed-size cache of already seen transactions as this // reduces pressure on the proxyApp. cache TxCache @@ -84,10 +82,6 @@ type TxMempool struct { // index. i.e. older transactions are first. timestampIndex *WrappedTxList - // pendingTxs stores transactions that are not valid yet but might become valid - // if its checker returns Accepted - pendingTxs *PendingTxs - // A read/write lock is used to safe guard updates, insertions and deletions // from the mempool. A read-lock is implicitly acquired when executing CheckTx, // however, a caller must explicitly grab a write-lock via Lock when updating @@ -127,7 +121,6 @@ func NewTxMempool( timestampIndex: NewWrappedTxList(func(wtx1, wtx2 *WrappedTx) bool { return wtx1.timestamp.After(wtx2.timestamp) || wtx1.timestamp.Equal(wtx2.timestamp) }), - pendingTxs: NewPendingTxs(cfg), failedCheckTxCounts: map[types.NodeID]uint64{}, peerManager: peerManager, } @@ -180,28 +173,15 @@ func (txmp *TxMempool) Unlock() { // Size returns the number of valid transactions in the mempool. It is // thread-safe. func (txmp *TxMempool) Size() int { - return txmp.SizeWithoutPending() + txmp.PendingSize() -} - -func (txmp *TxMempool) SizeWithoutPending() int { return txmp.txStore.Size() } -// PendingSize returns the number of pending transactions in the mempool. -func (txmp *TxMempool) PendingSize() int { - return txmp.pendingTxs.Size() -} - // SizeBytes return the total sum in bytes of all the valid transactions in the // mempool. It is thread-safe. func (txmp *TxMempool) SizeBytes() int64 { return atomic.LoadInt64(&txmp.sizeBytes) } -func (txmp *TxMempool) PendingSizeBytes() int64 { - return atomic.LoadInt64(&txmp.pendingSizeBytes) -} - // FlushAppConn executes FlushSync on the mempool's proxyAppConn. // // NOTE: The caller must obtain a write-lock prior to execution. @@ -295,54 +275,24 @@ func (txmp *TxMempool) CheckTx( } res, err := txmp.proxyAppConn.CheckTx(ctx, &abci.RequestCheckTx{Tx: tx}) - - // when a transaction is removed/expired/rejected, this should be called - // The expire tx handler unreserves the pending nonce - removeHandler := func(removeFromCache bool) { - if removeFromCache { - txmp.cache.Remove(tx) - } - if res.ExpireTxHandler != nil { - res.ExpireTxHandler() - } - } - if err != nil { - removeHandler(true) + txmp.cache.Remove(tx) res.Log = txmp.AppendCheckTxErr(res.Log, err.Error()) } wtx := &WrappedTx{ - tx: tx, - hash: txHash, - timestamp: time.Now().UTC(), - height: txmp.height, - evmNonce: res.EVMNonce, - evmAddress: res.EVMSenderAddress, - isEVM: res.IsEVM, - removeHandler: removeHandler, + tx: tx, + hash: txHash, + timestamp: time.Now().UTC(), + height: txmp.height, } + // only add new transaction if checkTx passes if err == nil { - // only add new transaction if checkTx passes and is not pending - if !res.IsPendingTransaction { - err = txmp.addNewTransaction(wtx, res.ResponseCheckTx, txInfo) - if err != nil { - return err - } - } else { - // otherwise add to pending txs store - if res.Checker == nil { - return errors.New("no checker available for pending transaction") - } - if err := txmp.canAddPendingTx(wtx); err != nil { - // TODO: eviction strategy for pending transactions - return err - } - atomic.AddInt64(&txmp.pendingSizeBytes, int64(wtx.Size())) - if err := txmp.pendingTxs.Insert(wtx, res, txInfo); err != nil { - return err - } + err = txmp.addNewTransaction(wtx, res.ResponseCheckTx, txInfo) + + if err != nil { + return err } } @@ -353,18 +303,13 @@ func (txmp *TxMempool) CheckTx( return nil } -func (txmp *TxMempool) isInMempool(tx types.Tx) bool { - existingTx := txmp.txStore.GetTxByHash(tx.Key()) - return existingTx != nil && !existingTx.removed -} - func (txmp *TxMempool) RemoveTxByKey(txKey types.TxKey) error { txmp.Lock() defer txmp.Unlock() // remove the committed transaction from the transaction store and indexes if wtx := txmp.txStore.GetTxByHash(txKey); wtx != nil { - txmp.removeTx(wtx, false, true, true) + txmp.removeTx(wtx, false) return nil } @@ -403,7 +348,7 @@ func (txmp *TxMempool) Flush() { txmp.timestampIndex.Reset() for _, wtx := range txmp.txStore.GetAllTxs() { - txmp.removeTx(wtx, false, false, true) + txmp.removeTx(wtx, false) } atomic.SwapInt64(&txmp.sizeBytes, 0) @@ -425,28 +370,42 @@ func (txmp *TxMempool) ReapMaxBytesMaxGas(maxBytes, maxGas int64) types.Txs { totalSize int64 ) - var txs []types.Tx - if uint64(txmp.SizeWithoutPending()) < txmp.config.TxNotifyThreshold { + // wTxs contains a list of *WrappedTx retrieved from the priority queue that + // need to be re-enqueued prior to returning. + wTxs := make([]*WrappedTx, 0, txmp.priorityIndex.NumTxs()) + defer func() { + for _, wtx := range wTxs { + txmp.priorityIndex.PushTx(wtx) + } + }() + + txs := make([]types.Tx, 0, txmp.priorityIndex.NumTxs()) + if uint64(txmp.Size()) < txmp.config.TxNotifyThreshold { // do not reap anything if threshold is not met return txs } - txmp.priorityIndex.ForEachTx(func(wtx *WrappedTx) bool { + for txmp.priorityIndex.NumTxs() > 0 { + wtx := txmp.priorityIndex.PopTx() + txs = append(txs, wtx.tx) + wTxs = append(wTxs, wtx) size := types.ComputeProtoSizeForTxs([]types.Tx{wtx.tx}) + // Ensure we have capacity for the transaction with respect to the + // transaction size. if maxBytes > -1 && totalSize+size > maxBytes { - return false + return txs[:len(txs)-1] } + totalSize += size + + // ensure we have capacity for the transaction with respect to total gas gas := totalGas + wtx.gasWanted if maxGas > -1 && gas > maxGas { - return false + return txs[:len(txs)-1] } totalGas = gas - - txs = append(txs, wtx.tx) - return true - }) + } return txs } @@ -461,17 +420,24 @@ func (txmp *TxMempool) ReapMaxTxs(max int) types.Txs { txmp.mtx.RLock() defer txmp.mtx.RUnlock() - wTxs := txmp.priorityIndex.PeekTxs(max) - txs := make([]types.Tx, 0, len(wTxs)) - for _, wtx := range wTxs { + numTxs := txmp.priorityIndex.NumTxs() + if max < 0 { + max = numTxs + } + + cap := tmmath.MinInt(numTxs, max) + + // wTxs contains a list of *WrappedTx retrieved from the priority queue that + // need to be re-enqueued prior to returning. + wTxs := make([]*WrappedTx, 0, cap) + txs := make([]types.Tx, 0, cap) + for txmp.priorityIndex.NumTxs() > 0 && len(txs) < max { + wtx := txmp.priorityIndex.PopTx() txs = append(txs, wtx.tx) + wTxs = append(wTxs, wtx) } - if len(txs) < max { - // retrieve more from pending txs - pending := txmp.pendingTxs.Peek(max - len(txs)) - for _, ptx := range pending { - txs = append(txs, ptx.tx.tx) - } + for _, wtx := range wTxs { + txmp.priorityIndex.PushTx(wtx) } return txs } @@ -515,22 +481,11 @@ func (txmp *TxMempool) Update( // remove the committed transaction from the transaction store and indexes if wtx := txmp.txStore.GetTxByHash(tx.Key()); wtx != nil { - txmp.removeTx(wtx, false, false, true) - } - if execTxResult[i].EvmTxInfo != nil { - // remove any tx that has the same nonce (because the committed tx - // may be from block proposal and is never in the local mempool) - if wtx, _ := txmp.priorityIndex.GetTxWithSameNonce(&WrappedTx{ - evmAddress: execTxResult[i].EvmTxInfo.SenderAddress, - evmNonce: execTxResult[i].EvmTxInfo.Nonce, - }); wtx != nil { - txmp.removeTx(wtx, false, false, true) - } + txmp.removeTx(wtx, false) } } txmp.purgeExpiredTxs(blockHeight) - txmp.handlePendingTransactions() // If there any uncommitted transactions left in the mempool, we either // initiate re-CheckTx per remaining transaction or notify that remaining @@ -548,8 +503,7 @@ func (txmp *TxMempool) Update( } } - txmp.metrics.Size.Set(float64(txmp.SizeWithoutPending())) - txmp.metrics.PendingSize.Set(float64(txmp.PendingSize())) + txmp.metrics.Size.Set(float64(txmp.Size())) return nil } @@ -592,7 +546,9 @@ func (txmp *TxMempool) addNewTransaction(wtx *WrappedTx, res *abci.ResponseCheck txmp.metrics.FailedTxs.Add(1) - wtx.removeHandler(!txmp.config.KeepInvalidTxsInCache) + if !txmp.config.KeepInvalidTxsInCache { + txmp.cache.Remove(wtx.tx) + } if res.Code != abci.CodeTypeOK { txmp.mtxFailedCheckTxCounts.Lock() defer txmp.mtxFailedCheckTxCounts.Unlock() @@ -630,7 +586,7 @@ func (txmp *TxMempool) addNewTransaction(wtx *WrappedTx, res *abci.ResponseCheck if len(evictTxs) == 0 { // No room for the new incoming transaction so we just remove it from // the cache. - wtx.removeHandler(true) + txmp.cache.Remove(wtx.tx) txmp.logger.Error( "rejected incoming good transaction; mempool full", "tx", fmt.Sprintf("%X", wtx.tx.Hash()), @@ -646,7 +602,7 @@ func (txmp *TxMempool) addNewTransaction(wtx *WrappedTx, res *abci.ResponseCheck // - The transaction, toEvict, can be removed while a concurrent // reCheckTx callback is being executed for the same transaction. for _, toEvict := range evictTxs { - txmp.removeTx(toEvict, true, true, true) + txmp.removeTx(toEvict, true) txmp.logger.Debug( "evicted existing good transaction; mempool full", "old_tx", fmt.Sprintf("%X", toEvict.tx.Hash()), @@ -665,29 +621,18 @@ func (txmp *TxMempool) addNewTransaction(wtx *WrappedTx, res *abci.ResponseCheck txInfo.SenderID: {}, } - replaced, shouldDrop := txmp.priorityIndex.TryReplacement(wtx) - if shouldDrop { - return nil - } - txmp.metrics.TxSizeBytes.Observe(float64(wtx.Size())) - txmp.metrics.Size.Set(float64(txmp.SizeWithoutPending())) - txmp.metrics.PendingSize.Set(float64(txmp.PendingSize())) - - if replaced != nil { - txmp.removeTx(replaced, true, false, false) - } - if txmp.insertTx(wtx, replaced == nil) { - txmp.logger.Debug( - "inserted good transaction", - "priority", wtx.priority, - "tx", fmt.Sprintf("%X", wtx.tx.Hash()), - "height", txmp.height, - "num_txs", txmp.SizeWithoutPending(), - ) - txmp.notifyTxsAvailable() - } + txmp.metrics.Size.Set(float64(txmp.Size())) + txmp.insertTx(wtx) + txmp.logger.Debug( + "inserted good transaction", + "priority", wtx.priority, + "tx", fmt.Sprintf("%X", wtx.tx.Hash()), + "height", txmp.height, + "num_txs", txmp.Size(), + ) + txmp.notifyTxsAvailable() return nil } @@ -703,7 +648,7 @@ func (txmp *TxMempool) addNewTransaction(wtx *WrappedTx, res *abci.ResponseCheck // // This method is NOT executed for the initial CheckTx on a new transaction; // that case is handled by addNewTransaction instead. -func (txmp *TxMempool) handleRecheckResult(tx types.Tx, res *abci.ResponseCheckTxV2) { +func (txmp *TxMempool) handleRecheckResult(tx types.Tx, res *abci.ResponseCheckTx) { if txmp.recheckCursor == nil { return } @@ -746,11 +691,10 @@ func (txmp *TxMempool) handleRecheckResult(tx types.Tx, res *abci.ResponseCheckT if !txmp.txStore.IsTxRemoved(wtx.hash) { var err error if txmp.postCheck != nil { - err = txmp.postCheck(tx, res.ResponseCheckTx) + err = txmp.postCheck(tx, res) } - // we will treat a transaction that turns pending in a recheck as invalid and evict it - if res.Code == abci.CodeTypeOK && err == nil && !res.IsPendingTransaction { + if res.Code == abci.CodeTypeOK && err == nil { wtx.priority = res.Priority } else { txmp.logger.Debug( @@ -765,7 +709,7 @@ func (txmp *TxMempool) handleRecheckResult(tx types.Tx, res *abci.ResponseCheckT panic("corrupted reCheckTx cursor") } - txmp.removeTx(wtx, !txmp.config.KeepInvalidTxsInCache, true, true) + txmp.removeTx(wtx, !txmp.config.KeepInvalidTxsInCache) } } @@ -779,13 +723,12 @@ func (txmp *TxMempool) handleRecheckResult(tx types.Tx, res *abci.ResponseCheckT if txmp.recheckCursor == nil { txmp.logger.Debug("finished rechecking transactions") - if txmp.SizeWithoutPending() > 0 { + if txmp.Size() > 0 { txmp.notifyTxsAvailable() } } - txmp.metrics.Size.Set(float64(txmp.SizeWithoutPending())) - txmp.metrics.PendingSize.Set(float64(txmp.PendingSize())) + txmp.metrics.Size.Set(float64(txmp.Size())) } // updateReCheckTxs updates the recheck cursors using the gossipIndex. For @@ -823,7 +766,7 @@ func (txmp *TxMempool) updateReCheckTxs(ctx context.Context) { txmp.logger.Debug("failed to execute CheckTx during recheck", "err", err, "hash", fmt.Sprintf("%x", wtx.tx.Hash())) continue } - txmp.handleRecheckResult(wtx.tx, res) + txmp.handleRecheckResult(wtx.tx, res.ResponseCheckTx) } } @@ -837,7 +780,7 @@ func (txmp *TxMempool) updateReCheckTxs(ctx context.Context) { // the transaction can be inserted into the mempool. func (txmp *TxMempool) canAddTx(wtx *WrappedTx) error { var ( - numTxs = txmp.SizeWithoutPending() + numTxs = txmp.Size() sizeBytes = txmp.SizeBytes() ) @@ -853,33 +796,9 @@ func (txmp *TxMempool) canAddTx(wtx *WrappedTx) error { return nil } -func (txmp *TxMempool) canAddPendingTx(wtx *WrappedTx) error { - var ( - numTxs = txmp.PendingSize() - sizeBytes = txmp.PendingSizeBytes() - ) - - if numTxs >= txmp.config.PendingSize || int64(wtx.Size())+sizeBytes > txmp.config.MaxPendingTxsBytes { - return types.ErrMempoolPendingIsFull{ - NumTxs: numTxs, - MaxTxs: txmp.config.PendingSize, - TxsBytes: sizeBytes, - MaxTxsBytes: txmp.config.MaxPendingTxsBytes, - } - } - - return nil -} - -func (txmp *TxMempool) insertTx(wtx *WrappedTx, updatePriorityIndex bool) bool { - if txmp.isInMempool(wtx.tx) { - return false - } - +func (txmp *TxMempool) insertTx(wtx *WrappedTx) { txmp.txStore.SetTx(wtx) - if updatePriorityIndex { - txmp.priorityIndex.PushTx(wtx) - } + txmp.priorityIndex.PushTx(wtx) txmp.heightIndex.Insert(wtx) txmp.timestampIndex.Insert(wtx) @@ -890,19 +809,15 @@ func (txmp *TxMempool) insertTx(wtx *WrappedTx, updatePriorityIndex bool) bool { wtx.gossipEl = gossipEl atomic.AddInt64(&txmp.sizeBytes, int64(wtx.Size())) - return true } -func (txmp *TxMempool) removeTx(wtx *WrappedTx, removeFromCache bool, shouldReenqueue bool, updatePriorityIndex bool) { +func (txmp *TxMempool) removeTx(wtx *WrappedTx, removeFromCache bool) { if txmp.txStore.IsTxRemoved(wtx.hash) { return } txmp.txStore.RemoveTx(wtx) - toBeReenqueued := []*WrappedTx{} - if updatePriorityIndex { - toBeReenqueued = txmp.priorityIndex.RemoveTx(wtx, shouldReenqueue) - } + txmp.priorityIndex.RemoveTx(wtx) txmp.heightIndex.Remove(wtx) txmp.timestampIndex.Remove(wtx) @@ -913,52 +828,14 @@ func (txmp *TxMempool) removeTx(wtx *WrappedTx, removeFromCache bool, shouldReen atomic.AddInt64(&txmp.sizeBytes, int64(-wtx.Size())) - wtx.removeHandler(removeFromCache) - - if shouldReenqueue { - for _, reenqueue := range toBeReenqueued { - txmp.removeTx(reenqueue, removeFromCache, false, true) - } - for _, reenqueue := range toBeReenqueued { - rtx := reenqueue.tx - go func() { - if err := txmp.CheckTx(context.Background(), rtx, nil, TxInfo{}); err != nil { - txmp.logger.Error(fmt.Sprintf("failed to reenqueue transaction %X due to %s", rtx.Hash(), err)) - } - }() - } - } -} - -func (txmp *TxMempool) expire(blockHeight int64, wtx *WrappedTx) { - txmp.metrics.ExpiredTxs.Add(1) - txmp.logExpiredTx(blockHeight, wtx) - wtx.removeHandler(!txmp.config.KeepInvalidTxsInCache) -} - -func (txmp *TxMempool) logExpiredTx(blockHeight int64, wtx *WrappedTx) { - // defensive check - if wtx == nil { - return + if removeFromCache { + txmp.cache.Remove(wtx.tx) } - - txmp.logger.Info( - "transaction expired", - "priority", wtx.priority, - "tx", fmt.Sprintf("%X", wtx.tx.Hash()), - "address", wtx.evmAddress, - "evm", wtx.isEVM, - "nonce", wtx.evmNonce, - "height", blockHeight, - "tx_height", wtx.height, - "tx_timestamp", wtx.timestamp, - "age", time.Since(wtx.timestamp), - ) } // purgeExpiredTxs removes all transactions that have exceeded their respective // height- and/or time-based TTLs from their respective indexes. Every expired -// transaction will be removed from the mempool, but preserved in the cache (except for pending txs). +// transaction will be removed from the mempool, but preserved in the cache. // // NOTE: purgeExpiredTxs must only be called during TxMempool#Update in which // the caller has a write-lock on the mempool and so we can safely iterate over @@ -1002,18 +879,12 @@ func (txmp *TxMempool) purgeExpiredTxs(blockHeight int64) { } for _, wtx := range expiredTxs { - txmp.expire(blockHeight, wtx) + txmp.removeTx(wtx, false) } - - // remove pending txs that have expired - txmp.pendingTxs.PurgeExpired(blockHeight, now, func(wtx *WrappedTx) { - atomic.AddInt64(&txmp.pendingSizeBytes, int64(-wtx.Size())) - txmp.expire(blockHeight, wtx) - }) } func (txmp *TxMempool) notifyTxsAvailable() { - if txmp.SizeWithoutPending() == 0 { + if txmp.Size() == 0 { return } @@ -1036,31 +907,14 @@ func (txmp *TxMempool) GetPeerFailedCheckTxCount(nodeID types.NodeID) uint64 { // AppendCheckTxErr wraps error message into an ABCIMessageLogs json string func (txmp *TxMempool) AppendCheckTxErr(existingLogs string, log string) string { - var logs []map[string]interface{} - json.Unmarshal([]byte(existingLogs), &logs) - - // Append the new ABCIMessageLog to the slice - logs = append(logs, map[string]interface{}{ - "log": log, - }) - - // Marshal the updated slice back into a JSON string - jsonData, _ := json.Marshal(logs) - return string(jsonData) -} + var builder strings.Builder -func (txmp *TxMempool) handlePendingTransactions() { - accepted, rejected := txmp.pendingTxs.EvaluatePendingTransactions() - for _, tx := range accepted { - atomic.AddInt64(&txmp.pendingSizeBytes, int64(-tx.tx.Size())) - if err := txmp.addNewTransaction(tx.tx, tx.checkTxResponse.ResponseCheckTx, tx.txInfo); err != nil { - txmp.logger.Error(fmt.Sprintf("error adding pending transaction: %s", err)) - } - } - for _, tx := range rejected { - atomic.AddInt64(&txmp.pendingSizeBytes, int64(-tx.tx.Size())) - if !txmp.config.KeepInvalidTxsInCache { - tx.tx.removeHandler(true) - } + builder.WriteString(existingLogs) + // If there are already logs, append the new log with a separator + if builder.Len() > 0 { + builder.WriteString("; ") } + builder.WriteString(log) + + return builder.String() } diff --git a/internal/mempool/priority_queue.go b/internal/mempool/priority_queue.go index 0dcc6edd0..e31997397 100644 --- a/internal/mempool/priority_queue.go +++ b/internal/mempool/priority_queue.go @@ -4,8 +4,6 @@ import ( "container/heap" "sort" "sync" - - tmmath "github.com/tendermint/tendermint/libs/math" ) var _ heap.Interface = (*TxPriorityQueue)(nil) @@ -13,39 +11,12 @@ var _ heap.Interface = (*TxPriorityQueue)(nil) // TxPriorityQueue defines a thread-safe priority queue for valid transactions. type TxPriorityQueue struct { mtx sync.RWMutex - txs []*WrappedTx // priority heap - // invariant 1: no duplicate nonce in the same queue - // invariant 2: no nonce gap in the same queue - // invariant 3: head of the queue must be in heap - evmQueue map[string][]*WrappedTx // sorted by nonce -} - -func insertToEVMQueue(queue []*WrappedTx, tx *WrappedTx, i int) []*WrappedTx { - // Make room for new value and add it - queue = append(queue, nil) - copy(queue[i+1:], queue[i:]) - queue[i] = tx - return queue -} - -// binarySearch finds the index at which tx should be inserted in queue -func binarySearch(queue []*WrappedTx, tx *WrappedTx) int { - low, high := 0, len(queue) - for low < high { - mid := low + (high-low)/2 - if queue[mid].IsBefore(tx) { - low = mid + 1 - } else { - high = mid - } - } - return low + txs []*WrappedTx } func NewTxPriorityQueue() *TxPriorityQueue { pq := &TxPriorityQueue{ - txs: make([]*WrappedTx, 0), - evmQueue: make(map[string][]*WrappedTx), + txs: make([]*WrappedTx, 0), } heap.Init(pq) @@ -53,51 +24,6 @@ func NewTxPriorityQueue() *TxPriorityQueue { return pq } -func (pq *TxPriorityQueue) GetTxWithSameNonce(tx *WrappedTx) (*WrappedTx, int) { - pq.mtx.RLock() - defer pq.mtx.RUnlock() - return pq.getTxWithSameNonceUnsafe(tx) -} - -func (pq *TxPriorityQueue) getTxWithSameNonceUnsafe(tx *WrappedTx) (*WrappedTx, int) { - queue, ok := pq.evmQueue[tx.evmAddress] - if !ok { - return nil, -1 - } - idx := binarySearch(queue, tx) - if idx < len(queue) && queue[idx].evmNonce == tx.evmNonce { - return queue[idx], idx - } - return nil, -1 -} - -func (pq *TxPriorityQueue) TryReplacement(tx *WrappedTx) (replaced *WrappedTx, shouldDrop bool) { - if !tx.isEVM { - return nil, false - } - pq.mtx.Lock() - defer pq.mtx.Unlock() - queue, ok := pq.evmQueue[tx.evmAddress] - if ok && len(queue) > 0 { - existing, idx := pq.getTxWithSameNonceUnsafe(tx) - if existing != nil { - if tx.priority > existing.priority { - // should replace - // replace heap if applicable - if hi, ok := pq.findTxIndexUnsafe(existing); ok { - heap.Remove(pq, hi) - heap.Push(pq, tx) // need to be in the heap since it has the same nonce - } - pq.evmQueue[tx.evmAddress][idx] = tx // replace queue item in-place - return existing, false - } - // tx should be dropped since it's dominated by an existing tx - return nil, true - } - } - return nil, false -} - // GetEvictableTxs attempts to find and return a list of *WrappedTx than can be // evicted to make room for another *WrappedTx with higher priority. If no such // list of *WrappedTx exists, nil will be returned. The returned list of *WrappedTx @@ -108,11 +34,8 @@ func (pq *TxPriorityQueue) GetEvictableTxs(priority, txSize, totalSize, cap int6 pq.mtx.RLock() defer pq.mtx.RUnlock() - txs := []*WrappedTx{} - txs = append(txs, pq.txs...) - for _, queue := range pq.evmQueue { - txs = append(txs, queue[1:]...) - } + txs := make([]*WrappedTx, len(pq.txs)) + copy(txs, pq.txs) sort.Slice(txs, func(i, j int) bool { return txs[i].priority < txs[j].priority @@ -143,237 +66,31 @@ func (pq *TxPriorityQueue) GetEvictableTxs(priority, txSize, totalSize, cap int6 return nil } -// requires read lock -func (pq *TxPriorityQueue) numQueuedUnsafe() int { - var result int - for _, queue := range pq.evmQueue { - result += len(queue) - } - // first items in queue are also in heap, subtract one - return result - len(pq.evmQueue) -} - // NumTxs returns the number of transactions in the priority queue. It is // thread safe. func (pq *TxPriorityQueue) NumTxs() int { pq.mtx.RLock() defer pq.mtx.RUnlock() - return len(pq.txs) + pq.numQueuedUnsafe() -} - -func (pq *TxPriorityQueue) removeQueuedEvmTxUnsafe(tx *WrappedTx) (removedIdx int) { - if queue, ok := pq.evmQueue[tx.evmAddress]; ok { - for i, t := range queue { - if t.tx.Key() == tx.tx.Key() { - pq.evmQueue[tx.evmAddress] = append(queue[:i], queue[i+1:]...) - if len(pq.evmQueue[tx.evmAddress]) == 0 { - delete(pq.evmQueue, tx.evmAddress) - } - return i - } - } - } - return -1 -} - -func (pq *TxPriorityQueue) findTxIndexUnsafe(tx *WrappedTx) (int, bool) { - for i, t := range pq.txs { - if t.tx.Key() == tx.tx.Key() { - return i, true - } - } - return 0, false + return len(pq.txs) } // RemoveTx removes a specific transaction from the priority queue. -func (pq *TxPriorityQueue) RemoveTx(tx *WrappedTx, shouldReenqueue bool) (toBeReenqueued []*WrappedTx) { +func (pq *TxPriorityQueue) RemoveTx(tx *WrappedTx) { pq.mtx.Lock() defer pq.mtx.Unlock() - var removedIdx int - - if idx, ok := pq.findTxIndexUnsafe(tx); ok { - heap.Remove(pq, idx) - if tx.isEVM { - removedIdx = pq.removeQueuedEvmTxUnsafe(tx) - if !shouldReenqueue && len(pq.evmQueue[tx.evmAddress]) > 0 { - heap.Push(pq, pq.evmQueue[tx.evmAddress][0]) - } - } - } else if tx.isEVM { - removedIdx = pq.removeQueuedEvmTxUnsafe(tx) - } - if tx.isEVM && shouldReenqueue && len(pq.evmQueue[tx.evmAddress]) > 0 && removedIdx >= 0 { - toBeReenqueued = pq.evmQueue[tx.evmAddress][removedIdx:] - } - return -} - -func (pq *TxPriorityQueue) pushTxUnsafe(tx *WrappedTx) { - if !tx.isEVM { - heap.Push(pq, tx) - return - } - - // if there aren't other waiting txs, init and return - queue, exists := pq.evmQueue[tx.evmAddress] - if !exists { - pq.evmQueue[tx.evmAddress] = []*WrappedTx{tx} - heap.Push(pq, tx) - return - } - - // this item is on the heap at the moment - first := queue[0] - - // the queue's first item (and ONLY the first item) must be on the heap - // if this tx is before the first item, then we need to remove the first - // item from the heap - if tx.IsBefore(first) { - if idx, ok := pq.findTxIndexUnsafe(first); ok { - heap.Remove(pq, idx) - } - heap.Push(pq, tx) + if tx.heapIndex < len(pq.txs) { + heap.Remove(pq, tx.heapIndex) } - pq.evmQueue[tx.evmAddress] = insertToEVMQueue(queue, tx, binarySearch(queue, tx)) } -// These are available if we need to test the invariant checks -// these can be used to troubleshoot invariant violations -//func (pq *TxPriorityQueue) checkInvariants(msg string) { -// uniqHashes := make(map[string]bool) -// for idx, tx := range pq.txs { -// if tx == nil { -// pq.print() -// panic(fmt.Sprintf("DEBUG PRINT: found nil item on heap: idx=%d\n", idx)) -// } -// if tx.tx == nil { -// pq.print() -// panic(fmt.Sprintf("DEBUG PRINT: found nil tx.tx on heap: idx=%d\n", idx)) -// } -// if _, ok := uniqHashes[fmt.Sprintf("%x", tx.tx.Key())]; ok { -// pq.print() -// panic(fmt.Sprintf("INVARIANT (%s): duplicate hash=%x in heap", msg, tx.tx.Key())) -// } -// uniqHashes[fmt.Sprintf("%x", tx.tx.Key())] = true -// -// //if _, ok := pq.keys[tx.tx.Key()]; !ok { -// // pq.print() -// // panic(fmt.Sprintf("INVARIANT (%s): tx in heap but not in keys hash=%x", msg, tx.tx.Key())) -// //} -// -// if tx.isEVM { -// if queue, ok := pq.evmQueue[tx.evmAddress]; ok { -// if queue[0].tx.Key() != tx.tx.Key() { -// pq.print() -// panic(fmt.Sprintf("INVARIANT (%s): tx in heap but not at front of evmQueue hash=%x", msg, tx.tx.Key())) -// } -// } else { -// pq.print() -// panic(fmt.Sprintf("INVARIANT (%s): tx in heap but not in evmQueue hash=%x", msg, tx.tx.Key())) -// } -// } -// } -// -// // each item in all queues should be unique nonce -// for _, queue := range pq.evmQueue { -// hashes := make(map[string]bool) -// for idx, tx := range queue { -// if idx == 0 { -// _, ok := pq.findTxIndexUnsafe(tx) -// if !ok { -// pq.print() -// panic(fmt.Sprintf("INVARIANT (%s): did not find tx[0] hash=%x nonce=%d in heap", msg, tx.tx.Key(), tx.evmNonce)) -// } -// } -// //if _, ok := pq.keys[tx.tx.Key()]; !ok { -// // pq.print() -// // panic(fmt.Sprintf("INVARIANT (%s): tx in heap but not in keys hash=%x", msg, tx.tx.Key())) -// //} -// if _, ok := hashes[fmt.Sprintf("%x", tx.tx.Key())]; ok { -// pq.print() -// panic(fmt.Sprintf("INVARIANT (%s): duplicate hash=%x in queue nonce=%d", msg, tx.tx.Key(), tx.evmNonce)) -// } -// hashes[fmt.Sprintf("%x", tx.tx.Key())] = true -// } -// } -//} - -// for debugging situations where invariant violations occur -//func (pq *TxPriorityQueue) print() { -// fmt.Println("PRINT PRIORITY QUEUE ****************** ") -// for _, tx := range pq.txs { -// if tx == nil { -// fmt.Printf("DEBUG PRINT: heap (nil): nonce=?, hash=?\n") -// continue -// } -// if tx.tx == nil { -// fmt.Printf("DEBUG PRINT: heap (%s): nonce=%d, tx.tx is nil \n", tx.evmAddress, tx.evmNonce) -// continue -// } -// fmt.Printf("DEBUG PRINT: heap (%s): nonce=%d, hash=%x, time=%d\n", tx.evmAddress, tx.evmNonce, tx.tx.Key(), tx.timestamp.UnixNano()) -// } -// -// for addr, queue := range pq.evmQueue { -// for idx, tx := range queue { -// if tx == nil { -// fmt.Printf("DEBUG PRINT: found nil item on evmQueue(%s): idx=%d\n", addr, idx) -// continue -// } -// if tx.tx == nil { -// fmt.Printf("DEBUG PRINT: found nil tx.tx on evmQueue(%s): idx=%d\n", addr, idx) -// continue -// } -// -// fmt.Printf("DEBUG PRINT: evmQueue(%s)[%d]: nonce=%d, hash=%x, time=%d\n", tx.evmAddress, idx, tx.evmNonce, tx.tx.Key(), tx.timestamp.UnixNano()) -// } -// } -//} - // PushTx adds a valid transaction to the priority queue. It is thread safe. func (pq *TxPriorityQueue) PushTx(tx *WrappedTx) { pq.mtx.Lock() defer pq.mtx.Unlock() - pq.pushTxUnsafe(tx) -} - -func (pq *TxPriorityQueue) popTxUnsafe() *WrappedTx { - if len(pq.txs) == 0 { - return nil - } - - // remove the first item from the heap - x := heap.Pop(pq) - if x == nil { - return nil - } - tx := x.(*WrappedTx) - - // this situation is primarily for a test case that inserts nils - if tx == nil { - return nil - } - - // non-evm transactions do not have txs waiting on a nonce - if !tx.isEVM { - return tx - } - - // evm transactions can have txs waiting on this nonce - // if there are any, we should replace the heap with the next nonce - // for the address - - // remove the first item from the evmQueue - pq.removeQueuedEvmTxUnsafe(tx) - - // if there is a next item, now it can be added to the heap - if len(pq.evmQueue[tx.evmAddress]) > 0 { - heap.Push(pq, pq.evmQueue[tx.evmAddress][0]) - } - - return tx + heap.Push(pq, tx) } // PopTx removes the top priority transaction from the queue. It is thread safe. @@ -381,69 +98,21 @@ func (pq *TxPriorityQueue) PopTx() *WrappedTx { pq.mtx.Lock() defer pq.mtx.Unlock() - return pq.popTxUnsafe() -} - -// dequeue up to `max` transactions and reenqueue while locked -func (pq *TxPriorityQueue) ForEachTx(handler func(wtx *WrappedTx) bool) { - pq.mtx.Lock() - defer pq.mtx.Unlock() - - numTxs := len(pq.txs) + pq.numQueuedUnsafe() - - txs := make([]*WrappedTx, 0, numTxs) - - defer func() { - for _, tx := range txs { - pq.pushTxUnsafe(tx) - } - }() - - for i := 0; i < numTxs; i++ { - popped := pq.popTxUnsafe() - if popped == nil { - break - } - txs = append(txs, popped) - if !handler(popped) { - return - } - } -} - -// dequeue up to `max` transactions and reenqueue while locked -// TODO: use ForEachTx instead -func (pq *TxPriorityQueue) PeekTxs(max int) []*WrappedTx { - pq.mtx.Lock() - defer pq.mtx.Unlock() - - numTxs := len(pq.txs) + pq.numQueuedUnsafe() - if max < 0 { - max = numTxs - } - - cap := tmmath.MinInt(numTxs, max) - res := make([]*WrappedTx, 0, cap) - for i := 0; i < cap; i++ { - popped := pq.popTxUnsafe() - if popped == nil { - break - } - - res = append(res, popped) + x := heap.Pop(pq) + if x != nil { + return x.(*WrappedTx) } - for _, tx := range res { - pq.pushTxUnsafe(tx) - } - return res + return nil } // Push implements the Heap interface. // // NOTE: A caller should never call Push. Use PushTx instead. func (pq *TxPriorityQueue) Push(x interface{}) { + n := len(pq.txs) item := x.(*WrappedTx) + item.heapIndex = n pq.txs = append(pq.txs, item) } @@ -454,7 +123,8 @@ func (pq *TxPriorityQueue) Pop() interface{} { old := pq.txs n := len(old) item := old[n-1] - old[n-1] = nil // avoid memory leak + old[n-1] = nil // avoid memory leak + item.heapIndex = -1 // for safety pq.txs = old[0 : n-1] return item } @@ -483,4 +153,6 @@ func (pq *TxPriorityQueue) Less(i, j int) bool { // Swap implements the Heap interface. It swaps two transactions in the queue. func (pq *TxPriorityQueue) Swap(i, j int) { pq.txs[i], pq.txs[j] = pq.txs[j], pq.txs[i] + pq.txs[i].heapIndex = i + pq.txs[j].heapIndex = j } diff --git a/internal/mempool/tx.go b/internal/mempool/tx.go index 17846d5bc..9b51e3eb4 100644 --- a/internal/mempool/tx.go +++ b/internal/mempool/tx.go @@ -1,13 +1,10 @@ package mempool import ( - "errors" "sort" "sync" "time" - abci "github.com/tendermint/tendermint/abci/types" - "github.com/tendermint/tendermint/config" "github.com/tendermint/tendermint/internal/libs/clist" "github.com/tendermint/tendermint/types" ) @@ -66,20 +63,6 @@ type WrappedTx struct { // transaction in the mempool can be evicted when it is simultaneously having // a reCheckTx callback executed. removed bool - - // this is the callback that can be called when a transaction is removed - removeHandler func(removeFromCache bool) - - // evm properties that aid in prioritization - evmAddress string - evmNonce uint64 - isEVM bool -} - -// IsBefore returns true if the WrappedTx is before the given WrappedTx -// this applies to EVM transactions only -func (wtx *WrappedTx) IsBefore(tx *WrappedTx) bool { - return wtx.evmNonce < tx.evmNonce } func (wtx *WrappedTx) Size() int { @@ -308,144 +291,3 @@ func (wtl *WrappedTxList) Remove(wtx *WrappedTx) { i++ } } - -type PendingTxs struct { - mtx *sync.RWMutex - txs []TxWithResponse - config *config.MempoolConfig - sizeBytes uint64 -} - -type TxWithResponse struct { - tx *WrappedTx - checkTxResponse *abci.ResponseCheckTxV2 - txInfo TxInfo -} - -func NewPendingTxs(conf *config.MempoolConfig) *PendingTxs { - return &PendingTxs{ - mtx: &sync.RWMutex{}, - txs: []TxWithResponse{}, - config: conf, - sizeBytes: 0, - } -} -func (p *PendingTxs) EvaluatePendingTransactions() ( - acceptedTxs []TxWithResponse, - rejectedTxs []TxWithResponse, -) { - poppedIndices := []int{} - p.mtx.Lock() - defer p.mtx.Unlock() - for i := 0; i < len(p.txs); i++ { - switch p.txs[i].checkTxResponse.Checker() { - case abci.Accepted: - acceptedTxs = append(acceptedTxs, p.txs[i]) - poppedIndices = append(poppedIndices, i) - case abci.Rejected: - rejectedTxs = append(rejectedTxs, p.txs[i]) - poppedIndices = append(poppedIndices, i) - } - } - p.popTxsAtIndices(poppedIndices) - return -} - -// assume mtx is already acquired -func (p *PendingTxs) popTxsAtIndices(indices []int) { - if len(indices) == 0 { - return - } - newTxs := []TxWithResponse{} - start := 0 - for _, idx := range indices { - if idx <= start-1 { - panic("indices popped from pending tx store should be sorted without duplicate") - } - if idx >= len(p.txs) { - panic("indices popped from pending tx store out of range") - } - p.sizeBytes -= uint64(p.txs[idx].tx.Size()) - newTxs = append(newTxs, p.txs[start:idx]...) - start = idx + 1 - } - newTxs = append(newTxs, p.txs[start:]...) - p.txs = newTxs -} - -func (p *PendingTxs) Insert(tx *WrappedTx, resCheckTx *abci.ResponseCheckTxV2, txInfo TxInfo) error { - p.mtx.Lock() - defer p.mtx.Unlock() - - if len(p.txs) >= p.config.PendingSize && uint64(tx.Size())+p.sizeBytes > uint64(p.config.MaxPendingTxsBytes) { - return errors.New("pending store is full") - } - - p.txs = append(p.txs, TxWithResponse{ - tx: tx, - checkTxResponse: resCheckTx, - txInfo: txInfo, - }) - p.sizeBytes += uint64(tx.Size()) - return nil -} - -func (p *PendingTxs) Peek(max int) []TxWithResponse { - p.mtx.RLock() - defer p.mtx.RUnlock() - // priority is fifo - if max > len(p.txs) { - return p.txs - } - return p.txs[:max] -} - -func (p *PendingTxs) Size() int { - p.mtx.RLock() - defer p.mtx.RUnlock() - return len(p.txs) -} - -func (p *PendingTxs) PurgeExpired(blockHeight int64, now time.Time, cb func(wtx *WrappedTx)) { - p.mtx.Lock() - defer p.mtx.Unlock() - - if len(p.txs) == 0 { - return - } - - // txs retains the ordering of insertion - if p.config.TTLNumBlocks > 0 { - idxFirstNotExpiredTx := len(p.txs) - for i, ptx := range p.txs { - // once found, we can break because these are ordered - if (blockHeight - ptx.tx.height) <= p.config.TTLNumBlocks { - idxFirstNotExpiredTx = i - break - } else { - cb(ptx.tx) - p.sizeBytes -= uint64(ptx.tx.Size()) - } - } - p.txs = p.txs[idxFirstNotExpiredTx:] - } - - if len(p.txs) == 0 { - return - } - - if p.config.TTLDuration > 0 { - idxFirstNotExpiredTx := len(p.txs) - for i, ptx := range p.txs { - // once found, we can break because these are ordered - if now.Sub(ptx.tx.timestamp) <= p.config.TTLDuration { - idxFirstNotExpiredTx = i - break - } else { - cb(ptx.tx) - p.sizeBytes -= uint64(ptx.tx.Size()) - } - } - p.txs = p.txs[idxFirstNotExpiredTx:] - } -}