mirror of
				https://github.com/0glabs/0g-chain.git
				synced 2025-10-31 23:17:27 +00:00 
			
		
		
		
	impl priority nonce mempool for ethtx
This commit is contained in:
		
							parent
							
								
									aed6a6161a
								
							
						
					
					
						commit
						0a79fe53a2
					
				
							
								
								
									
										338
									
								
								app/abci_utils.go
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										338
									
								
								app/abci_utils.go
									
									
									
									
									
										Normal file
									
								
							| @ -0,0 +1,338 @@ | |||||||
|  | package app | ||||||
|  | 
 | ||||||
|  | import ( | ||||||
|  | 	"fmt" | ||||||
|  | 
 | ||||||
|  | 	"github.com/cockroachdb/errors" | ||||||
|  | 	abci "github.com/cometbft/cometbft/abci/types" | ||||||
|  | 	gethtypes "github.com/ethereum/go-ethereum/core/types" | ||||||
|  | 	evmtypes "github.com/evmos/ethermint/x/evm/types" | ||||||
|  | 
 | ||||||
|  | 	sdk "github.com/cosmos/cosmos-sdk/types" | ||||||
|  | 	"github.com/cosmos/cosmos-sdk/types/mempool" | ||||||
|  | 	"github.com/cosmos/cosmos-sdk/x/auth/signing" | ||||||
|  | ) | ||||||
|  | 
 | ||||||
|  | type ( | ||||||
|  | 	// GasTx defines the contract that a transaction with a gas limit must implement.
 | ||||||
|  | 	GasTx interface { | ||||||
|  | 		GetGas() uint64 | ||||||
|  | 	} | ||||||
|  | 
 | ||||||
|  | 	// ProposalTxVerifier defines the interface that is implemented by BaseApp,
 | ||||||
|  | 	// that any custom ABCI PrepareProposal and ProcessProposal handler can use
 | ||||||
|  | 	// to verify a transaction.
 | ||||||
|  | 	ProposalTxVerifier interface { | ||||||
|  | 		PrepareProposalVerifyTx(tx sdk.Tx) ([]byte, error) | ||||||
|  | 		ProcessProposalVerifyTx(txBz []byte) (sdk.Tx, error) | ||||||
|  | 	} | ||||||
|  | 
 | ||||||
|  | 	// DefaultProposalHandler defines the default ABCI PrepareProposal and
 | ||||||
|  | 	// ProcessProposal handlers.
 | ||||||
|  | 	DefaultProposalHandler struct { | ||||||
|  | 		mempool    mempool.Mempool | ||||||
|  | 		txVerifier ProposalTxVerifier | ||||||
|  | 		txSelector TxSelector | ||||||
|  | 	} | ||||||
|  | ) | ||||||
|  | 
 | ||||||
|  | func NewDefaultProposalHandler(mp mempool.Mempool, txVerifier ProposalTxVerifier) *DefaultProposalHandler { | ||||||
|  | 	return &DefaultProposalHandler{ | ||||||
|  | 		mempool:    mp, | ||||||
|  | 		txVerifier: txVerifier, | ||||||
|  | 		txSelector: NewDefaultTxSelector(), | ||||||
|  | 	} | ||||||
|  | } | ||||||
|  | 
 | ||||||
|  | // SetTxSelector sets the TxSelector function on the DefaultProposalHandler.
 | ||||||
|  | func (h *DefaultProposalHandler) SetTxSelector(ts TxSelector) { | ||||||
|  | 	h.txSelector = ts | ||||||
|  | } | ||||||
|  | 
 | ||||||
|  | // PrepareProposalHandler returns the default implementation for processing an
 | ||||||
|  | // ABCI proposal. The application's mempool is enumerated and all valid
 | ||||||
|  | // transactions are added to the proposal. Transactions are valid if they:
 | ||||||
|  | //
 | ||||||
|  | // 1) Successfully encode to bytes.
 | ||||||
|  | // 2) Are valid (i.e. pass runTx, AnteHandler only).
 | ||||||
|  | //
 | ||||||
|  | // Enumeration is halted once RequestPrepareProposal.MaxBytes of transactions is
 | ||||||
|  | // reached or the mempool is exhausted.
 | ||||||
|  | //
 | ||||||
|  | // Note:
 | ||||||
|  | //
 | ||||||
|  | // - Step (2) is identical to the validation step performed in
 | ||||||
|  | // DefaultProcessProposal. It is very important that the same validation logic
 | ||||||
|  | // is used in both steps, and applications must ensure that this is the case in
 | ||||||
|  | // non-default handlers.
 | ||||||
|  | //
 | ||||||
|  | // - If no mempool is set or if the mempool is a no-op mempool, the transactions
 | ||||||
|  | // requested from CometBFT will simply be returned, which, by default, are in
 | ||||||
|  | // FIFO order.
 | ||||||
|  | func (h *DefaultProposalHandler) PrepareProposalHandler() sdk.PrepareProposalHandler { | ||||||
|  | 	return func(ctx sdk.Context, req abci.RequestPrepareProposal) abci.ResponsePrepareProposal { | ||||||
|  | 		var maxBlockGas uint64 | ||||||
|  | 		if b := ctx.ConsensusParams().Block; b != nil { | ||||||
|  | 			maxBlockGas = uint64(b.MaxGas) | ||||||
|  | 		} | ||||||
|  | 
 | ||||||
|  | 		defer h.txSelector.Clear() | ||||||
|  | 
 | ||||||
|  | 		// If the mempool is nil or NoOp we simply return the transactions
 | ||||||
|  | 		// requested from CometBFT, which, by default, should be in FIFO order.
 | ||||||
|  | 		//
 | ||||||
|  | 		// Note, we still need to ensure the transactions returned respect req.MaxTxBytes.
 | ||||||
|  | 		_, isNoOp := h.mempool.(mempool.NoOpMempool) | ||||||
|  | 		if h.mempool == nil || isNoOp { | ||||||
|  | 			for _, txBz := range req.Txs { | ||||||
|  | 				// XXX: We pass nil as the memTx because we have no way of decoding the
 | ||||||
|  | 				// txBz. We'd need to break (update) the ProposalTxVerifier interface.
 | ||||||
|  | 				// As a result, we CANNOT account for block max gas.
 | ||||||
|  | 				stop := h.txSelector.SelectTxForProposal(uint64(req.MaxTxBytes), maxBlockGas, nil, txBz) | ||||||
|  | 				if stop { | ||||||
|  | 					break | ||||||
|  | 				} | ||||||
|  | 			} | ||||||
|  | 
 | ||||||
|  | 			return abci.ResponsePrepareProposal{Txs: h.txSelector.SelectedTxs()} | ||||||
|  | 		} | ||||||
|  | 
 | ||||||
|  | 		iterator := h.mempool.Select(ctx, req.Txs) | ||||||
|  | 		selectedTxsSignersSeqs := make(map[string]uint64) | ||||||
|  | 		var selectedTxsNums int | ||||||
|  | 		for iterator != nil { | ||||||
|  | 			memTx := iterator.Tx() | ||||||
|  | 			sigs, err := memTx.(signing.SigVerifiableTx).GetSignaturesV2() | ||||||
|  | 			if err != nil { | ||||||
|  | 				panic(fmt.Errorf("failed to get signatures: %w", err)) | ||||||
|  | 			} | ||||||
|  | 			// If the signers aren't in selectedTxsSignersSeqs then we haven't seen them before
 | ||||||
|  | 			// so we add them and continue given that we don't need to check the sequence.
 | ||||||
|  | 			shouldAdd := true | ||||||
|  | 			txSignersSeqs := make(map[string]uint64) | ||||||
|  | 			if len(sigs) == 0 { | ||||||
|  | 				msgs := memTx.GetMsgs() | ||||||
|  | 				if len(msgs) == 1 { | ||||||
|  | 					msgEthTx, ok := msgs[0].(*evmtypes.MsgEthereumTx) | ||||||
|  | 					if ok { | ||||||
|  | 						ethTx := msgEthTx.AsTransaction() | ||||||
|  | 						signer := gethtypes.NewEIP2930Signer(ethTx.ChainId()) | ||||||
|  | 						ethSender, err := signer.Sender(ethTx) | ||||||
|  | 						if err == nil { | ||||||
|  | 							signer := sdk.AccAddress(ethSender.Bytes()).String() | ||||||
|  | 							nonce := ethTx.Nonce() | ||||||
|  | 							seq, ok := selectedTxsSignersSeqs[signer] | ||||||
|  | 							if !ok { | ||||||
|  | 								txSignersSeqs[signer] = nonce | ||||||
|  | 							} else { | ||||||
|  | 								// If we have seen this signer before in this block, we must make
 | ||||||
|  | 								// sure that the current sequence is seq+1; otherwise is invalid
 | ||||||
|  | 								// and we skip it.
 | ||||||
|  | 								if seq+1 != nonce { | ||||||
|  | 									shouldAdd = false | ||||||
|  | 								} else { | ||||||
|  | 									txSignersSeqs[signer] = nonce | ||||||
|  | 								} | ||||||
|  | 							} | ||||||
|  | 						} | ||||||
|  | 					} | ||||||
|  | 				} | ||||||
|  | 			} else { | ||||||
|  | 				for _, sig := range sigs { | ||||||
|  | 					signer := sdk.AccAddress(sig.PubKey.Address()).String() | ||||||
|  | 					seq, ok := selectedTxsSignersSeqs[signer] | ||||||
|  | 					if !ok { | ||||||
|  | 						txSignersSeqs[signer] = sig.Sequence | ||||||
|  | 						continue | ||||||
|  | 					} | ||||||
|  | 
 | ||||||
|  | 					// If we have seen this signer before in this block, we must make
 | ||||||
|  | 					// sure that the current sequence is seq+1; otherwise is invalid
 | ||||||
|  | 					// and we skip it.
 | ||||||
|  | 					if seq+1 != sig.Sequence { | ||||||
|  | 						shouldAdd = false | ||||||
|  | 						break | ||||||
|  | 					} | ||||||
|  | 					txSignersSeqs[signer] = sig.Sequence | ||||||
|  | 				} | ||||||
|  | 			} | ||||||
|  | 
 | ||||||
|  | 			if !shouldAdd { | ||||||
|  | 				iterator = iterator.Next() | ||||||
|  | 				continue | ||||||
|  | 			} | ||||||
|  | 
 | ||||||
|  | 			// NOTE: Since transaction verification was already executed in CheckTx,
 | ||||||
|  | 			// which calls mempool.Insert, in theory everything in the pool should be
 | ||||||
|  | 			// valid. But some mempool implementations may insert invalid txs, so we
 | ||||||
|  | 			// check again.
 | ||||||
|  | 			txBz, err := h.txVerifier.PrepareProposalVerifyTx(memTx) | ||||||
|  | 			if err != nil { | ||||||
|  | 				err := h.mempool.Remove(memTx) | ||||||
|  | 				if err != nil && !errors.Is(err, mempool.ErrTxNotFound) { | ||||||
|  | 					panic(err) | ||||||
|  | 				} | ||||||
|  | 			} else { | ||||||
|  | 				stop := h.txSelector.SelectTxForProposal(uint64(req.MaxTxBytes), maxBlockGas, memTx, txBz) | ||||||
|  | 				if stop { | ||||||
|  | 					break | ||||||
|  | 				} | ||||||
|  | 
 | ||||||
|  | 				txsLen := len(h.txSelector.SelectedTxs()) | ||||||
|  | 				for sender, seq := range txSignersSeqs { | ||||||
|  | 					// If txsLen != selectedTxsNums is true, it means that we've
 | ||||||
|  | 					// added a new tx to the selected txs, so we need to update
 | ||||||
|  | 					// the sequence of the sender.
 | ||||||
|  | 					if txsLen != selectedTxsNums { | ||||||
|  | 						selectedTxsSignersSeqs[sender] = seq | ||||||
|  | 					} else if _, ok := selectedTxsSignersSeqs[sender]; !ok { | ||||||
|  | 						// The transaction hasn't been added but it passed the
 | ||||||
|  | 						// verification, so we know that the sequence is correct.
 | ||||||
|  | 						// So we set this sender's sequence to seq-1, in order
 | ||||||
|  | 						// to avoid unnecessary calls to PrepareProposalVerifyTx.
 | ||||||
|  | 						selectedTxsSignersSeqs[sender] = seq - 1 | ||||||
|  | 					} | ||||||
|  | 				} | ||||||
|  | 				selectedTxsNums = txsLen | ||||||
|  | 			} | ||||||
|  | 
 | ||||||
|  | 			iterator = iterator.Next() | ||||||
|  | 		} | ||||||
|  | 		return abci.ResponsePrepareProposal{Txs: h.txSelector.SelectedTxs()} | ||||||
|  | 	} | ||||||
|  | } | ||||||
|  | 
 | ||||||
|  | // ProcessProposalHandler returns the default implementation for processing an
 | ||||||
|  | // ABCI proposal. Every transaction in the proposal must pass 2 conditions:
 | ||||||
|  | //
 | ||||||
|  | // 1. The transaction bytes must decode to a valid transaction.
 | ||||||
|  | // 2. The transaction must be valid (i.e. pass runTx, AnteHandler only)
 | ||||||
|  | //
 | ||||||
|  | // If any transaction fails to pass either condition, the proposal is rejected.
 | ||||||
|  | // Note that step (2) is identical to the validation step performed in
 | ||||||
|  | // DefaultPrepareProposal. It is very important that the same validation logic
 | ||||||
|  | // is used in both steps, and applications must ensure that this is the case in
 | ||||||
|  | // non-default handlers.
 | ||||||
|  | func (h *DefaultProposalHandler) ProcessProposalHandler() sdk.ProcessProposalHandler { | ||||||
|  | 	// If the mempool is nil or NoOp we simply return ACCEPT,
 | ||||||
|  | 	// because PrepareProposal may have included txs that could fail verification.
 | ||||||
|  | 	_, isNoOp := h.mempool.(mempool.NoOpMempool) | ||||||
|  | 	if h.mempool == nil || isNoOp { | ||||||
|  | 		return NoOpProcessProposal() | ||||||
|  | 	} | ||||||
|  | 
 | ||||||
|  | 	return func(ctx sdk.Context, req abci.RequestProcessProposal) abci.ResponseProcessProposal { | ||||||
|  | 		var totalTxGas uint64 | ||||||
|  | 
 | ||||||
|  | 		var maxBlockGas int64 | ||||||
|  | 		if b := ctx.ConsensusParams().Block; b != nil { | ||||||
|  | 			maxBlockGas = b.MaxGas | ||||||
|  | 		} | ||||||
|  | 
 | ||||||
|  | 		for _, txBytes := range req.Txs { | ||||||
|  | 			tx, err := h.txVerifier.ProcessProposalVerifyTx(txBytes) | ||||||
|  | 			if err != nil { | ||||||
|  | 				return abci.ResponseProcessProposal{Status: abci.ResponseProcessProposal_REJECT} | ||||||
|  | 			} | ||||||
|  | 
 | ||||||
|  | 			if maxBlockGas > 0 { | ||||||
|  | 				gasTx, ok := tx.(GasTx) | ||||||
|  | 				if ok { | ||||||
|  | 					totalTxGas += gasTx.GetGas() | ||||||
|  | 				} | ||||||
|  | 
 | ||||||
|  | 				if totalTxGas > uint64(maxBlockGas) { | ||||||
|  | 					return abci.ResponseProcessProposal{Status: abci.ResponseProcessProposal_REJECT} | ||||||
|  | 				} | ||||||
|  | 			} | ||||||
|  | 		} | ||||||
|  | 
 | ||||||
|  | 		return abci.ResponseProcessProposal{Status: abci.ResponseProcessProposal_ACCEPT} | ||||||
|  | 	} | ||||||
|  | } | ||||||
|  | 
 | ||||||
|  | // NoOpPrepareProposal defines a no-op PrepareProposal handler. It will always
 | ||||||
|  | // return the transactions sent by the client's request.
 | ||||||
|  | func NoOpPrepareProposal() sdk.PrepareProposalHandler { | ||||||
|  | 	return func(_ sdk.Context, req abci.RequestPrepareProposal) abci.ResponsePrepareProposal { | ||||||
|  | 		return abci.ResponsePrepareProposal{Txs: req.Txs} | ||||||
|  | 	} | ||||||
|  | } | ||||||
|  | 
 | ||||||
|  | // NoOpProcessProposal defines a no-op ProcessProposal Handler. It will always
 | ||||||
|  | // return ACCEPT.
 | ||||||
|  | func NoOpProcessProposal() sdk.ProcessProposalHandler { | ||||||
|  | 	return func(_ sdk.Context, _ abci.RequestProcessProposal) abci.ResponseProcessProposal { | ||||||
|  | 		return abci.ResponseProcessProposal{Status: abci.ResponseProcessProposal_ACCEPT} | ||||||
|  | 	} | ||||||
|  | } | ||||||
|  | 
 | ||||||
|  | // TxSelector defines a helper type that assists in selecting transactions during
 | ||||||
|  | // mempool transaction selection in PrepareProposal. It keeps track of the total
 | ||||||
|  | // number of bytes and total gas of the selected transactions. It also keeps
 | ||||||
|  | // track of the selected transactions themselves.
 | ||||||
|  | type TxSelector interface { | ||||||
|  | 	// SelectedTxs should return a copy of the selected transactions.
 | ||||||
|  | 	SelectedTxs() [][]byte | ||||||
|  | 
 | ||||||
|  | 	// Clear should clear the TxSelector, nulling out all relevant fields.
 | ||||||
|  | 	Clear() | ||||||
|  | 
 | ||||||
|  | 	// SelectTxForProposal should attempt to select a transaction for inclusion in
 | ||||||
|  | 	// a proposal based on inclusion criteria defined by the TxSelector. It must
 | ||||||
|  | 	// return <true> if the caller should halt the transaction selection loop
 | ||||||
|  | 	// (typically over a mempool) or <false> otherwise.
 | ||||||
|  | 	SelectTxForProposal(maxTxBytes, maxBlockGas uint64, memTx sdk.Tx, txBz []byte) bool | ||||||
|  | } | ||||||
|  | 
 | ||||||
|  | type defaultTxSelector struct { | ||||||
|  | 	totalTxBytes uint64 | ||||||
|  | 	totalTxGas   uint64 | ||||||
|  | 	selectedTxs  [][]byte | ||||||
|  | } | ||||||
|  | 
 | ||||||
|  | func NewDefaultTxSelector() TxSelector { | ||||||
|  | 	return &defaultTxSelector{} | ||||||
|  | } | ||||||
|  | 
 | ||||||
|  | func (ts *defaultTxSelector) SelectedTxs() [][]byte { | ||||||
|  | 	txs := make([][]byte, len(ts.selectedTxs)) | ||||||
|  | 	copy(txs, ts.selectedTxs) | ||||||
|  | 	return txs | ||||||
|  | } | ||||||
|  | 
 | ||||||
|  | func (ts *defaultTxSelector) Clear() { | ||||||
|  | 	ts.totalTxBytes = 0 | ||||||
|  | 	ts.totalTxGas = 0 | ||||||
|  | 	ts.selectedTxs = nil | ||||||
|  | } | ||||||
|  | 
 | ||||||
|  | func (ts *defaultTxSelector) SelectTxForProposal(maxTxBytes, maxBlockGas uint64, memTx sdk.Tx, txBz []byte) bool { | ||||||
|  | 	txSize := uint64(len(txBz)) | ||||||
|  | 
 | ||||||
|  | 	var txGasLimit uint64 | ||||||
|  | 	if memTx != nil { | ||||||
|  | 		if gasTx, ok := memTx.(GasTx); ok { | ||||||
|  | 			txGasLimit = gasTx.GetGas() | ||||||
|  | 		} | ||||||
|  | 	} | ||||||
|  | 
 | ||||||
|  | 	// only add the transaction to the proposal if we have enough capacity
 | ||||||
|  | 	if (txSize + ts.totalTxBytes) <= maxTxBytes { | ||||||
|  | 		// If there is a max block gas limit, add the tx only if the limit has
 | ||||||
|  | 		// not been met.
 | ||||||
|  | 		if maxBlockGas > 0 { | ||||||
|  | 			if (txGasLimit + ts.totalTxGas) <= maxBlockGas { | ||||||
|  | 				ts.totalTxGas += txGasLimit | ||||||
|  | 				ts.totalTxBytes += txSize | ||||||
|  | 				ts.selectedTxs = append(ts.selectedTxs, txBz) | ||||||
|  | 			} | ||||||
|  | 		} else { | ||||||
|  | 			ts.totalTxBytes += txSize | ||||||
|  | 			ts.selectedTxs = append(ts.selectedTxs, txBz) | ||||||
|  | 		} | ||||||
|  | 	} | ||||||
|  | 
 | ||||||
|  | 	// check if we've reached capacity; if so, we cannot select any more transactions
 | ||||||
|  | 	return ts.totalTxBytes >= maxTxBytes || (maxBlockGas > 0 && (ts.totalTxGas >= maxBlockGas)) | ||||||
|  | } | ||||||
							
								
								
									
										11
									
								
								app/app.go
									
									
									
									
									
								
							
							
						
						
									
										11
									
								
								app/app.go
									
									
									
									
									
								
							| @ -300,21 +300,24 @@ type App struct { | |||||||
| func init() { | func init() { | ||||||
| } | } | ||||||
| 
 | 
 | ||||||
|  | func NewBaseApp(logger tmlog.Logger, db dbm.DB, encodingConfig chainparams.EncodingConfig, | ||||||
|  | 	baseAppOptions ...func(*baseapp.BaseApp)) *baseapp.BaseApp { | ||||||
|  | 	bApp := baseapp.NewBaseApp(chaincfg.AppName, logger, db, encodingConfig.TxConfig.TxDecoder(), baseAppOptions...) | ||||||
|  | 	return bApp | ||||||
|  | } | ||||||
|  | 
 | ||||||
| // NewApp returns a reference to an initialized App.
 | // NewApp returns a reference to an initialized App.
 | ||||||
| func NewApp( | func NewApp( | ||||||
| 	logger tmlog.Logger, |  | ||||||
| 	db dbm.DB, |  | ||||||
| 	homePath string, | 	homePath string, | ||||||
| 	traceStore io.Writer, | 	traceStore io.Writer, | ||||||
| 	encodingConfig chainparams.EncodingConfig, | 	encodingConfig chainparams.EncodingConfig, | ||||||
| 	options Options, | 	options Options, | ||||||
| 	baseAppOptions ...func(*baseapp.BaseApp), | 	bApp *baseapp.BaseApp, | ||||||
| ) *App { | ) *App { | ||||||
| 	appCodec := encodingConfig.Marshaler | 	appCodec := encodingConfig.Marshaler | ||||||
| 	legacyAmino := encodingConfig.Amino | 	legacyAmino := encodingConfig.Amino | ||||||
| 	interfaceRegistry := encodingConfig.InterfaceRegistry | 	interfaceRegistry := encodingConfig.InterfaceRegistry | ||||||
| 
 | 
 | ||||||
| 	bApp := baseapp.NewBaseApp(chaincfg.AppName, logger, db, encodingConfig.TxConfig.TxDecoder(), baseAppOptions...) |  | ||||||
| 	bApp.SetCommitMultiStoreTracer(traceStore) | 	bApp.SetCommitMultiStoreTracer(traceStore) | ||||||
| 	bApp.SetVersion(version.Version) | 	bApp.SetVersion(version.Version) | ||||||
| 	bApp.SetInterfaceRegistry(interfaceRegistry) | 	bApp.SetInterfaceRegistry(interfaceRegistry) | ||||||
|  | |||||||
							
								
								
									
										488
									
								
								app/priority_nonce.go
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										488
									
								
								app/priority_nonce.go
									
									
									
									
									
										Normal file
									
								
							| @ -0,0 +1,488 @@ | |||||||
|  | package app | ||||||
|  | 
 | ||||||
|  | import ( | ||||||
|  | 	"context" | ||||||
|  | 	"fmt" | ||||||
|  | 	"math" | ||||||
|  | 
 | ||||||
|  | 	"github.com/huandu/skiplist" | ||||||
|  | 
 | ||||||
|  | 	sdk "github.com/cosmos/cosmos-sdk/types" | ||||||
|  | 	"github.com/cosmos/cosmos-sdk/types/mempool" | ||||||
|  | 	"github.com/cosmos/cosmos-sdk/x/auth/signing" | ||||||
|  | 	gethtypes "github.com/ethereum/go-ethereum/core/types" | ||||||
|  | 	evmtypes "github.com/evmos/ethermint/x/evm/types" | ||||||
|  | ) | ||||||
|  | 
 | ||||||
|  | var ( | ||||||
|  | 	_ mempool.Mempool  = (*PriorityNonceMempool)(nil) | ||||||
|  | 	_ mempool.Iterator = (*PriorityNonceIterator)(nil) | ||||||
|  | ) | ||||||
|  | 
 | ||||||
|  | // PriorityNonceMempool is a mempool implementation that stores txs
 | ||||||
|  | // in a partially ordered set by 2 dimensions: priority, and sender-nonce
 | ||||||
|  | // (sequence number). Internally it uses one priority ordered skip list and one
 | ||||||
|  | // skip list per sender ordered by sender-nonce (sequence number). When there
 | ||||||
|  | // are multiple txs from the same sender, they are not always comparable by
 | ||||||
|  | // priority to other sender txs and must be partially ordered by both sender-nonce
 | ||||||
|  | // and priority.
 | ||||||
|  | type PriorityNonceMempool struct { | ||||||
|  | 	priorityIndex  *skiplist.SkipList | ||||||
|  | 	priorityCounts map[int64]int | ||||||
|  | 	senderIndices  map[string]*skiplist.SkipList | ||||||
|  | 	scores         map[txMeta]txMeta | ||||||
|  | 	onRead         func(tx sdk.Tx) | ||||||
|  | 	txReplacement  func(op, np int64, oTx, nTx sdk.Tx) bool | ||||||
|  | 	maxTx          int | ||||||
|  | } | ||||||
|  | 
 | ||||||
|  | type PriorityNonceIterator struct { | ||||||
|  | 	senderCursors map[string]*skiplist.Element | ||||||
|  | 	nextPriority  int64 | ||||||
|  | 	sender        string | ||||||
|  | 	priorityNode  *skiplist.Element | ||||||
|  | 	mempool       *PriorityNonceMempool | ||||||
|  | } | ||||||
|  | 
 | ||||||
|  | // txMeta stores transaction metadata used in indices
 | ||||||
|  | type txMeta struct { | ||||||
|  | 	// nonce is the sender's sequence number
 | ||||||
|  | 	nonce uint64 | ||||||
|  | 	// priority is the transaction's priority
 | ||||||
|  | 	priority int64 | ||||||
|  | 	// sender is the transaction's sender
 | ||||||
|  | 	sender string | ||||||
|  | 	// weight is the transaction's weight, used as a tiebreaker for transactions with the same priority
 | ||||||
|  | 	weight int64 | ||||||
|  | 	// senderElement is a pointer to the transaction's element in the sender index
 | ||||||
|  | 	senderElement *skiplist.Element | ||||||
|  | } | ||||||
|  | 
 | ||||||
|  | // txMetaLess is a comparator for txKeys that first compares priority, then weight,
 | ||||||
|  | // then sender, then nonce, uniquely identifying a transaction.
 | ||||||
|  | //
 | ||||||
|  | // Note, txMetaLess is used as the comparator in the priority index.
 | ||||||
|  | func txMetaLess(a, b any) int { | ||||||
|  | 	keyA := a.(txMeta) | ||||||
|  | 	keyB := b.(txMeta) | ||||||
|  | 	res := skiplist.Int64.Compare(keyA.priority, keyB.priority) | ||||||
|  | 	if res != 0 { | ||||||
|  | 		return res | ||||||
|  | 	} | ||||||
|  | 
 | ||||||
|  | 	// Weight is used as a tiebreaker for transactions with the same priority.
 | ||||||
|  | 	// Weight is calculated in a single pass in .Select(...) and so will be 0
 | ||||||
|  | 	// on .Insert(...).
 | ||||||
|  | 	res = skiplist.Int64.Compare(keyA.weight, keyB.weight) | ||||||
|  | 	if res != 0 { | ||||||
|  | 		return res | ||||||
|  | 	} | ||||||
|  | 
 | ||||||
|  | 	// Because weight will be 0 on .Insert(...), we must also compare sender and
 | ||||||
|  | 	// nonce to resolve priority collisions. If we didn't then transactions with
 | ||||||
|  | 	// the same priority would overwrite each other in the priority index.
 | ||||||
|  | 	res = skiplist.String.Compare(keyA.sender, keyB.sender) | ||||||
|  | 	if res != 0 { | ||||||
|  | 		return res | ||||||
|  | 	} | ||||||
|  | 
 | ||||||
|  | 	return skiplist.Uint64.Compare(keyA.nonce, keyB.nonce) | ||||||
|  | } | ||||||
|  | 
 | ||||||
|  | type PriorityNonceMempoolOption func(*PriorityNonceMempool) | ||||||
|  | 
 | ||||||
|  | // PriorityNonceWithOnRead sets a callback to be called when a tx is read from
 | ||||||
|  | // the mempool.
 | ||||||
|  | func PriorityNonceWithOnRead(onRead func(tx sdk.Tx)) PriorityNonceMempoolOption { | ||||||
|  | 	return func(mp *PriorityNonceMempool) { | ||||||
|  | 		mp.onRead = onRead | ||||||
|  | 	} | ||||||
|  | } | ||||||
|  | 
 | ||||||
|  | // PriorityNonceWithTxReplacement sets a callback to be called when duplicated
 | ||||||
|  | // transaction nonce detected during mempool insert. An application can define a
 | ||||||
|  | // transaction replacement rule based on tx priority or certain transaction fields.
 | ||||||
|  | func PriorityNonceWithTxReplacement(txReplacementRule func(op, np int64, oTx, nTx sdk.Tx) bool) PriorityNonceMempoolOption { | ||||||
|  | 	return func(mp *PriorityNonceMempool) { | ||||||
|  | 		mp.txReplacement = txReplacementRule | ||||||
|  | 	} | ||||||
|  | } | ||||||
|  | 
 | ||||||
|  | // PriorityNonceWithMaxTx sets the maximum number of transactions allowed in the
 | ||||||
|  | // mempool with the semantics:
 | ||||||
|  | //
 | ||||||
|  | // <0: disabled, `Insert` is a no-op
 | ||||||
|  | // 0: unlimited
 | ||||||
|  | // >0: maximum number of transactions allowed
 | ||||||
|  | func PriorityNonceWithMaxTx(maxTx int) PriorityNonceMempoolOption { | ||||||
|  | 	return func(mp *PriorityNonceMempool) { | ||||||
|  | 		mp.maxTx = maxTx | ||||||
|  | 	} | ||||||
|  | } | ||||||
|  | 
 | ||||||
|  | // DefaultPriorityMempool returns a priorityNonceMempool with no options.
 | ||||||
|  | func DefaultPriorityMempool() mempool.Mempool { | ||||||
|  | 	return NewPriorityMempool() | ||||||
|  | } | ||||||
|  | 
 | ||||||
|  | // NewPriorityMempool returns the SDK's default mempool implementation which
 | ||||||
|  | // returns txs in a partial order by 2 dimensions; priority, and sender-nonce.
 | ||||||
|  | func NewPriorityMempool(opts ...PriorityNonceMempoolOption) *PriorityNonceMempool { | ||||||
|  | 	mp := &PriorityNonceMempool{ | ||||||
|  | 		priorityIndex:  skiplist.New(skiplist.LessThanFunc(txMetaLess)), | ||||||
|  | 		priorityCounts: make(map[int64]int), | ||||||
|  | 		senderIndices:  make(map[string]*skiplist.SkipList), | ||||||
|  | 		scores:         make(map[txMeta]txMeta), | ||||||
|  | 	} | ||||||
|  | 
 | ||||||
|  | 	for _, opt := range opts { | ||||||
|  | 		opt(mp) | ||||||
|  | 	} | ||||||
|  | 
 | ||||||
|  | 	return mp | ||||||
|  | } | ||||||
|  | 
 | ||||||
|  | // NextSenderTx returns the next transaction for a given sender by nonce order,
 | ||||||
|  | // i.e. the next valid transaction for the sender. If no such transaction exists,
 | ||||||
|  | // nil will be returned.
 | ||||||
|  | func (mp *PriorityNonceMempool) NextSenderTx(sender string) sdk.Tx { | ||||||
|  | 	senderIndex, ok := mp.senderIndices[sender] | ||||||
|  | 	if !ok { | ||||||
|  | 		return nil | ||||||
|  | 	} | ||||||
|  | 
 | ||||||
|  | 	cursor := senderIndex.Front() | ||||||
|  | 	return cursor.Value.(sdk.Tx) | ||||||
|  | } | ||||||
|  | 
 | ||||||
|  | // Insert attempts to insert a Tx into the app-side mempool in O(log n) time,
 | ||||||
|  | // returning an error if unsuccessful. Sender and nonce are derived from the
 | ||||||
|  | // transaction's first signature.
 | ||||||
|  | //
 | ||||||
|  | // Transactions are unique by sender and nonce. Inserting a duplicate tx is an
 | ||||||
|  | // O(log n) no-op.
 | ||||||
|  | //
 | ||||||
|  | // Inserting a duplicate tx with a different priority overwrites the existing tx,
 | ||||||
|  | // changing the total order of the mempool.
 | ||||||
|  | func (mp *PriorityNonceMempool) Insert(ctx context.Context, tx sdk.Tx) error { | ||||||
|  | 	if mp.maxTx > 0 && mp.CountTx() >= mp.maxTx { | ||||||
|  | 		return mempool.ErrMempoolTxMaxCapacity | ||||||
|  | 	} else if mp.maxTx < 0 { | ||||||
|  | 		return nil | ||||||
|  | 	} | ||||||
|  | 
 | ||||||
|  | 	sigs, err := tx.(signing.SigVerifiableTx).GetSignaturesV2() | ||||||
|  | 	if err != nil { | ||||||
|  | 		return err | ||||||
|  | 	} | ||||||
|  | 	sdkContext := sdk.UnwrapSDKContext(ctx) | ||||||
|  | 	priority := sdkContext.Priority() | ||||||
|  | 
 | ||||||
|  | 	var sender string | ||||||
|  | 	var nonce uint64 | ||||||
|  | 
 | ||||||
|  | 	if len(sigs) == 0 { | ||||||
|  | 		msgs := tx.GetMsgs() | ||||||
|  | 		if len(msgs) != 1 { | ||||||
|  | 			return fmt.Errorf("tx must have at least one signer") | ||||||
|  | 		} | ||||||
|  | 		msgEthTx, ok := msgs[0].(*evmtypes.MsgEthereumTx) | ||||||
|  | 		if !ok { | ||||||
|  | 			return fmt.Errorf("tx must have at least one signer") | ||||||
|  | 		} | ||||||
|  | 		ethTx := msgEthTx.AsTransaction() | ||||||
|  | 		signer := gethtypes.NewEIP2930Signer(ethTx.ChainId()) | ||||||
|  | 		ethSender, err := signer.Sender(ethTx) | ||||||
|  | 		if err != nil { | ||||||
|  | 			return fmt.Errorf("tx must have at least one signer") | ||||||
|  | 		} | ||||||
|  | 		sender = sdk.AccAddress(ethSender.Bytes()).String() | ||||||
|  | 		nonce = ethTx.Nonce() | ||||||
|  | 	} else { | ||||||
|  | 		sig := sigs[0] | ||||||
|  | 		sender = sdk.AccAddress(sig.PubKey.Address()).String() | ||||||
|  | 		nonce = sig.Sequence | ||||||
|  | 	} | ||||||
|  | 
 | ||||||
|  | 	key := txMeta{nonce: nonce, priority: priority, sender: sender} | ||||||
|  | 
 | ||||||
|  | 	senderIndex, ok := mp.senderIndices[sender] | ||||||
|  | 	if !ok { | ||||||
|  | 		senderIndex = skiplist.New(skiplist.LessThanFunc(func(a, b any) int { | ||||||
|  | 			return skiplist.Uint64.Compare(b.(txMeta).nonce, a.(txMeta).nonce) | ||||||
|  | 		})) | ||||||
|  | 
 | ||||||
|  | 		// initialize sender index if not found
 | ||||||
|  | 		mp.senderIndices[sender] = senderIndex | ||||||
|  | 	} | ||||||
|  | 
 | ||||||
|  | 	// Since mp.priorityIndex is scored by priority, then sender, then nonce, a
 | ||||||
|  | 	// changed priority will create a new key, so we must remove the old key and
 | ||||||
|  | 	// re-insert it to avoid having the same tx with different priorityIndex indexed
 | ||||||
|  | 	// twice in the mempool.
 | ||||||
|  | 	//
 | ||||||
|  | 	// This O(log n) remove operation is rare and only happens when a tx's priority
 | ||||||
|  | 	// changes.
 | ||||||
|  | 	sk := txMeta{nonce: nonce, sender: sender} | ||||||
|  | 	if oldScore, txExists := mp.scores[sk]; txExists { | ||||||
|  | 		if mp.txReplacement != nil && !mp.txReplacement(oldScore.priority, priority, senderIndex.Get(key).Value.(sdk.Tx), tx) { | ||||||
|  | 			return fmt.Errorf( | ||||||
|  | 				"tx doesn't fit the replacement rule, oldPriority: %v, newPriority: %v, oldTx: %v, newTx: %v", | ||||||
|  | 				oldScore.priority, | ||||||
|  | 				priority, | ||||||
|  | 				senderIndex.Get(key).Value.(sdk.Tx), | ||||||
|  | 				tx, | ||||||
|  | 			) | ||||||
|  | 		} | ||||||
|  | 
 | ||||||
|  | 		mp.priorityIndex.Remove(txMeta{ | ||||||
|  | 			nonce:    nonce, | ||||||
|  | 			sender:   sender, | ||||||
|  | 			priority: oldScore.priority, | ||||||
|  | 			weight:   oldScore.weight, | ||||||
|  | 		}) | ||||||
|  | 		mp.priorityCounts[oldScore.priority]-- | ||||||
|  | 	} | ||||||
|  | 
 | ||||||
|  | 	mp.priorityCounts[priority]++ | ||||||
|  | 
 | ||||||
|  | 	// Since senderIndex is scored by nonce, a changed priority will overwrite the
 | ||||||
|  | 	// existing key.
 | ||||||
|  | 	key.senderElement = senderIndex.Set(key, tx) | ||||||
|  | 
 | ||||||
|  | 	mp.scores[sk] = txMeta{priority: priority} | ||||||
|  | 	mp.priorityIndex.Set(key, tx) | ||||||
|  | 
 | ||||||
|  | 	return nil | ||||||
|  | } | ||||||
|  | 
 | ||||||
|  | func (i *PriorityNonceIterator) iteratePriority() mempool.Iterator { | ||||||
|  | 	// beginning of priority iteration
 | ||||||
|  | 	if i.priorityNode == nil { | ||||||
|  | 		i.priorityNode = i.mempool.priorityIndex.Front() | ||||||
|  | 	} else { | ||||||
|  | 		i.priorityNode = i.priorityNode.Next() | ||||||
|  | 	} | ||||||
|  | 
 | ||||||
|  | 	// end of priority iteration
 | ||||||
|  | 	if i.priorityNode == nil { | ||||||
|  | 		return nil | ||||||
|  | 	} | ||||||
|  | 
 | ||||||
|  | 	i.sender = i.priorityNode.Key().(txMeta).sender | ||||||
|  | 
 | ||||||
|  | 	nextPriorityNode := i.priorityNode.Next() | ||||||
|  | 	if nextPriorityNode != nil { | ||||||
|  | 		i.nextPriority = nextPriorityNode.Key().(txMeta).priority | ||||||
|  | 	} else { | ||||||
|  | 		i.nextPriority = math.MinInt64 | ||||||
|  | 	} | ||||||
|  | 
 | ||||||
|  | 	return i.Next() | ||||||
|  | } | ||||||
|  | 
 | ||||||
|  | func (i *PriorityNonceIterator) Next() mempool.Iterator { | ||||||
|  | 	if i.priorityNode == nil { | ||||||
|  | 		return nil | ||||||
|  | 	} | ||||||
|  | 
 | ||||||
|  | 	cursor, ok := i.senderCursors[i.sender] | ||||||
|  | 	if !ok { | ||||||
|  | 		// beginning of sender iteration
 | ||||||
|  | 		cursor = i.mempool.senderIndices[i.sender].Front() | ||||||
|  | 	} else { | ||||||
|  | 		// middle of sender iteration
 | ||||||
|  | 		cursor = cursor.Next() | ||||||
|  | 	} | ||||||
|  | 
 | ||||||
|  | 	// end of sender iteration
 | ||||||
|  | 	if cursor == nil { | ||||||
|  | 		return i.iteratePriority() | ||||||
|  | 	} | ||||||
|  | 
 | ||||||
|  | 	key := cursor.Key().(txMeta) | ||||||
|  | 
 | ||||||
|  | 	// We've reached a transaction with a priority lower than the next highest
 | ||||||
|  | 	// priority in the pool.
 | ||||||
|  | 	if key.priority < i.nextPriority { | ||||||
|  | 		return i.iteratePriority() | ||||||
|  | 	} else if key.priority == i.nextPriority && i.priorityNode.Next() != nil { | ||||||
|  | 		// Weight is incorporated into the priority index key only (not sender index)
 | ||||||
|  | 		// so we must fetch it here from the scores map.
 | ||||||
|  | 		weight := i.mempool.scores[txMeta{nonce: key.nonce, sender: key.sender}].weight | ||||||
|  | 		if weight < i.priorityNode.Next().Key().(txMeta).weight { | ||||||
|  | 			return i.iteratePriority() | ||||||
|  | 		} | ||||||
|  | 	} | ||||||
|  | 
 | ||||||
|  | 	i.senderCursors[i.sender] = cursor | ||||||
|  | 	return i | ||||||
|  | } | ||||||
|  | 
 | ||||||
|  | func (i *PriorityNonceIterator) Tx() sdk.Tx { | ||||||
|  | 	return i.senderCursors[i.sender].Value.(sdk.Tx) | ||||||
|  | } | ||||||
|  | 
 | ||||||
|  | // Select returns a set of transactions from the mempool, ordered by priority
 | ||||||
|  | // and sender-nonce in O(n) time. The passed in list of transactions are ignored.
 | ||||||
|  | // This is a readonly operation, the mempool is not modified.
 | ||||||
|  | //
 | ||||||
|  | // The maxBytes parameter defines the maximum number of bytes of transactions to
 | ||||||
|  | // return.
 | ||||||
|  | //
 | ||||||
|  | // NOTE: It is not safe to use this iterator while removing transactions from
 | ||||||
|  | // the underlying mempool.
 | ||||||
|  | func (mp *PriorityNonceMempool) Select(_ context.Context, _ [][]byte) mempool.Iterator { | ||||||
|  | 	if mp.priorityIndex.Len() == 0 { | ||||||
|  | 		return nil | ||||||
|  | 	} | ||||||
|  | 
 | ||||||
|  | 	mp.reorderPriorityTies() | ||||||
|  | 
 | ||||||
|  | 	iterator := &PriorityNonceIterator{ | ||||||
|  | 		mempool:       mp, | ||||||
|  | 		senderCursors: make(map[string]*skiplist.Element), | ||||||
|  | 	} | ||||||
|  | 
 | ||||||
|  | 	return iterator.iteratePriority() | ||||||
|  | } | ||||||
|  | 
 | ||||||
|  | type reorderKey struct { | ||||||
|  | 	deleteKey txMeta | ||||||
|  | 	insertKey txMeta | ||||||
|  | 	tx        sdk.Tx | ||||||
|  | } | ||||||
|  | 
 | ||||||
|  | func (mp *PriorityNonceMempool) reorderPriorityTies() { | ||||||
|  | 	node := mp.priorityIndex.Front() | ||||||
|  | 
 | ||||||
|  | 	var reordering []reorderKey | ||||||
|  | 	for node != nil { | ||||||
|  | 		key := node.Key().(txMeta) | ||||||
|  | 		if mp.priorityCounts[key.priority] > 1 { | ||||||
|  | 			newKey := key | ||||||
|  | 			newKey.weight = senderWeight(key.senderElement) | ||||||
|  | 			reordering = append(reordering, reorderKey{deleteKey: key, insertKey: newKey, tx: node.Value.(sdk.Tx)}) | ||||||
|  | 		} | ||||||
|  | 
 | ||||||
|  | 		node = node.Next() | ||||||
|  | 	} | ||||||
|  | 
 | ||||||
|  | 	for _, k := range reordering { | ||||||
|  | 		mp.priorityIndex.Remove(k.deleteKey) | ||||||
|  | 		delete(mp.scores, txMeta{nonce: k.deleteKey.nonce, sender: k.deleteKey.sender}) | ||||||
|  | 		mp.priorityIndex.Set(k.insertKey, k.tx) | ||||||
|  | 		mp.scores[txMeta{nonce: k.insertKey.nonce, sender: k.insertKey.sender}] = k.insertKey | ||||||
|  | 	} | ||||||
|  | } | ||||||
|  | 
 | ||||||
|  | // senderWeight returns the weight of a given tx (t) at senderCursor. Weight is
 | ||||||
|  | // defined as the first (nonce-wise) same sender tx with a priority not equal to
 | ||||||
|  | // t. It is used to resolve priority collisions, that is when 2 or more txs from
 | ||||||
|  | // different senders have the same priority.
 | ||||||
|  | func senderWeight(senderCursor *skiplist.Element) int64 { | ||||||
|  | 	if senderCursor == nil { | ||||||
|  | 		return 0 | ||||||
|  | 	} | ||||||
|  | 
 | ||||||
|  | 	weight := senderCursor.Key().(txMeta).priority | ||||||
|  | 	senderCursor = senderCursor.Next() | ||||||
|  | 	for senderCursor != nil { | ||||||
|  | 		p := senderCursor.Key().(txMeta).priority | ||||||
|  | 		if p != weight { | ||||||
|  | 			weight = p | ||||||
|  | 		} | ||||||
|  | 
 | ||||||
|  | 		senderCursor = senderCursor.Next() | ||||||
|  | 	} | ||||||
|  | 
 | ||||||
|  | 	return weight | ||||||
|  | } | ||||||
|  | 
 | ||||||
|  | // CountTx returns the number of transactions in the mempool.
 | ||||||
|  | func (mp *PriorityNonceMempool) CountTx() int { | ||||||
|  | 	return mp.priorityIndex.Len() | ||||||
|  | } | ||||||
|  | 
 | ||||||
|  | // Remove removes a transaction from the mempool in O(log n) time, returning an
 | ||||||
|  | // error if unsuccessful.
 | ||||||
|  | func (mp *PriorityNonceMempool) Remove(tx sdk.Tx) error { | ||||||
|  | 	sigs, err := tx.(signing.SigVerifiableTx).GetSignaturesV2() | ||||||
|  | 	if err != nil { | ||||||
|  | 		return err | ||||||
|  | 	} | ||||||
|  | 	var sender string | ||||||
|  | 	var nonce uint64 | ||||||
|  | 	if len(sigs) == 0 { | ||||||
|  | 		msgs := tx.GetMsgs() | ||||||
|  | 		if len(msgs) != 1 { | ||||||
|  | 			return fmt.Errorf("attempted to remove a tx with no signatures") | ||||||
|  | 		} | ||||||
|  | 		msgEthTx, ok := msgs[0].(*evmtypes.MsgEthereumTx) | ||||||
|  | 		if !ok { | ||||||
|  | 			return fmt.Errorf("attempted to remove a tx with no signatures") | ||||||
|  | 		} | ||||||
|  | 		ethTx := msgEthTx.AsTransaction() | ||||||
|  | 		signer := gethtypes.NewEIP2930Signer(ethTx.ChainId()) | ||||||
|  | 		ethSender, err := signer.Sender(ethTx) | ||||||
|  | 		if err != nil { | ||||||
|  | 			return fmt.Errorf("attempted to remove a tx with no signatures") | ||||||
|  | 		} | ||||||
|  | 		sender = sdk.AccAddress(ethSender.Bytes()).String() | ||||||
|  | 		nonce = ethTx.Nonce() | ||||||
|  | 	} else { | ||||||
|  | 		sig := sigs[0] | ||||||
|  | 		sender = sdk.AccAddress(sig.PubKey.Address()).String() | ||||||
|  | 		nonce = sig.Sequence | ||||||
|  | 	} | ||||||
|  | 
 | ||||||
|  | 	scoreKey := txMeta{nonce: nonce, sender: sender} | ||||||
|  | 	score, ok := mp.scores[scoreKey] | ||||||
|  | 	if !ok { | ||||||
|  | 		return mempool.ErrTxNotFound | ||||||
|  | 	} | ||||||
|  | 	tk := txMeta{nonce: nonce, priority: score.priority, sender: sender, weight: score.weight} | ||||||
|  | 
 | ||||||
|  | 	senderTxs, ok := mp.senderIndices[sender] | ||||||
|  | 	if !ok { | ||||||
|  | 		return fmt.Errorf("sender %s not found", sender) | ||||||
|  | 	} | ||||||
|  | 
 | ||||||
|  | 	mp.priorityIndex.Remove(tk) | ||||||
|  | 	senderTxs.Remove(tk) | ||||||
|  | 	delete(mp.scores, scoreKey) | ||||||
|  | 	mp.priorityCounts[score.priority]-- | ||||||
|  | 
 | ||||||
|  | 	return nil | ||||||
|  | } | ||||||
|  | 
 | ||||||
|  | func IsEmpty(mempool mempool.Mempool) error { | ||||||
|  | 	mp := mempool.(*PriorityNonceMempool) | ||||||
|  | 	if mp.priorityIndex.Len() != 0 { | ||||||
|  | 		return fmt.Errorf("priorityIndex not empty") | ||||||
|  | 	} | ||||||
|  | 
 | ||||||
|  | 	var countKeys []int64 | ||||||
|  | 	for k := range mp.priorityCounts { | ||||||
|  | 		countKeys = append(countKeys, k) | ||||||
|  | 	} | ||||||
|  | 
 | ||||||
|  | 	for _, k := range countKeys { | ||||||
|  | 		if mp.priorityCounts[k] != 0 { | ||||||
|  | 			return fmt.Errorf("priorityCounts not zero at %v, got %v", k, mp.priorityCounts[k]) | ||||||
|  | 		} | ||||||
|  | 	} | ||||||
|  | 
 | ||||||
|  | 	var senderKeys []string | ||||||
|  | 	for k := range mp.senderIndices { | ||||||
|  | 		senderKeys = append(senderKeys, k) | ||||||
|  | 	} | ||||||
|  | 
 | ||||||
|  | 	for _, k := range senderKeys { | ||||||
|  | 		if mp.senderIndices[k].Len() != 0 { | ||||||
|  | 			return fmt.Errorf("senderIndex not empty for sender %v", k) | ||||||
|  | 		} | ||||||
|  | 	} | ||||||
|  | 
 | ||||||
|  | 	return nil | ||||||
|  | } | ||||||
| @ -93,9 +93,10 @@ func NewTestAppFromSealed() TestApp { | |||||||
| 
 | 
 | ||||||
| 	encCfg := MakeEncodingConfig() | 	encCfg := MakeEncodingConfig() | ||||||
| 
 | 
 | ||||||
|  | 	bApp := NewBaseApp(log.NewNopLogger(), db, encCfg, baseapp.SetChainID(TestChainId)) | ||||||
| 	app := NewApp( | 	app := NewApp( | ||||||
| 		log.NewNopLogger(), db, chaincfg.DefaultNodeHome, nil, | 		chaincfg.DefaultNodeHome, nil, | ||||||
| 		encCfg, DefaultOptions, baseapp.SetChainID(TestChainId), | 		encCfg, DefaultOptions, bApp, | ||||||
| 	) | 	) | ||||||
| 	return TestApp{App: *app} | 	return TestApp{App: *app} | ||||||
| } | } | ||||||
|  | |||||||
| @ -107,18 +107,9 @@ func (ac appCreator) newApp( | |||||||
| 		skipLoadLatest = cast.ToBool(appOpts.Get(flagSkipLoadLatest)) | 		skipLoadLatest = cast.ToBool(appOpts.Get(flagSkipLoadLatest)) | ||||||
| 	} | 	} | ||||||
| 
 | 
 | ||||||
| 	return app.NewApp( | 	mempool := app.NewPriorityMempool() | ||||||
| 		logger, db, homeDir, traceStore, ac.encodingConfig, | 
 | ||||||
| 		app.Options{ | 	bApp := app.NewBaseApp(logger, db, ac.encodingConfig, | ||||||
| 			SkipLoadLatest:        skipLoadLatest, |  | ||||||
| 			SkipUpgradeHeights:    skipUpgradeHeights, |  | ||||||
| 			SkipGenesisInvariants: cast.ToBool(appOpts.Get(crisis.FlagSkipGenesisInvariants)), |  | ||||||
| 			InvariantCheckPeriod:  cast.ToUint(appOpts.Get(server.FlagInvCheckPeriod)), |  | ||||||
| 			MempoolEnableAuth:     mempoolEnableAuth, |  | ||||||
| 			MempoolAuthAddresses:  mempoolAuthAddresses, |  | ||||||
| 			EVMTrace:              cast.ToString(appOpts.Get(ethermintflags.EVMTracer)), |  | ||||||
| 			EVMMaxGasWanted:       cast.ToUint64(appOpts.Get(ethermintflags.EVMMaxTxGasWanted)), |  | ||||||
| 		}, |  | ||||||
| 		baseapp.SetPruning(pruningOpts), | 		baseapp.SetPruning(pruningOpts), | ||||||
| 		baseapp.SetMinGasPrices(strings.Replace(cast.ToString(appOpts.Get(server.FlagMinGasPrices)), ";", ",", -1)), | 		baseapp.SetMinGasPrices(strings.Replace(cast.ToString(appOpts.Get(server.FlagMinGasPrices)), ";", ",", -1)), | ||||||
| 		baseapp.SetHaltHeight(cast.ToUint64(appOpts.Get(server.FlagHaltHeight))), | 		baseapp.SetHaltHeight(cast.ToUint64(appOpts.Get(server.FlagHaltHeight))), | ||||||
| @ -132,7 +123,28 @@ func (ac appCreator) newApp( | |||||||
| 		baseapp.SetIAVLDisableFastNode(cast.ToBool(iavlDisableFastNode)), | 		baseapp.SetIAVLDisableFastNode(cast.ToBool(iavlDisableFastNode)), | ||||||
| 		baseapp.SetIAVLLazyLoading(cast.ToBool(appOpts.Get(server.FlagIAVLLazyLoading))), | 		baseapp.SetIAVLLazyLoading(cast.ToBool(appOpts.Get(server.FlagIAVLLazyLoading))), | ||||||
| 		baseapp.SetChainID(chainID), | 		baseapp.SetChainID(chainID), | ||||||
|  | 		baseapp.SetMempool(mempool), | ||||||
| 	) | 	) | ||||||
|  | 	bApp.SetTxEncoder(ac.encodingConfig.TxConfig.TxEncoder()) | ||||||
|  | 	abciProposalHandler := app.NewDefaultProposalHandler(mempool, bApp) | ||||||
|  | 	bApp.SetPrepareProposal(abciProposalHandler.PrepareProposalHandler()) | ||||||
|  | 
 | ||||||
|  | 	newApp := app.NewApp( | ||||||
|  | 		homeDir, traceStore, ac.encodingConfig, | ||||||
|  | 		app.Options{ | ||||||
|  | 			SkipLoadLatest:        skipLoadLatest, | ||||||
|  | 			SkipUpgradeHeights:    skipUpgradeHeights, | ||||||
|  | 			SkipGenesisInvariants: cast.ToBool(appOpts.Get(crisis.FlagSkipGenesisInvariants)), | ||||||
|  | 			InvariantCheckPeriod:  cast.ToUint(appOpts.Get(server.FlagInvCheckPeriod)), | ||||||
|  | 			MempoolEnableAuth:     mempoolEnableAuth, | ||||||
|  | 			MempoolAuthAddresses:  mempoolAuthAddresses, | ||||||
|  | 			EVMTrace:              cast.ToString(appOpts.Get(ethermintflags.EVMTracer)), | ||||||
|  | 			EVMMaxGasWanted:       cast.ToUint64(appOpts.Get(ethermintflags.EVMMaxTxGasWanted)), | ||||||
|  | 		}, | ||||||
|  | 		bApp, | ||||||
|  | 	) | ||||||
|  | 
 | ||||||
|  | 	return newApp | ||||||
| } | } | ||||||
| 
 | 
 | ||||||
| // appExport writes out an app's state to json.
 | // appExport writes out an app's state to json.
 | ||||||
| @ -157,13 +169,15 @@ func (ac appCreator) appExport( | |||||||
| 
 | 
 | ||||||
| 	var tempApp *app.App | 	var tempApp *app.App | ||||||
| 	if height != -1 { | 	if height != -1 { | ||||||
| 		tempApp = app.NewApp(logger, db, homePath, traceStore, ac.encodingConfig, options) | 		bApp := app.NewBaseApp(logger, db, ac.encodingConfig) | ||||||
|  | 		tempApp = app.NewApp(homePath, traceStore, ac.encodingConfig, options, bApp) | ||||||
| 
 | 
 | ||||||
| 		if err := tempApp.LoadHeight(height); err != nil { | 		if err := tempApp.LoadHeight(height); err != nil { | ||||||
| 			return servertypes.ExportedApp{}, err | 			return servertypes.ExportedApp{}, err | ||||||
| 		} | 		} | ||||||
| 	} else { | 	} else { | ||||||
| 		tempApp = app.NewApp(logger, db, homePath, traceStore, ac.encodingConfig, options) | 		bApp := app.NewBaseApp(logger, db, ac.encodingConfig) | ||||||
|  | 		tempApp = app.NewApp(homePath, traceStore, ac.encodingConfig, options, bApp) | ||||||
| 	} | 	} | ||||||
| 	return tempApp.ExportAppStateAndValidators(forZeroHeight, jailAllowedAddrs, modulesToExport) | 	return tempApp.ExportAppStateAndValidators(forZeroHeight, jailAllowedAddrs, modulesToExport) | ||||||
| } | } | ||||||
|  | |||||||
		Loading…
	
		Reference in New Issue
	
	Block a user
	 0g-wh
						0g-wh