mirror of
https://github.com/paradigmxyz/reth.git
synced 2026-01-13 17:28:10 -05:00
feat: add precompile cache for execution (#15928)
Co-authored-by: Arsenii Kulikov <klkvrr@gmail.com> Co-authored-by: graphite-app[bot] <96075541+graphite-app[bot]@users.noreply.github.com> Co-authored-by: Matthias Seitz <matthias.seitz@outlook.de>
This commit is contained in:
544
Cargo.lock
generated
544
Cargo.lock
generated
File diff suppressed because it is too large
Load Diff
28
Cargo.toml
28
Cargo.toml
@@ -444,24 +444,24 @@ reth-ress-protocol = { path = "crates/ress/protocol" }
|
||||
reth-ress-provider = { path = "crates/ress/provider" }
|
||||
|
||||
# revm
|
||||
revm = { version = "22.0.1", default-features = false }
|
||||
revm-bytecode = { version = "3.0.0", default-features = false }
|
||||
revm-state = { version = "=3.0.0", default-features = false }
|
||||
revm-primitives = { version = "18.0.0", default-features = false }
|
||||
revm-interpreter = { version = "18.0.0", default-features = false }
|
||||
revm-inspector = { version = "3.0.0", default-features = false }
|
||||
revm-context = { version = "3.0.0", default-features = false }
|
||||
revm-context-interface = { version = "3.0.0", default-features = false }
|
||||
revm-database = { version = "=3.0.0", default-features = false }
|
||||
revm-database-interface = { version = "=3.0.0", default-features = false }
|
||||
op-revm = { version = "=3.0.2", default-features = false }
|
||||
revm-inspectors = "0.20.0"
|
||||
revm = { version = "23.1.0", default-features = false }
|
||||
revm-bytecode = { version = "4.0.0", default-features = false }
|
||||
revm-database = { version = "4.0.0", default-features = false }
|
||||
revm-state = { version = "4.0.0", default-features = false }
|
||||
revm-primitives = { version = "19.0.0", default-features = false }
|
||||
revm-interpreter = { version = "19.0.0", default-features = false }
|
||||
revm-inspector = { version = "4.0.0", default-features = false }
|
||||
revm-context = { version = "4.0.0", default-features = false }
|
||||
revm-context-interface = { version = "4.0.0", default-features = false }
|
||||
revm-database-interface = { version = "4.0.0", default-features = false }
|
||||
op-revm = { version = "4.0.0", default-features = false }
|
||||
revm-inspectors = "0.21.0"
|
||||
|
||||
# eth
|
||||
alloy-chains = { version = "0.2.0", default-features = false }
|
||||
alloy-dyn-abi = "1.1.0"
|
||||
alloy-eip2124 = { version = "0.2.0", default-features = false }
|
||||
alloy-evm = { version = "0.6.0", default-features = false }
|
||||
alloy-evm = { version = "0.7.0", default-features = false }
|
||||
alloy-primitives = { version = "1.1.0", default-features = false, features = ["map-foldhash"] }
|
||||
alloy-rlp = { version = "0.3.10", default-features = false, features = ["core-net"] }
|
||||
alloy-sol-macro = "1.1.0"
|
||||
@@ -499,7 +499,7 @@ alloy-transport-ipc = { version = "0.15.10", default-features = false }
|
||||
alloy-transport-ws = { version = "0.15.10", default-features = false }
|
||||
|
||||
# op
|
||||
alloy-op-evm = { version = "0.6.0", default-features = false }
|
||||
alloy-op-evm = { version = "0.7.0", default-features = false }
|
||||
alloy-op-hardforks = "0.2.0"
|
||||
op-alloy-rpc-types = { version = "0.15.4", default-features = false }
|
||||
op-alloy-rpc-types-engine = { version = "0.15.4", default-features = false }
|
||||
|
||||
3
book/cli/reth/node.md
vendored
3
book/cli/reth/node.md
vendored
@@ -764,6 +764,9 @@ Engine:
|
||||
|
||||
[default: 1]
|
||||
|
||||
--engine.precompile-cache
|
||||
Enable precompile cache
|
||||
|
||||
Ress:
|
||||
--ress.enable
|
||||
Enable support for `ress` subprotocol
|
||||
|
||||
@@ -75,6 +75,8 @@ pub struct TreeConfig {
|
||||
max_proof_task_concurrency: u64,
|
||||
/// Number of reserved CPU cores for non-reth processes
|
||||
reserved_cpu_cores: usize,
|
||||
/// Whether to enable the precompile cache
|
||||
precompile_cache_enabled: bool,
|
||||
}
|
||||
|
||||
impl Default for TreeConfig {
|
||||
@@ -93,6 +95,7 @@ impl Default for TreeConfig {
|
||||
has_enough_parallelism: has_enough_parallelism(),
|
||||
max_proof_task_concurrency: DEFAULT_MAX_PROOF_TASK_CONCURRENCY,
|
||||
reserved_cpu_cores: DEFAULT_RESERVED_CPU_CORES,
|
||||
precompile_cache_enabled: false,
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -114,6 +117,7 @@ impl TreeConfig {
|
||||
has_enough_parallelism: bool,
|
||||
max_proof_task_concurrency: u64,
|
||||
reserved_cpu_cores: usize,
|
||||
precompile_cache_enabled: bool,
|
||||
) -> Self {
|
||||
Self {
|
||||
persistence_threshold,
|
||||
@@ -129,6 +133,7 @@ impl TreeConfig {
|
||||
has_enough_parallelism,
|
||||
max_proof_task_concurrency,
|
||||
reserved_cpu_cores,
|
||||
precompile_cache_enabled,
|
||||
}
|
||||
}
|
||||
|
||||
@@ -189,11 +194,16 @@ impl TreeConfig {
|
||||
self.always_compare_trie_updates
|
||||
}
|
||||
|
||||
/// Return the cross-block cache size.
|
||||
/// Returns the cross-block cache size.
|
||||
pub const fn cross_block_cache_size(&self) -> u64 {
|
||||
self.cross_block_cache_size
|
||||
}
|
||||
|
||||
/// Returns whether precompile cache is enabled.
|
||||
pub const fn precompile_cache_enabled(&self) -> bool {
|
||||
self.precompile_cache_enabled
|
||||
}
|
||||
|
||||
/// Setter for persistence threshold.
|
||||
pub const fn with_persistence_threshold(mut self, persistence_threshold: u64) -> Self {
|
||||
self.persistence_threshold = persistence_threshold;
|
||||
@@ -291,6 +301,12 @@ impl TreeConfig {
|
||||
self
|
||||
}
|
||||
|
||||
/// Setter for whether to use the precompile cache.
|
||||
pub const fn with_precompile_cache_enabled(mut self, precompile_cache_enabled: bool) -> Self {
|
||||
self.precompile_cache_enabled = precompile_cache_enabled;
|
||||
self
|
||||
}
|
||||
|
||||
/// Whether or not to use state root task
|
||||
pub const fn use_state_root_task(&self) -> bool {
|
||||
self.has_enough_parallelism && !self.legacy_state_root
|
||||
|
||||
@@ -42,6 +42,7 @@ alloy-primitives.workspace = true
|
||||
alloy-rlp.workspace = true
|
||||
alloy-rpc-types-engine.workspace = true
|
||||
|
||||
revm.workspace = true
|
||||
revm-primitives.workspace = true
|
||||
|
||||
# common
|
||||
|
||||
@@ -13,7 +13,8 @@ use reth_chain_state::EthPrimitives;
|
||||
use reth_chainspec::ChainSpec;
|
||||
use reth_db_common::init::init_genesis;
|
||||
use reth_engine_tree::tree::{
|
||||
executor::WorkloadExecutor, PayloadProcessor, StateProviderBuilder, TreeConfig,
|
||||
executor::WorkloadExecutor, precompile_cache::PrecompileCache, PayloadProcessor,
|
||||
StateProviderBuilder, TreeConfig,
|
||||
};
|
||||
use reth_evm::OnStateHook;
|
||||
use reth_evm_ethereum::EthEvmConfig;
|
||||
@@ -220,6 +221,7 @@ fn bench_state_root(c: &mut Criterion) {
|
||||
WorkloadExecutor::default(),
|
||||
EthEvmConfig::new(factory.chain_spec()),
|
||||
&TreeConfig::default(),
|
||||
PrecompileCache::default(),
|
||||
);
|
||||
let provider = BlockchainProvider::new(factory).unwrap();
|
||||
|
||||
|
||||
@@ -9,6 +9,7 @@ use crate::{
|
||||
};
|
||||
use alloy_consensus::BlockHeader;
|
||||
use alloy_eips::{merge::EPOCH_SLOTS, BlockNumHash, NumHash};
|
||||
use alloy_evm::block::BlockExecutor;
|
||||
use alloy_primitives::{
|
||||
map::{HashMap, HashSet},
|
||||
BlockNumber, B256,
|
||||
@@ -20,6 +21,7 @@ use error::{InsertBlockError, InsertBlockErrorKind, InsertBlockFatalError};
|
||||
use instrumented_state::InstrumentedStateProvider;
|
||||
use payload_processor::sparse_trie::StateRootComputeOutcome;
|
||||
use persistence_state::CurrentPersistenceAction;
|
||||
use precompile_cache::{CachedPrecompile, PrecompileCache};
|
||||
use reth_chain_state::{
|
||||
CanonicalInMemoryState, ExecutedBlock, ExecutedBlockWithTrieUpdates,
|
||||
MemoryOverlayStateProvider, NewCanonicalChain,
|
||||
@@ -32,7 +34,7 @@ use reth_engine_primitives::{
|
||||
};
|
||||
use reth_errors::{ConsensusError, ProviderResult};
|
||||
use reth_ethereum_primitives::EthPrimitives;
|
||||
use reth_evm::ConfigureEvm;
|
||||
use reth_evm::{ConfigureEvm, Evm};
|
||||
use reth_payload_builder::PayloadBuilderHandle;
|
||||
use reth_payload_primitives::{EngineApiMessageVersion, PayloadBuilderAttributes, PayloadTypes};
|
||||
use reth_primitives_traits::{
|
||||
@@ -74,6 +76,7 @@ mod invalid_headers;
|
||||
mod metrics;
|
||||
mod payload_processor;
|
||||
mod persistence_state;
|
||||
pub mod precompile_cache;
|
||||
// TODO(alexey): compare trie updates in `insert_block_inner`
|
||||
#[expect(unused)]
|
||||
mod trie_updates;
|
||||
@@ -613,6 +616,8 @@ where
|
||||
payload_processor: PayloadProcessor<N, C>,
|
||||
/// The EVM configuration.
|
||||
evm_config: C,
|
||||
/// Precompile cache.
|
||||
precompile_cache: PrecompileCache,
|
||||
}
|
||||
|
||||
impl<N, P: Debug, T: PayloadTypes + Debug, V: Debug, C: Debug> std::fmt::Debug
|
||||
@@ -636,6 +641,8 @@ where
|
||||
.field("metrics", &self.metrics)
|
||||
.field("invalid_block_hook", &format!("{:p}", self.invalid_block_hook))
|
||||
.field("engine_kind", &self.engine_kind)
|
||||
.field("payload_processor", &self.payload_processor)
|
||||
.field("evm_config", &self.evm_config)
|
||||
.finish()
|
||||
}
|
||||
}
|
||||
@@ -675,8 +682,14 @@ where
|
||||
) -> Self {
|
||||
let (incoming_tx, incoming) = std::sync::mpsc::channel();
|
||||
|
||||
let payload_processor =
|
||||
PayloadProcessor::new(WorkloadExecutor::default(), evm_config.clone(), &config);
|
||||
let precompile_cache = PrecompileCache::default();
|
||||
|
||||
let payload_processor = PayloadProcessor::new(
|
||||
WorkloadExecutor::default(),
|
||||
evm_config.clone(),
|
||||
&config,
|
||||
precompile_cache.clone(),
|
||||
);
|
||||
|
||||
Self {
|
||||
provider,
|
||||
@@ -697,6 +710,7 @@ where
|
||||
engine_kind,
|
||||
payload_processor,
|
||||
evm_config,
|
||||
precompile_cache,
|
||||
}
|
||||
}
|
||||
|
||||
@@ -2619,7 +2633,14 @@ where
|
||||
.with_bundle_update()
|
||||
.without_state_clear()
|
||||
.build();
|
||||
let executor = self.evm_config.executor_for_block(&mut db, block);
|
||||
let mut executor = self.evm_config.executor_for_block(&mut db, block);
|
||||
|
||||
if self.config.precompile_cache_enabled() {
|
||||
executor.evm_mut().precompiles_mut().map_precompiles(|_, precompile| {
|
||||
CachedPrecompile::wrap(precompile, self.precompile_cache.clone())
|
||||
});
|
||||
}
|
||||
|
||||
let execution_start = Instant::now();
|
||||
let output = self.metrics.executor.execute_metered(
|
||||
executor,
|
||||
|
||||
@@ -6,6 +6,7 @@ use crate::tree::{
|
||||
prewarm::{PrewarmCacheTask, PrewarmContext, PrewarmTaskEvent},
|
||||
sparse_trie::StateRootComputeOutcome,
|
||||
},
|
||||
precompile_cache::PrecompileCache,
|
||||
sparse_trie::SparseTrieTask,
|
||||
StateProviderBuilder, TreeConfig,
|
||||
};
|
||||
@@ -58,12 +59,21 @@ pub struct PayloadProcessor<N, Evm> {
|
||||
disable_transaction_prewarming: bool,
|
||||
/// Determines how to configure the evm for execution.
|
||||
evm_config: Evm,
|
||||
/// whether precompile cache should be enabled.
|
||||
precompile_cache_enabled: bool,
|
||||
/// Precompile cache.
|
||||
precompile_cache: PrecompileCache,
|
||||
_marker: std::marker::PhantomData<N>,
|
||||
}
|
||||
|
||||
impl<N, Evm> PayloadProcessor<N, Evm> {
|
||||
/// Creates a new payload processor.
|
||||
pub fn new(executor: WorkloadExecutor, evm_config: Evm, config: &TreeConfig) -> Self {
|
||||
pub fn new(
|
||||
executor: WorkloadExecutor,
|
||||
evm_config: Evm,
|
||||
config: &TreeConfig,
|
||||
precompile_cache: PrecompileCache,
|
||||
) -> Self {
|
||||
Self {
|
||||
executor,
|
||||
execution_cache: Default::default(),
|
||||
@@ -71,6 +81,8 @@ impl<N, Evm> PayloadProcessor<N, Evm> {
|
||||
cross_block_cache_size: config.cross_block_cache_size(),
|
||||
disable_transaction_prewarming: config.disable_caching_and_prewarming(),
|
||||
evm_config,
|
||||
precompile_cache_enabled: config.precompile_cache_enabled(),
|
||||
precompile_cache,
|
||||
_marker: Default::default(),
|
||||
}
|
||||
}
|
||||
@@ -253,6 +265,8 @@ where
|
||||
provider: provider_builder,
|
||||
metrics: PrewarmMetrics::default(),
|
||||
terminate_execution: Arc::new(AtomicBool::new(false)),
|
||||
precompile_cache_enabled: self.precompile_cache_enabled,
|
||||
precompile_cache: self.precompile_cache.clone(),
|
||||
};
|
||||
|
||||
let prewarm_task = PrewarmCacheTask::new(
|
||||
@@ -426,6 +440,7 @@ mod tests {
|
||||
payload_processor::{
|
||||
evm_state_to_hashed_post_state, executor::WorkloadExecutor, PayloadProcessor,
|
||||
},
|
||||
precompile_cache::PrecompileCache,
|
||||
StateProviderBuilder, TreeConfig,
|
||||
};
|
||||
use alloy_evm::block::StateChangeSource;
|
||||
@@ -548,6 +563,7 @@ mod tests {
|
||||
WorkloadExecutor::default(),
|
||||
EthEvmConfig::new(factory.chain_spec()),
|
||||
&TreeConfig::default(),
|
||||
PrecompileCache::default(),
|
||||
);
|
||||
let provider = BlockchainProvider::new(factory).unwrap();
|
||||
let mut handle = payload_processor.spawn(
|
||||
|
||||
@@ -5,6 +5,7 @@ use crate::tree::{
|
||||
payload_processor::{
|
||||
executor::WorkloadExecutor, multiproof::MultiProofMessage, ExecutionCache,
|
||||
},
|
||||
precompile_cache::{CachedPrecompile, PrecompileCache},
|
||||
StateProviderBuilder,
|
||||
};
|
||||
use alloy_consensus::transaction::Recovered;
|
||||
@@ -205,6 +206,8 @@ pub(super) struct PrewarmContext<N: NodePrimitives, P, Evm> {
|
||||
pub(super) metrics: PrewarmMetrics,
|
||||
/// An atomic bool that tells prewarm tasks to not start any more execution.
|
||||
pub(super) terminate_execution: Arc<AtomicBool>,
|
||||
pub(super) precompile_cache_enabled: bool,
|
||||
pub(super) precompile_cache: PrecompileCache,
|
||||
}
|
||||
|
||||
impl<N, P, Evm> PrewarmContext<N, P, Evm>
|
||||
@@ -226,6 +229,8 @@ where
|
||||
provider,
|
||||
metrics,
|
||||
terminate_execution,
|
||||
precompile_cache_enabled,
|
||||
precompile_cache,
|
||||
} = self;
|
||||
|
||||
let state_provider = match provider.build() {
|
||||
@@ -253,7 +258,13 @@ where
|
||||
evm_env.cfg_env.disable_nonce_check = true;
|
||||
|
||||
// create a new executor and disable nonce checks in the env
|
||||
let evm = evm_config.evm_with_env(state_provider, evm_env);
|
||||
let mut evm = evm_config.evm_with_env(state_provider, evm_env);
|
||||
|
||||
if precompile_cache_enabled {
|
||||
evm.precompiles_mut().map_precompiles(|_, precompile| {
|
||||
CachedPrecompile::wrap(precompile, precompile_cache.clone())
|
||||
});
|
||||
}
|
||||
|
||||
Some((evm, evm_config, metrics, terminate_execution))
|
||||
}
|
||||
|
||||
171
crates/engine/tree/src/tree/precompile_cache.rs
Normal file
171
crates/engine/tree/src/tree/precompile_cache.rs
Normal file
@@ -0,0 +1,171 @@
|
||||
//! Contains a precompile cache that is backed by a moka cache.
|
||||
|
||||
use reth_evm::precompiles::{DynPrecompile, Precompile};
|
||||
use revm::precompile::{PrecompileOutput, PrecompileResult};
|
||||
use revm_primitives::Bytes;
|
||||
use std::sync::Arc;
|
||||
|
||||
/// Cache for precompiles, for each input stores the result.
|
||||
#[derive(Debug, Clone)]
|
||||
pub struct PrecompileCache(
|
||||
Arc<mini_moka::sync::Cache<CacheKey, CacheEntry, alloy_primitives::map::DefaultHashBuilder>>,
|
||||
);
|
||||
|
||||
impl Default for PrecompileCache {
|
||||
fn default() -> Self {
|
||||
Self(Arc::new(
|
||||
mini_moka::sync::CacheBuilder::new(100_000)
|
||||
.build_with_hasher(alloy_primitives::map::DefaultHashBuilder::default()),
|
||||
))
|
||||
}
|
||||
}
|
||||
|
||||
impl PrecompileCache {
|
||||
fn get(&self, key: &CacheKey) -> Option<CacheEntry> {
|
||||
self.0.get(key)
|
||||
}
|
||||
|
||||
fn insert(&self, key: CacheKey, value: CacheEntry) {
|
||||
self.0.insert(key, value);
|
||||
}
|
||||
|
||||
fn weighted_size(&self) -> u64 {
|
||||
self.0.weighted_size()
|
||||
}
|
||||
}
|
||||
|
||||
/// Cache key, precompile call input.
|
||||
#[derive(Debug, Clone, PartialEq, Eq, Hash)]
|
||||
pub struct CacheKey(alloy_primitives::Bytes);
|
||||
|
||||
/// Cache entry, precompile successful output.
|
||||
#[derive(Debug, Clone, PartialEq, Eq)]
|
||||
pub struct CacheEntry(PrecompileOutput);
|
||||
|
||||
impl CacheEntry {
|
||||
const fn gas_used(&self) -> u64 {
|
||||
self.0.gas_used
|
||||
}
|
||||
|
||||
fn to_precompile_result(&self) -> PrecompileResult {
|
||||
Ok(self.0.clone())
|
||||
}
|
||||
}
|
||||
|
||||
/// A cache for precompile inputs / outputs.
|
||||
#[derive(Debug)]
|
||||
pub(crate) struct CachedPrecompile {
|
||||
/// Cache for precompile results and gas bounds.
|
||||
cache: PrecompileCache,
|
||||
/// The precompile.
|
||||
precompile: DynPrecompile,
|
||||
/// Cache metrics.
|
||||
metrics: CachedPrecompileMetrics,
|
||||
}
|
||||
|
||||
impl CachedPrecompile {
|
||||
/// `CachedPrecompile` constructor.
|
||||
pub(crate) fn new(precompile: DynPrecompile, cache: PrecompileCache) -> Self {
|
||||
Self { precompile, cache, metrics: Default::default() }
|
||||
}
|
||||
|
||||
pub(crate) fn wrap(precompile: DynPrecompile, cache: PrecompileCache) -> DynPrecompile {
|
||||
let wrapped = Self::new(precompile, cache);
|
||||
move |data: &[u8], gas_limit: u64| -> PrecompileResult { wrapped.call(data, gas_limit) }
|
||||
.into()
|
||||
}
|
||||
|
||||
fn increment_by_one_precompile_cache_hits(&self) {
|
||||
self.metrics.precompile_cache_hits.increment(1);
|
||||
}
|
||||
|
||||
fn increment_by_one_precompile_cache_misses(&self) {
|
||||
self.metrics.precompile_cache_misses.increment(1);
|
||||
}
|
||||
|
||||
fn increment_by_one_precompile_errors(&self) {
|
||||
self.metrics.precompile_errors.increment(1);
|
||||
}
|
||||
|
||||
fn update_precompile_cache_size(&self) {
|
||||
let new_size = self.cache.weighted_size();
|
||||
self.metrics.precompile_cache_size.set(new_size as f64);
|
||||
}
|
||||
}
|
||||
|
||||
impl Precompile for CachedPrecompile {
|
||||
fn call(&self, data: &[u8], gas_limit: u64) -> PrecompileResult {
|
||||
let key = CacheKey(Bytes::copy_from_slice(data));
|
||||
|
||||
if let Some(entry) = &self.cache.get(&key) {
|
||||
self.increment_by_one_precompile_cache_hits();
|
||||
if gas_limit >= entry.gas_used() {
|
||||
return entry.to_precompile_result()
|
||||
}
|
||||
}
|
||||
|
||||
let result = self.precompile.call(data, gas_limit);
|
||||
|
||||
match &result {
|
||||
Ok(output) => {
|
||||
self.increment_by_one_precompile_cache_misses();
|
||||
self.cache.insert(key, CacheEntry(output.clone()));
|
||||
}
|
||||
_ => {
|
||||
self.increment_by_one_precompile_errors();
|
||||
}
|
||||
}
|
||||
|
||||
self.update_precompile_cache_size();
|
||||
result
|
||||
}
|
||||
}
|
||||
|
||||
/// Metrics for the cached precompile.
|
||||
#[derive(reth_metrics::Metrics, Clone)]
|
||||
#[metrics(scope = "sync.caching")]
|
||||
pub(crate) struct CachedPrecompileMetrics {
|
||||
/// Precompile cache hits
|
||||
precompile_cache_hits: metrics::Counter,
|
||||
|
||||
/// Precompile cache misses
|
||||
precompile_cache_misses: metrics::Counter,
|
||||
|
||||
/// Precompile cache size
|
||||
///
|
||||
/// NOTE: this uses the moka caches`weighted_size` method to calculate size.
|
||||
precompile_cache_size: metrics::Gauge,
|
||||
|
||||
/// Precompile execution errors.
|
||||
precompile_errors: metrics::Counter,
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
mod tests {
|
||||
use super::*;
|
||||
use revm::precompile::PrecompileOutput;
|
||||
|
||||
#[test]
|
||||
fn test_precompile_cache_basic() {
|
||||
let dyn_precompile: DynPrecompile = |_input: &[u8], _gas: u64| -> PrecompileResult {
|
||||
Ok(PrecompileOutput { gas_used: 0, bytes: Bytes::default() })
|
||||
}
|
||||
.into();
|
||||
|
||||
let cache = CachedPrecompile::new(dyn_precompile, PrecompileCache::default());
|
||||
|
||||
let key = CacheKey(b"test_input".into());
|
||||
|
||||
let output = PrecompileOutput {
|
||||
gas_used: 50,
|
||||
bytes: alloy_primitives::Bytes::copy_from_slice(b"cached_result"),
|
||||
};
|
||||
|
||||
let expected = CacheEntry(output);
|
||||
cache.cache.insert(key.clone(), expected.clone());
|
||||
|
||||
let actual = cache.cache.get(&key).unwrap();
|
||||
|
||||
assert_eq!(actual, expected);
|
||||
}
|
||||
}
|
||||
@@ -28,7 +28,10 @@ use alloy_primitives::{Bytes, U256};
|
||||
use core::{convert::Infallible, fmt::Debug};
|
||||
use reth_chainspec::{ChainSpec, EthChainSpec, MAINNET};
|
||||
use reth_ethereum_primitives::{Block, EthPrimitives, TransactionSigned};
|
||||
use reth_evm::{ConfigureEvm, EvmEnv, EvmFactory, NextBlockEnvAttributes, TransactionEnv};
|
||||
use reth_evm::{
|
||||
precompiles::PrecompilesMap, ConfigureEvm, EvmEnv, EvmFactory, NextBlockEnvAttributes,
|
||||
TransactionEnv,
|
||||
};
|
||||
use reth_primitives_traits::{SealedBlock, SealedHeader};
|
||||
use revm::{
|
||||
context::{BlockEnv, CfgEnv},
|
||||
@@ -123,6 +126,7 @@ where
|
||||
+ FromRecoveredTx<TransactionSigned>
|
||||
+ FromTxWithEncoded<TransactionSigned>,
|
||||
Spec = SpecId,
|
||||
Precompiles = PrecompilesMap,
|
||||
> + Clone
|
||||
+ Debug
|
||||
+ Send
|
||||
|
||||
@@ -2,6 +2,7 @@ use crate::EthEvmConfig;
|
||||
use alloc::{boxed::Box, sync::Arc, vec::Vec};
|
||||
use alloy_consensus::Header;
|
||||
use alloy_eips::eip7685::Requests;
|
||||
use alloy_evm::precompiles::PrecompilesMap;
|
||||
use parking_lot::Mutex;
|
||||
use reth_ethereum_primitives::{Receipt, TransactionSigned};
|
||||
use reth_evm::{
|
||||
@@ -54,7 +55,7 @@ impl BlockExecutorFactory for MockEvmConfig {
|
||||
|
||||
fn create_executor<'a, DB, I>(
|
||||
&'a self,
|
||||
evm: EthEvm<&'a mut State<DB>, I>,
|
||||
evm: EthEvm<&'a mut State<DB>, I, PrecompilesMap>,
|
||||
_ctx: Self::ExecutionCtx<'a>,
|
||||
) -> impl BlockExecutorFor<'a, Self, DB, I>
|
||||
where
|
||||
@@ -69,7 +70,7 @@ impl BlockExecutorFactory for MockEvmConfig {
|
||||
#[derive(derive_more::Debug)]
|
||||
pub struct MockExecutor<'a, DB: Database, I> {
|
||||
result: ExecutionOutcome,
|
||||
evm: EthEvm<&'a mut State<DB>, I>,
|
||||
evm: EthEvm<&'a mut State<DB>, I, PrecompilesMap>,
|
||||
#[debug(skip)]
|
||||
hook: Option<Box<dyn reth_evm::OnStateHook>>,
|
||||
}
|
||||
@@ -77,7 +78,7 @@ pub struct MockExecutor<'a, DB: Database, I> {
|
||||
impl<'a, DB: Database, I: Inspector<EthEvmContext<&'a mut State<DB>>>> BlockExecutor
|
||||
for MockExecutor<'a, DB, I>
|
||||
{
|
||||
type Evm = EthEvm<&'a mut State<DB>, I>;
|
||||
type Evm = EthEvm<&'a mut State<DB>, I, PrecompilesMap>;
|
||||
type Transaction = TransactionSigned;
|
||||
type Receipt = Receipt;
|
||||
|
||||
|
||||
@@ -21,7 +21,6 @@ reth-storage-errors.workspace = true
|
||||
reth-trie-common.workspace = true
|
||||
|
||||
revm.workspace = true
|
||||
op-revm = { workspace = true, optional = true }
|
||||
|
||||
# alloy
|
||||
alloy-primitives.workspace = true
|
||||
@@ -50,7 +49,6 @@ std = [
|
||||
"revm/std",
|
||||
"reth-ethereum-forks/std",
|
||||
"alloy-evm/std",
|
||||
"op-revm?/std",
|
||||
"reth-execution-errors/std",
|
||||
"reth-execution-types/std",
|
||||
"reth-storage-errors/std",
|
||||
@@ -66,4 +64,4 @@ test-utils = [
|
||||
"reth-trie-common/test-utils",
|
||||
"reth-ethereum-primitives/test-utils",
|
||||
]
|
||||
op = ["op-revm", "alloy-evm/op", "reth-primitives-traits/op"]
|
||||
op = ["alloy-evm/op", "reth-primitives-traits/op"]
|
||||
|
||||
@@ -19,8 +19,15 @@ extern crate alloc;
|
||||
|
||||
use crate::execute::BasicBlockBuilder;
|
||||
use alloc::vec::Vec;
|
||||
use alloy_eips::{eip2930::AccessList, eip4895::Withdrawals};
|
||||
use alloy_evm::block::{BlockExecutorFactory, BlockExecutorFor};
|
||||
use alloy_eips::{
|
||||
eip2718::{EIP2930_TX_TYPE_ID, LEGACY_TX_TYPE_ID},
|
||||
eip2930::AccessList,
|
||||
eip4895::Withdrawals,
|
||||
};
|
||||
use alloy_evm::{
|
||||
block::{BlockExecutorFactory, BlockExecutorFor},
|
||||
precompiles::PrecompilesMap,
|
||||
};
|
||||
use alloy_primitives::{Address, B256};
|
||||
use core::{error::Error, fmt::Debug};
|
||||
use execute::{BasicBlockExecutor, BlockAssembler, BlockBuilder};
|
||||
@@ -109,6 +116,7 @@ pub trait ConfigureEvm: Clone + Debug + Send + Sync + Unpin {
|
||||
Tx: TransactionEnv
|
||||
+ FromRecoveredTx<TxTy<Self::Primitives>>
|
||||
+ FromTxWithEncoded<TxTy<Self::Primitives>>,
|
||||
Precompiles = PrecompilesMap,
|
||||
>,
|
||||
>;
|
||||
|
||||
@@ -354,6 +362,12 @@ impl TransactionEnv for TxEnv {
|
||||
|
||||
fn set_access_list(&mut self, access_list: AccessList) {
|
||||
self.access_list = access_list;
|
||||
|
||||
if self.tx_type == LEGACY_TX_TYPE_ID {
|
||||
// if this was previously marked as legacy tx, this must be upgraded to eip2930 with an
|
||||
// accesslist
|
||||
self.tx_type = EIP2930_TX_TYPE_ID;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@@ -59,6 +59,10 @@ pub struct EngineArgs {
|
||||
/// Configure the number of reserved CPU cores for non-reth processes
|
||||
#[arg(long = "engine.reserved-cpu-cores", default_value_t = DEFAULT_RESERVED_CPU_CORES)]
|
||||
pub reserved_cpu_cores: usize,
|
||||
|
||||
/// Enable precompile cache
|
||||
#[arg(long = "engine.precompile-cache", default_value = "false")]
|
||||
pub precompile_cache_enabled: bool,
|
||||
}
|
||||
|
||||
impl Default for EngineArgs {
|
||||
@@ -75,6 +79,7 @@ impl Default for EngineArgs {
|
||||
accept_execution_requests_hash: false,
|
||||
max_proof_task_concurrency: DEFAULT_MAX_PROOF_TASK_CONCURRENCY,
|
||||
reserved_cpu_cores: DEFAULT_RESERVED_CPU_CORES,
|
||||
precompile_cache_enabled: false,
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -92,6 +97,7 @@ impl EngineArgs {
|
||||
.with_cross_block_cache_size(self.cross_block_cache_size * 1024 * 1024)
|
||||
.with_max_proof_task_concurrency(self.max_proof_task_concurrency)
|
||||
.with_reserved_cpu_cores(self.reserved_cpu_cores)
|
||||
.with_precompile_cache_enabled(self.precompile_cache_enabled)
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@@ -1,6 +1,6 @@
|
||||
use super::OpNodeCore;
|
||||
use crate::{OpEthApi, OpEthApiError};
|
||||
use alloy_consensus::TxType;
|
||||
use alloy_consensus::{transaction::Either, TxType};
|
||||
use alloy_primitives::{Bytes, TxKind, U256};
|
||||
use alloy_rpc_types_eth::transaction::TransactionRequest;
|
||||
use op_revm::OpTransaction;
|
||||
@@ -149,7 +149,11 @@ where
|
||||
.map(|v| v.saturating_to())
|
||||
.unwrap_or_default(),
|
||||
// EIP-7702 fields
|
||||
authorization_list: authorization_list.unwrap_or_default(),
|
||||
authorization_list: authorization_list
|
||||
.unwrap_or_default()
|
||||
.into_iter()
|
||||
.map(Either::Left)
|
||||
.collect(),
|
||||
};
|
||||
|
||||
Ok(OpTransaction { base, enveloped_tx: Some(Bytes::new()), deposit: Default::default() })
|
||||
|
||||
@@ -296,12 +296,12 @@ mod tests {
|
||||
let mut buf = vec![];
|
||||
let bytecode = Bytecode::new_raw(Bytes::default());
|
||||
let len = bytecode.to_compact(&mut buf);
|
||||
assert_eq!(len, 51);
|
||||
assert_eq!(len, 14);
|
||||
|
||||
let mut buf = vec![];
|
||||
let bytecode = Bytecode::new_raw(Bytes::from(&hex!("ffff")));
|
||||
let len = bytecode.to_compact(&mut buf);
|
||||
assert_eq!(len, 53);
|
||||
assert_eq!(len, 17);
|
||||
|
||||
let mut buf = vec![];
|
||||
let bytecode = Bytecode(RevmBytecode::LegacyAnalyzed(LegacyAnalyzedBytecode::new(
|
||||
|
||||
@@ -580,7 +580,11 @@ impl From<InvalidTransaction> for RpcInvalidTransactionError {
|
||||
InvalidTransaction::Eip2930NotSupported |
|
||||
InvalidTransaction::Eip1559NotSupported |
|
||||
InvalidTransaction::Eip4844NotSupported |
|
||||
InvalidTransaction::Eip7702NotSupported => Self::TxTypeNotSupported,
|
||||
InvalidTransaction::Eip7702NotSupported |
|
||||
InvalidTransaction::Eip7873NotSupported => Self::TxTypeNotSupported,
|
||||
InvalidTransaction::Eip7873MissingTarget => {
|
||||
Self::other(internal_rpc_err(err.to_string()))
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -5,6 +5,7 @@ use alloy_consensus::TxType;
|
||||
use alloy_evm::block::BlockExecutorFactory;
|
||||
use alloy_primitives::{TxKind, U256};
|
||||
use alloy_rpc_types::TransactionRequest;
|
||||
use alloy_signer::Either;
|
||||
use reth_evm::{ConfigureEvm, EvmEnv, EvmFactory, SpecFor};
|
||||
use reth_node_api::NodePrimitives;
|
||||
use reth_rpc_eth_api::{
|
||||
@@ -139,7 +140,11 @@ where
|
||||
.map(|v| v.saturating_to())
|
||||
.unwrap_or_default(),
|
||||
// EIP-7702 fields
|
||||
authorization_list: authorization_list.unwrap_or_default(),
|
||||
authorization_list: authorization_list
|
||||
.unwrap_or_default()
|
||||
.into_iter()
|
||||
.map(Either::Left)
|
||||
.collect(),
|
||||
};
|
||||
|
||||
Ok(env)
|
||||
|
||||
@@ -53,6 +53,7 @@ allow = [
|
||||
# https://github.com/rustls/webpki/blob/main/LICENSE ISC Style
|
||||
"LicenseRef-rustls-webpki",
|
||||
"CDLA-Permissive-2.0",
|
||||
"MPL-2.0",
|
||||
]
|
||||
|
||||
# Allow 1 or more licenses on a per-crate basis, so that particular licenses
|
||||
|
||||
@@ -7,6 +7,7 @@ use alloy_eips::eip4895::Withdrawal;
|
||||
use alloy_evm::{
|
||||
block::{BlockExecutorFactory, BlockExecutorFor, ExecutableTx},
|
||||
eth::{EthBlockExecutionCtx, EthBlockExecutor},
|
||||
precompiles::PrecompilesMap,
|
||||
EthEvm, EthEvmFactory,
|
||||
};
|
||||
use alloy_sol_macro::sol;
|
||||
@@ -101,7 +102,7 @@ impl BlockExecutorFactory for CustomEvmConfig {
|
||||
|
||||
fn create_executor<'a, DB, I>(
|
||||
&'a self,
|
||||
evm: EthEvm<&'a mut State<DB>, I>,
|
||||
evm: EthEvm<&'a mut State<DB>, I, PrecompilesMap>,
|
||||
ctx: EthBlockExecutionCtx<'a>,
|
||||
) -> impl BlockExecutorFor<'a, Self, DB, I>
|
||||
where
|
||||
|
||||
@@ -2,9 +2,9 @@
|
||||
|
||||
#![warn(unused_crate_dependencies)]
|
||||
|
||||
use alloy_evm::{eth::EthEvmContext, EvmFactory};
|
||||
use alloy_evm::{eth::EthEvmContext, precompiles::PrecompilesMap, EvmFactory};
|
||||
use alloy_genesis::Genesis;
|
||||
use alloy_primitives::{address, Address, Bytes};
|
||||
use alloy_primitives::{address, Bytes};
|
||||
use reth::{
|
||||
builder::{components::ExecutorBuilder, BuilderContext, NodeBuilder},
|
||||
tasks::TaskManager,
|
||||
@@ -14,14 +14,11 @@ use reth_ethereum::{
|
||||
evm::{
|
||||
primitives::{Database, EvmEnv},
|
||||
revm::{
|
||||
context::{Cfg, Context, TxEnv},
|
||||
context_interface::{
|
||||
result::{EVMError, HaltReason},
|
||||
ContextTr,
|
||||
},
|
||||
handler::{EthPrecompiles, PrecompileProvider},
|
||||
context::{Context, TxEnv},
|
||||
context_interface::result::{EVMError, HaltReason},
|
||||
handler::EthPrecompiles,
|
||||
inspector::{Inspector, NoOpInspector},
|
||||
interpreter::{interpreter::EthInterpreter, InputsImpl, InterpreterResult},
|
||||
interpreter::interpreter::EthInterpreter,
|
||||
precompile::{PrecompileFn, PrecompileOutput, PrecompileResult, Precompiles},
|
||||
primitives::hardfork::SpecId,
|
||||
MainBuilder, MainContext,
|
||||
@@ -46,20 +43,26 @@ pub struct MyEvmFactory;
|
||||
|
||||
impl EvmFactory for MyEvmFactory {
|
||||
type Evm<DB: Database, I: Inspector<EthEvmContext<DB>, EthInterpreter>> =
|
||||
EthEvm<DB, I, CustomPrecompiles>;
|
||||
EthEvm<DB, I, Self::Precompiles>;
|
||||
type Tx = TxEnv;
|
||||
type Error<DBError: core::error::Error + Send + Sync + 'static> = EVMError<DBError>;
|
||||
type HaltReason = HaltReason;
|
||||
type Context<DB: Database> = EthEvmContext<DB>;
|
||||
type Spec = SpecId;
|
||||
type Precompiles = PrecompilesMap;
|
||||
|
||||
fn create_evm<DB: Database>(&self, db: DB, input: EvmEnv) -> Self::Evm<DB, NoOpInspector> {
|
||||
let evm = Context::mainnet()
|
||||
let spec = input.cfg_env.spec;
|
||||
let mut evm = Context::mainnet()
|
||||
.with_db(db)
|
||||
.with_cfg(input.cfg_env)
|
||||
.with_block(input.block_env)
|
||||
.build_mainnet_with_inspector(NoOpInspector {})
|
||||
.with_precompiles(CustomPrecompiles::new());
|
||||
.with_precompiles(PrecompilesMap::from_static(EthPrecompiles::default().precompiles));
|
||||
|
||||
if spec == SpecId::PRAGUE {
|
||||
evm = evm.with_precompiles(PrecompilesMap::from_static(prague_custom()));
|
||||
}
|
||||
|
||||
EthEvm::new(evm, false)
|
||||
}
|
||||
@@ -92,20 +95,6 @@ where
|
||||
}
|
||||
}
|
||||
|
||||
/// A custom precompile that contains static precompiles.
|
||||
#[derive(Clone)]
|
||||
pub struct CustomPrecompiles {
|
||||
pub precompiles: EthPrecompiles,
|
||||
}
|
||||
|
||||
impl CustomPrecompiles {
|
||||
/// Given a [`PrecompileProvider`] and cache for a specific precompiles, create a
|
||||
/// wrapper that can be used inside Evm.
|
||||
fn new() -> Self {
|
||||
Self { precompiles: EthPrecompiles::default() }
|
||||
}
|
||||
}
|
||||
|
||||
/// Returns precompiles for Fjor spec.
|
||||
pub fn prague_custom() -> &'static Precompiles {
|
||||
static INSTANCE: OnceLock<Precompiles> = OnceLock::new();
|
||||
@@ -123,39 +112,6 @@ pub fn prague_custom() -> &'static Precompiles {
|
||||
})
|
||||
}
|
||||
|
||||
impl<CTX: ContextTr> PrecompileProvider<CTX> for CustomPrecompiles {
|
||||
type Output = InterpreterResult;
|
||||
|
||||
fn set_spec(&mut self, spec: <CTX::Cfg as Cfg>::Spec) -> bool {
|
||||
let spec_id = spec.clone().into();
|
||||
if spec_id == SpecId::PRAGUE {
|
||||
self.precompiles = EthPrecompiles { precompiles: prague_custom(), spec: spec.into() }
|
||||
} else {
|
||||
PrecompileProvider::<CTX>::set_spec(&mut self.precompiles, spec);
|
||||
}
|
||||
true
|
||||
}
|
||||
|
||||
fn run(
|
||||
&mut self,
|
||||
context: &mut CTX,
|
||||
address: &Address,
|
||||
inputs: &InputsImpl,
|
||||
is_static: bool,
|
||||
gas_limit: u64,
|
||||
) -> Result<Option<Self::Output>, String> {
|
||||
self.precompiles.run(context, address, inputs, is_static, gas_limit)
|
||||
}
|
||||
|
||||
fn warm_addresses(&self) -> Box<impl Iterator<Item = Address>> {
|
||||
self.precompiles.warm_addresses()
|
||||
}
|
||||
|
||||
fn contains(&self, address: &Address) -> bool {
|
||||
self.precompiles.contains(address)
|
||||
}
|
||||
}
|
||||
|
||||
#[tokio::main]
|
||||
async fn main() -> eyre::Result<()> {
|
||||
let _guard = RethTracer::new().init()?;
|
||||
|
||||
@@ -5,6 +5,7 @@ use alloy_evm::{
|
||||
BlockExecutionError, BlockExecutionResult, BlockExecutor, BlockExecutorFactory,
|
||||
BlockExecutorFor, ExecutableTx, OnStateHook,
|
||||
},
|
||||
precompiles::PrecompilesMap,
|
||||
Database, Evm, EvmEnv,
|
||||
};
|
||||
use alloy_op_evm::{OpBlockExecutionCtx, OpBlockExecutor, OpEvm};
|
||||
@@ -116,7 +117,7 @@ impl BlockExecutorFactory for CustomEvmConfig {
|
||||
|
||||
fn create_executor<'a, DB, I>(
|
||||
&'a self,
|
||||
evm: OpEvm<&'a mut State<DB>, I>,
|
||||
evm: OpEvm<&'a mut State<DB>, I, PrecompilesMap>,
|
||||
ctx: OpBlockExecutionCtx,
|
||||
) -> impl BlockExecutorFor<'a, Self, DB, I>
|
||||
where
|
||||
|
||||
@@ -2,12 +2,17 @@
|
||||
|
||||
#![warn(unused_crate_dependencies)]
|
||||
|
||||
use alloy_evm::{eth::EthEvmContext, EvmFactory};
|
||||
use alloy_evm::{
|
||||
eth::EthEvmContext,
|
||||
precompiles::{DynPrecompile, Precompile, PrecompilesMap},
|
||||
Evm, EvmFactory,
|
||||
};
|
||||
use alloy_genesis::Genesis;
|
||||
use alloy_primitives::{Address, Bytes};
|
||||
use alloy_primitives::Bytes;
|
||||
use parking_lot::RwLock;
|
||||
use reth::{
|
||||
builder::{components::ExecutorBuilder, BuilderContext, NodeBuilder},
|
||||
revm::precompile::PrecompileResult,
|
||||
tasks::TaskManager,
|
||||
};
|
||||
use reth_ethereum::{
|
||||
@@ -15,14 +20,11 @@ use reth_ethereum::{
|
||||
evm::{
|
||||
primitives::{Database, EvmEnv},
|
||||
revm::{
|
||||
context::{Cfg, Context, TxEnv},
|
||||
context_interface::{
|
||||
result::{EVMError, HaltReason},
|
||||
ContextTr,
|
||||
},
|
||||
handler::{EthPrecompiles, PrecompileProvider},
|
||||
context::{Context, TxEnv},
|
||||
context_interface::result::{EVMError, HaltReason},
|
||||
handler::EthPrecompiles,
|
||||
inspector::{Inspector, NoOpInspector},
|
||||
interpreter::{interpreter::EthInterpreter, InputsImpl, InterpreterResult},
|
||||
interpreter::interpreter::EthInterpreter,
|
||||
primitives::hardfork::SpecId,
|
||||
MainBuilder, MainContext,
|
||||
},
|
||||
@@ -38,12 +40,10 @@ use reth_ethereum::{
|
||||
};
|
||||
use reth_tracing::{RethTracer, Tracer};
|
||||
use schnellru::{ByLength, LruMap};
|
||||
use std::{collections::HashMap, sync::Arc};
|
||||
use std::sync::Arc;
|
||||
|
||||
/// Type alias for the LRU cache used within the [`PrecompileCache`].
|
||||
type PrecompileLRUCache = LruMap<(SpecId, Bytes, u64), Result<InterpreterResult, String>>;
|
||||
|
||||
type WrappedEthEvm<DB, I> = EthEvm<DB, I, WrappedPrecompile<EthPrecompiles>>;
|
||||
type PrecompileLRUCache = LruMap<(Bytes, u64), PrecompileResult>;
|
||||
|
||||
/// A cache for precompile inputs / outputs.
|
||||
///
|
||||
@@ -52,26 +52,28 @@ type WrappedEthEvm<DB, I> = EthEvm<DB, I, WrappedPrecompile<EthPrecompiles>>;
|
||||
///
|
||||
/// NOTE: This does not work with "context stateful precompiles", ie `ContextStatefulPrecompile` or
|
||||
/// `ContextStatefulPrecompileMut`. They are explicitly banned.
|
||||
#[derive(Debug, Default)]
|
||||
#[derive(Debug)]
|
||||
pub struct PrecompileCache {
|
||||
/// Caches for each precompile input / output.
|
||||
cache: HashMap<Address, PrecompileLRUCache>,
|
||||
cache: PrecompileLRUCache,
|
||||
}
|
||||
|
||||
/// Custom EVM factory.
|
||||
#[derive(Debug, Clone, Default)]
|
||||
#[derive(Debug, Clone)]
|
||||
#[non_exhaustive]
|
||||
pub struct MyEvmFactory {
|
||||
precompile_cache: Arc<RwLock<PrecompileCache>>,
|
||||
}
|
||||
|
||||
impl EvmFactory for MyEvmFactory {
|
||||
type Evm<DB: Database, I: Inspector<EthEvmContext<DB>, EthInterpreter>> = WrappedEthEvm<DB, I>;
|
||||
type Evm<DB: Database, I: Inspector<EthEvmContext<DB>, EthInterpreter>> =
|
||||
EthEvm<DB, I, PrecompilesMap>;
|
||||
type Tx = TxEnv;
|
||||
type Error<DBError: core::error::Error + Send + Sync + 'static> = EVMError<DBError>;
|
||||
type HaltReason = HaltReason;
|
||||
type Context<DB: Database> = EthEvmContext<DB>;
|
||||
type Spec = SpecId;
|
||||
type Precompiles = PrecompilesMap;
|
||||
|
||||
fn create_evm<DB: Database>(&self, db: DB, input: EvmEnv) -> Self::Evm<DB, NoOpInspector> {
|
||||
let new_cache = self.precompile_cache.clone();
|
||||
@@ -81,9 +83,15 @@ impl EvmFactory for MyEvmFactory {
|
||||
.with_cfg(input.cfg_env)
|
||||
.with_block(input.block_env)
|
||||
.build_mainnet_with_inspector(NoOpInspector {})
|
||||
.with_precompiles(WrappedPrecompile::new(EthPrecompiles::default(), new_cache));
|
||||
.with_precompiles(PrecompilesMap::from_static(EthPrecompiles::default().precompiles));
|
||||
|
||||
EthEvm::new(evm, false)
|
||||
let mut evm = EthEvm::new(evm, false);
|
||||
|
||||
evm.precompiles_mut().map_precompiles(|_, precompile| {
|
||||
WrappedPrecompile::wrap(precompile, new_cache.clone())
|
||||
});
|
||||
|
||||
evm
|
||||
}
|
||||
|
||||
fn create_evm_with_inspector<DB: Database, I: Inspector<Self::Context<DB>, EthInterpreter>>(
|
||||
@@ -98,84 +106,64 @@ impl EvmFactory for MyEvmFactory {
|
||||
|
||||
/// A custom precompile that contains the cache and precompile it wraps.
|
||||
#[derive(Clone)]
|
||||
pub struct WrappedPrecompile<P> {
|
||||
pub struct WrappedPrecompile {
|
||||
/// The precompile to wrap.
|
||||
precompile: P,
|
||||
precompile: DynPrecompile,
|
||||
/// The cache to use.
|
||||
cache: Arc<RwLock<PrecompileCache>>,
|
||||
/// The spec id to use.
|
||||
spec: SpecId,
|
||||
}
|
||||
|
||||
impl<P> WrappedPrecompile<P> {
|
||||
/// Given a [`PrecompileProvider`] and cache for a specific precompiles, create a
|
||||
impl WrappedPrecompile {
|
||||
fn new(precompile: DynPrecompile, cache: Arc<RwLock<PrecompileCache>>) -> Self {
|
||||
Self { precompile, cache }
|
||||
}
|
||||
|
||||
/// Given a [`DynPrecompile`] and cache for a specific precompiles, create a
|
||||
/// wrapper that can be used inside Evm.
|
||||
fn new(precompile: P, cache: Arc<RwLock<PrecompileCache>>) -> Self {
|
||||
WrappedPrecompile { precompile, cache: cache.clone(), spec: SpecId::default() }
|
||||
fn wrap(precompile: DynPrecompile, cache: Arc<RwLock<PrecompileCache>>) -> DynPrecompile {
|
||||
let wrapped = Self::new(precompile, cache);
|
||||
move |data: &[u8], gas_limit: u64| -> PrecompileResult { wrapped.call(data, gas_limit) }
|
||||
.into()
|
||||
}
|
||||
}
|
||||
|
||||
impl<CTX: ContextTr, P: PrecompileProvider<CTX, Output = InterpreterResult>> PrecompileProvider<CTX>
|
||||
for WrappedPrecompile<P>
|
||||
{
|
||||
type Output = P::Output;
|
||||
|
||||
fn set_spec(&mut self, spec: <CTX::Cfg as Cfg>::Spec) -> bool {
|
||||
self.precompile.set_spec(spec.clone());
|
||||
self.spec = spec.into();
|
||||
true
|
||||
}
|
||||
|
||||
fn run(
|
||||
&mut self,
|
||||
context: &mut CTX,
|
||||
address: &Address,
|
||||
inputs: &InputsImpl,
|
||||
is_static: bool,
|
||||
gas_limit: u64,
|
||||
) -> Result<Option<Self::Output>, String> {
|
||||
impl Precompile for WrappedPrecompile {
|
||||
fn call(&self, data: &[u8], gas: u64) -> PrecompileResult {
|
||||
let mut cache = self.cache.write();
|
||||
let key = (self.spec, inputs.input.clone(), gas_limit);
|
||||
let key = (Bytes::copy_from_slice(data), gas);
|
||||
|
||||
// get the result if it exists
|
||||
if let Some(precompiles) = cache.cache.get_mut(address) {
|
||||
if let Some(result) = precompiles.get(&key) {
|
||||
return result.clone().map(Some)
|
||||
}
|
||||
if let Some(result) = cache.cache.get(&key) {
|
||||
return result.clone()
|
||||
}
|
||||
|
||||
// call the precompile if cache miss
|
||||
let output = self.precompile.run(context, address, inputs, is_static, gas_limit);
|
||||
let output = self.precompile.call(data, gas);
|
||||
|
||||
if let Some(output) = output.clone().transpose() {
|
||||
// insert the result into the cache
|
||||
cache
|
||||
.cache
|
||||
.entry(*address)
|
||||
.or_insert(PrecompileLRUCache::new(ByLength::new(1024)))
|
||||
.insert(key, output);
|
||||
}
|
||||
// insert the result into the cache
|
||||
cache.cache.insert(key, output.clone());
|
||||
|
||||
output
|
||||
}
|
||||
|
||||
fn warm_addresses(&self) -> Box<impl Iterator<Item = Address>> {
|
||||
self.precompile.warm_addresses()
|
||||
}
|
||||
|
||||
fn contains(&self, address: &Address) -> bool {
|
||||
self.precompile.contains(address)
|
||||
}
|
||||
}
|
||||
|
||||
/// Builds a regular ethereum block executor that uses the custom EVM.
|
||||
#[derive(Debug, Default, Clone)]
|
||||
#[derive(Debug, Clone)]
|
||||
#[non_exhaustive]
|
||||
pub struct MyExecutorBuilder {
|
||||
/// The precompile cache to use for all executors.
|
||||
precompile_cache: Arc<RwLock<PrecompileCache>>,
|
||||
}
|
||||
|
||||
impl Default for MyExecutorBuilder {
|
||||
fn default() -> Self {
|
||||
let precompile_cache = PrecompileCache {
|
||||
cache: LruMap::<(Bytes, u64), PrecompileResult>::new(ByLength::new(100)),
|
||||
};
|
||||
Self { precompile_cache: Arc::new(RwLock::new(precompile_cache)) }
|
||||
}
|
||||
}
|
||||
|
||||
impl<Node> ExecutorBuilder<Node> for MyExecutorBuilder
|
||||
where
|
||||
Node: FullNodeTypes<Types: NodeTypes<ChainSpec = ChainSpec, Primitives = EthPrimitives>>,
|
||||
|
||||
Reference in New Issue
Block a user