diff --git a/aggregator/Cargo.toml b/aggregator/Cargo.toml index 8f0535458b..5006aced73 100644 --- a/aggregator/Cargo.toml +++ b/aggregator/Cargo.toml @@ -46,5 +46,3 @@ csv = "1.1" [features] default = ["revm-precompile/c-kzg"] print-trace = ["ark-std/print-trace"] -# This feature is useful for unit tests where we check the SAT of pi batch circuit -disable_proof_aggregation = [] diff --git a/aggregator/src/aggregation/circuit.rs b/aggregator/src/aggregation/circuit.rs index 47d03c41eb..418d27dfe0 100644 --- a/aggregator/src/aggregation/circuit.rs +++ b/aggregator/src/aggregation/circuit.rs @@ -1,16 +1,4 @@ -use crate::{ - aggregation::decoder::WORKED_EXAMPLE, - blob::BatchData, - witgen::{zstd_encode, MultiBlockProcessResult}, - LOG_DEGREE, PI_CHAIN_ID, PI_CURRENT_BATCH_HASH, PI_CURRENT_STATE_ROOT, - PI_CURRENT_WITHDRAW_ROOT, PI_PARENT_BATCH_HASH, PI_PARENT_STATE_ROOT, -}; use ark_std::{end_timer, start_timer}; -use halo2_base::{Context, ContextParams}; - -#[cfg(not(feature = "disable_proof_aggregation"))] -use halo2_ecc::{ecc::EccChip, fields::fp::FpConfig}; - use halo2_proofs::{ circuit::{Layouter, SimpleFloorPlanner, Value}, halo2curves::bn256::{Bn256, Fr, G1Affine}, @@ -19,34 +7,34 @@ use halo2_proofs::{ }; use itertools::Itertools; use rand::Rng; -#[cfg(not(feature = "disable_proof_aggregation"))] -use std::rc::Rc; -use std::{env, fs::File}; - -#[cfg(not(feature = "disable_proof_aggregation"))] -use snark_verifier::loader::halo2::{halo2_ecc::halo2_base::AssignedValue, Halo2Loader}; -use snark_verifier::pcs::kzg::KzgSuccinctVerifyingKey; -#[cfg(not(feature = "disable_proof_aggregation"))] use snark_verifier::{ - loader::halo2::halo2_ecc::halo2_base, - pcs::kzg::{Bdfg21, Kzg}, + loader::halo2::{ + halo2_ecc::{ + ecc::EccChip, + fields::fp::FpConfig, + halo2_base::{AssignedValue, Context, ContextParams}, + }, + Halo2Loader, + }, + pcs::kzg::{Bdfg21, Kzg, KzgSuccinctVerifyingKey}, }; -#[cfg(not(feature = "disable_proof_aggregation"))] -use snark_verifier_sdk::{aggregate, flatten_accumulator}; -use snark_verifier_sdk::{CircuitExt, Snark, SnarkWitness}; +use snark_verifier_sdk::{aggregate, flatten_accumulator, CircuitExt, Snark, SnarkWitness}; +use std::{env, fs::File, rc::Rc}; use zkevm_circuits::util::Challenges; use crate::{ - aggregation::witgen::process, + aggregation::{decoder::WORKED_EXAMPLE, witgen::process, BatchCircuitConfig}, batch::BatchHash, + blob::BatchData, constants::{ACC_LEN, DIGEST_LEN}, core::{assign_batch_hashes, extract_proof_and_instances_with_pairing_check}, util::parse_hash_digest_cells, - AssignedBarycentricEvaluationConfig, ConfigParams, + witgen::{zstd_encode, MultiBlockProcessResult}, + AssignedBarycentricEvaluationConfig, ConfigParams, LOG_DEGREE, PI_CHAIN_ID, + PI_CURRENT_BATCH_HASH, PI_CURRENT_STATE_ROOT, PI_CURRENT_WITHDRAW_ROOT, PI_PARENT_BATCH_HASH, + PI_PARENT_STATE_ROOT, }; -use super::BatchCircuitConfig; - /// Batch circuit, the chunk aggregation routine below recursion circuit #[derive(Clone)] pub struct BatchCircuit { @@ -186,47 +174,10 @@ impl Circuit for BatchCircuit { .range() .load_lookup_table(&mut layouter) .expect("load range lookup table"); + // ============================================== // Step 1: snark aggregation circuit // ============================================== - #[cfg(feature = "disable_proof_aggregation")] - let barycentric = { - let mut first_pass = halo2_base::SKIP_FIRST_PASS; - layouter.assign_region( - || "barycentric evaluation", - |region| { - if first_pass { - first_pass = false; - return Ok(AssignedBarycentricEvaluationConfig::default()); - } - - let mut ctx = Context::new( - region, - ContextParams { - max_rows: config.flex_gate().max_rows, - num_context_ids: 1, - fixed_columns: config.flex_gate().constants.clone(), - }, - ); - - let barycentric = config.barycentric.assign( - &mut ctx, - &self.batch_hash.point_evaluation_assignments.coefficients, - self.batch_hash - .point_evaluation_assignments - .challenge_digest, - self.batch_hash.point_evaluation_assignments.evaluation, - ); - - config.barycentric.scalar.range.finalize(&mut ctx); - ctx.print_stats(&["barycentric evaluation"]); - - Ok(barycentric) - }, - )? - }; - - #[cfg(not(feature = "disable_proof_aggregation"))] let (accumulator_instances, snark_inputs, barycentric) = { use halo2_proofs::halo2curves::bn256::Fq; let mut first_pass = halo2_base::SKIP_FIRST_PASS; @@ -375,21 +326,13 @@ impl Circuit for BatchCircuit { }; // Extract digests - #[cfg(feature = "disable_proof_aggregation")] - let (_batch_hash_digest, _chunk_pi_hash_digests, _potential_batch_data_hash_digest) = - parse_hash_digest_cells::(&assigned_batch_hash.hash_output); - - #[cfg(not(feature = "disable_proof_aggregation"))] let (_batch_hash_digest, chunk_pi_hash_digests, _potential_batch_data_hash_digest) = parse_hash_digest_cells::(&assigned_batch_hash.hash_output); // ======================================================================== // step 2.a: check accumulator including public inputs to the snarks // ======================================================================== - #[cfg(not(feature = "disable_proof_aggregation"))] let mut first_pass = halo2_base::SKIP_FIRST_PASS; - - #[cfg(not(feature = "disable_proof_aggregation"))] layouter.assign_region( || "BatchCircuit: Chunk PI", |mut region| -> Result<(), Error> { @@ -424,7 +367,6 @@ impl Circuit for BatchCircuit { }, )?; - #[cfg(not(feature = "disable_proof_aggregation"))] { assert!(accumulator_instances.len() == ACC_LEN); for (i, v) in accumulator_instances.iter().enumerate() { diff --git a/aggregator/src/tests/aggregation.rs b/aggregator/src/tests/aggregation.rs index 345dc8de90..cfac56a830 100644 --- a/aggregator/src/tests/aggregation.rs +++ b/aggregator/src/tests/aggregation.rs @@ -16,19 +16,24 @@ use crate::{ BatchData, ChunkInfo, }; -// See https://github.com/scroll-tech/zkevm-circuits/pull/1311#issuecomment-2139559866 #[test] -fn test_max_agg_snarks_batch_circuit() { +fn batch_circuit_raw() { let k = 21; - - // This set up requires one round of keccak for chunk's data hash - // let circuit: BatchCircuit = build_new_batch_circuit(2, k); let circuit: BatchCircuit = build_batch_circuit_skip_encoding(); let instance = circuit.instances(); let mock_prover = MockProver::::run(k, &circuit, instance).unwrap(); mock_prover.assert_satisfied_par(); } +#[test] +fn batch_circuit_encode() { + let k = 21; + let circuit: BatchCircuit = build_new_batch_circuit(2, k); + let instance = circuit.instances(); + let mock_prover = MockProver::::run(k, &circuit, instance).unwrap(); + mock_prover.assert_satisfied_par(); +} + #[ignore] #[test] fn test_2_snark_batch_circuit() { @@ -143,6 +148,19 @@ fn build_new_batch_circuit( ) -> BatchCircuit { // inner circuit: Mock circuit let k0 = 8; + #[derive(Clone, Debug, serde::Deserialize, serde::Serialize)] + pub struct ChunkProof { + pub chunk_info: ChunkInfo, + } + #[derive(Debug, Clone, serde::Deserialize, serde::Serialize)] + struct BatchProvingTask { + pub chunk_proofs: Vec, + pub batch_header: BatchHeader, + } + let file = std::fs::File::open("data/batch-task.json").expect("batch-task.json exists"); + let reader = std::io::BufReader::new(file); + let batch_proving_task: BatchProvingTask = + serde_json::from_reader(reader).expect("deserialisation should succeed"); let mut rng = test_rng(); let params = gen_srs(k0); @@ -163,6 +181,16 @@ fn build_new_batch_circuit( let batch_data = BatchData::::new(num_real_chunks, &chunks_with_padding); let batch_bytes = batch_data.get_batch_data_bytes(); let blob_bytes = get_blob_bytes(&batch_bytes); + let batch_header = BatchHeader::construct_from_chunks( + batch_proving_task.batch_header.version, + batch_proving_task.batch_header.batch_index, + batch_proving_task.batch_header.l1_message_popped, + batch_proving_task.batch_header.total_l1_message_popped, + batch_proving_task.batch_header.parent_batch_hash, + batch_proving_task.batch_header.last_block_timestamp, + &chunks_with_padding, + &blob_bytes, + ); // ========================== // real chunks @@ -190,8 +218,7 @@ fn build_new_batch_circuit( // ========================== // batch // ========================== - let batch_hash = - BatchHash::construct(&chunks_with_padding, BatchHeader::default(), &blob_bytes); + let batch_hash = BatchHash::construct(&chunks_with_padding, batch_header, &blob_bytes); BatchCircuit::new( ¶ms, diff --git a/zkevm-circuits/src/evm_circuit/execution/begin_tx.rs b/zkevm-circuits/src/evm_circuit/execution/begin_tx.rs index d2338d78cf..e39dd0a1fa 100644 --- a/zkevm-circuits/src/evm_circuit/execution/begin_tx.rs +++ b/zkevm-circuits/src/evm_circuit/execution/begin_tx.rs @@ -6,8 +6,8 @@ use crate::{ util::{ and, common_gadget::{ - CurieGadget, TransferGadgetInfo, TransferWithGasFeeGadget, TxAccessListGadget, - TxEip1559Gadget, TxL1FeeGadget, TxL1MsgGadget, + TransferGadgetInfo, TransferWithGasFeeGadget, TxAccessListGadget, TxEip1559Gadget, + TxL1FeeGadget, TxL1MsgGadget, }, constraint_builder::{ ConstrainBuilderCommon, EVMConstraintBuilder, ReversionInfo, StepStateTransition, @@ -100,7 +100,6 @@ pub(crate) struct BeginTxGadget { tx_l1_msg: TxL1MsgGadget, tx_access_list: TxAccessListGadget, tx_eip1559: TxEip1559Gadget, - curie: CurieGadget, } impl ExecutionGadget for BeginTxGadget { @@ -140,8 +139,6 @@ impl ExecutionGadget for BeginTxGadget { let tx_access_list = TxAccessListGadget::construct(cb, tx_id.expr(), tx_type.expr()); let is_call_data_empty = IsZeroGadget::construct(cb, tx_call_data_length.expr()); - let curie = CurieGadget::construct(cb, cb.curr.state.block_number.expr()); - let tx_l1_msg = TxL1MsgGadget::construct(cb, tx_type.expr(), tx_caller_address.expr()); let tx_l1_fee = cb.condition(not::expr(tx_l1_msg.is_l1_msg()), |cb| { cb.require_equal( @@ -149,13 +146,7 @@ impl ExecutionGadget for BeginTxGadget { tx_nonce.expr(), sender_nonce.expr(), ); - TxL1FeeGadget::construct( - cb, - not::expr(curie.is_before_curie.expr()), - tx_id.expr(), - tx_data_gas_cost.expr(), - tx_signed_length.expr(), - ) + TxL1FeeGadget::construct(cb, tx_id.expr(), tx_signed_length.expr()) }); cb.condition(tx_l1_msg.is_l1_msg(), |cb| { cb.require_zero("l1fee is 0 for l1msg", tx_data_gas_cost.expr()); @@ -170,7 +161,7 @@ impl ExecutionGadget for BeginTxGadget { let l1_rw_delta = select::expr( tx_l1_msg.is_l1_msg(), tx_l1_msg.rw_delta(), - tx_l1_fee.rw_delta(not::expr(curie.is_before_curie.expr())), + tx_l1_fee.rw_delta(), ) + 1.expr(); // the cost caused by l1 @@ -845,7 +836,6 @@ impl ExecutionGadget for BeginTxGadget { tx_l1_msg, tx_access_list, tx_eip1559, - curie, } } @@ -897,10 +887,6 @@ impl ExecutionGadget for BeginTxGadget { self.tx_l1_msg .assign(region, offset, tx_type, caller_code_hash)?; - let is_curie = bus_mapping::circuit_input_builder::curie::is_curie_enabled( - block.chain_id, - tx.block_number, - ); // Add access-list RW offset. rws.offset_add(TxAccessListGadget::::rw_delta_value(tx) as usize); @@ -920,16 +906,9 @@ impl ExecutionGadget for BeginTxGadget { 0 } } else { - if is_curie { - 6 - } else { - 3 - } + 6 }); - self.curie - .assign(region, offset, block.chain_id, tx.block_number)?; - let rw = rws.next(); debug_assert_eq!(rw.tag(), RwTableTag::CallContext); debug_assert_eq!(rw.field_tag(), Some(CallContextFieldTag::L1Fee as u64)); diff --git a/zkevm-circuits/src/evm_circuit/execution/end_block.rs b/zkevm-circuits/src/evm_circuit/execution/end_block.rs index 358e2aa132..618824be5b 100644 --- a/zkevm-circuits/src/evm_circuit/execution/end_block.rs +++ b/zkevm-circuits/src/evm_circuit/execution/end_block.rs @@ -1,10 +1,17 @@ +use bus_mapping::l2_predeployed::message_queue::{ + ADDRESS as MESSAGE_QUEUE, WITHDRAW_TRIE_ROOT_SLOT, +}; +use gadgets::ToScalar; +use halo2_proofs::{ + circuit::{Cell as AssignedCell, Value}, + plonk::{Error, Expression}, +}; use std::sync::Mutex; use crate::{ evm_circuit::{ execution::ExecutionGadget, step::ExecutionState, - table::{FixedTableTag, Lookup}, util::{ constraint_builder::{ ConstrainBuilderCommon, EVMConstraintBuilder, StepStateTransition, Transition::Same, @@ -14,21 +21,9 @@ use crate::{ }, witness::{Block, Call, ExecStep, Transaction}, }, - table::{AccountFieldTag, BlockContextFieldTag, CallContextFieldTag, TxContextFieldTag}, + table::{BlockContextFieldTag, CallContextFieldTag, TxContextFieldTag}, util::{Expr, Field}, }; -use bus_mapping::l2_predeployed::message_queue::{ - ADDRESS as MESSAGE_QUEUE, WITHDRAW_TRIE_ROOT_SLOT, -}; -use eth_types::{ - forks::HardforkId, - utils::{hash_code, hash_code_keccak}, -}; -use gadgets::ToScalar; -use halo2_proofs::{ - circuit::{Cell as AssignedCell, Value}, - plonk::{Error, Expression}, -}; #[derive(Debug)] pub(crate) struct EndBlockGadget { @@ -40,7 +35,6 @@ pub(crate) struct EndBlockGadget { max_txs: Cell, phase2_withdraw_root: Cell, phase2_withdraw_root_prev: Cell, - is_curie_fork_block: Cell, pub withdraw_root_assigned: Mutex>, } @@ -58,24 +52,21 @@ impl Clone for EndBlockGadget { max_txs: self.max_txs.clone(), phase2_withdraw_root: self.phase2_withdraw_root.clone(), phase2_withdraw_root_prev: self.phase2_withdraw_root_prev.clone(), - is_curie_fork_block: self.is_curie_fork_block.clone(), } } } -/* -The goal of EndBlockGadget is to: - 0. expose withdraw root. Then it can be copied into pi circuit. - 1. constrain rws of evm circuit is same to rws of state circuit. - We use 2 StartOp rw lookup to do this. - 2. constrain all txs inside tx circuit are processed inside evm circuit. - (We don't need to constrain txs in evm circuit are not in tx circuit, - since there are tx lookups) -To achieve the above goal: - We need to pass "rwc" and "call_id" all the way to EndBlock. - Then "rwc" can be used for goal1. - For goal2 this gadget can read tx_id from CallContext using call_id. - */ +/// The goal of EndBlockGadget is to: +/// 0. expose withdraw root. Then it can be copied into pi circuit. +/// 1. constrain rws of evm circuit is same to rws of state circuit. +/// We use 2 StartOp rw lookup to do this. +/// 2. constrain all txs inside tx circuit are processed inside evm circuit. +/// (We don't need to constrain txs in evm circuit are not in tx circuit, +/// since there are tx lookups) +/// To achieve the above goal: +/// We need to pass "rwc" and "call_id" all the way to EndBlock. +/// Then "rwc" can be used for goal1. +/// For goal2 this gadget can read tx_id from CallContext using call_id. impl ExecutionGadget for EndBlockGadget { const NAME: &'static str = "EndBlock"; @@ -97,79 +88,6 @@ impl ExecutionGadget for EndBlockGadget { chain_id.expr(), ); - let is_curie_fork_block = cb.query_cell(); - // Sequencer is allowed to do a predefined state transition at the hardfork block. - cb.condition(is_curie_fork_block.expr(), |cb| { - cb.add_lookup( - "Hardfork lookup", - Lookup::Fixed { - tag: FixedTableTag::ChainFork.expr(), - values: [ - (HardforkId::Curie as u64).expr(), - chain_id.expr(), - cb.curr.state.block_number.expr(), - ]}, - ); - // Ref: bus-mapping/src/circuit_input_builder/curie.rs - // Bytecode changes - use bus_mapping::l2_predeployed::l1_gas_price_oracle; - let v1_codesize = l1_gas_price_oracle::V1_BYTECODE.len(); - let v1_codehash = hash_code(&l1_gas_price_oracle::V1_BYTECODE); - let v1_keccak_codehash = hash_code_keccak(&l1_gas_price_oracle::V1_BYTECODE); - log::debug!("l1_oracle poseidon codehash {:?}", v1_codehash); - log::debug!("l1_oracle keccak codehash {:?}", v1_keccak_codehash); - let v2_codesize = l1_gas_price_oracle::V2_BYTECODE.len(); - let v2_codehash = hash_code(&l1_gas_price_oracle::V2_BYTECODE); - let v2_keccak_codehash = hash_code_keccak(&l1_gas_price_oracle::V2_BYTECODE); - let l1_fee_address = Expression::Constant(l1_gas_price_oracle::ADDRESS.to_scalar().expect( - "Unexpected address of l2 gasprice oracle contract -> Scalar conversion failure", - )); - cb.account_write( - l1_fee_address.expr(), - AccountFieldTag::CodeHash, - cb.code_hash(v2_codehash), - cb.code_hash(v1_codehash), - None, - ); - cb.account_write( - l1_fee_address.expr(), - AccountFieldTag::KeccakCodeHash, - cb.keccak_code_hash(v2_keccak_codehash), - cb.keccak_code_hash(v1_keccak_codehash), - None, - ); - cb.account_write( - l1_fee_address.expr(), - AccountFieldTag::CodeSize, - Expression::Constant(F::from(v2_codesize as u64)), - Expression::Constant(F::from(v1_codesize as u64)), - None, - ); - // State changes - for (slot, value) in [ - (*l1_gas_price_oracle::IS_CURIE_SLOT, eth_types::Word::from(1u64)), - (*l1_gas_price_oracle::L1_BLOB_BASEFEE_SLOT, eth_types::Word::from(1u64)), - ( - *l1_gas_price_oracle::COMMIT_SCALAR_SLOT, - *l1_gas_price_oracle::INITIAL_COMMIT_SCALAR, - ), - ( - *l1_gas_price_oracle::BLOB_SCALAR_SLOT, - *l1_gas_price_oracle::INITIAL_BLOB_SCALAR, - ), - ] { - cb.account_storage_write( - l1_fee_address.expr(), - cb.word_rlc_constant(slot), - cb.word_rlc_constant(value), - 0.expr(), - 0.expr(), - 0.expr(), - None, - ); - } - }); - // Note that rw_counter starts at 1 let is_empty_block = IsZeroGadget::construct(cb, cb.curr.state.rw_counter.clone().expr() - 1.expr()); @@ -178,8 +96,7 @@ impl ExecutionGadget for EndBlockGadget { // and add 1 withdraw_root lookup let total_rws = not::expr(is_empty_block.expr()) * (cb.curr.state.rw_counter.clone().expr() - 1.expr() + 1.expr()) - + 1.expr() - + is_curie_fork_block.expr() * 7.expr(); + + 1.expr(); // 1. Constraint total_rws and total_txs witness values depending on the empty // block case. @@ -255,7 +172,6 @@ impl ExecutionGadget for EndBlockGadget { total_txs, total_txs_is_max_txs, is_empty_block, - is_curie_fork_block, withdraw_root_assigned: Default::default(), } } @@ -284,19 +200,6 @@ impl ExecutionGadget for EndBlockGadget { .assign(region, offset, total_txs, max_txs)?; let max_txs_assigned = self.max_txs.assign(region, offset, Value::known(max_txs))?; - let last_block_number = block - .context - .ctxs - .last_key_value() - .map(|(_, b)| b.number) - .unwrap_or_default(); - let is_curie = bus_mapping::circuit_input_builder::curie::is_curie_fork_block( - block.chain_id, - last_block_number.as_u64(), - ); - self.is_curie_fork_block - .assign(region, offset, Value::known(F::from(is_curie as u64)))?; - let withdraw_root = self.phase2_withdraw_root.assign( region, offset, diff --git a/zkevm-circuits/src/evm_circuit/util/common_gadget.rs b/zkevm-circuits/src/evm_circuit/util/common_gadget.rs index be35a54a9a..4406b0141e 100644 --- a/zkevm-circuits/src/evm_circuit/util/common_gadget.rs +++ b/zkevm-circuits/src/evm_circuit/util/common_gadget.rs @@ -32,13 +32,11 @@ use halo2_proofs::{ plonk::{Error, Expression}, }; -mod curie; mod tx_access_list; mod tx_eip1559; mod tx_l1_fee; mod tx_l1_msg; -pub(crate) use curie::CurieGadget; pub(crate) use tx_access_list::TxAccessListGadget; pub(crate) use tx_eip1559::TxEip1559Gadget; pub(crate) use tx_l1_fee::TxL1FeeGadget; diff --git a/zkevm-circuits/src/evm_circuit/util/common_gadget/curie.rs b/zkevm-circuits/src/evm_circuit/util/common_gadget/curie.rs deleted file mode 100644 index 6864ef6975..0000000000 --- a/zkevm-circuits/src/evm_circuit/util/common_gadget/curie.rs +++ /dev/null @@ -1,118 +0,0 @@ -use super::{CachedRegion, Cell}; -use crate::{ - evm_circuit::{ - table::{FixedTableTag, Lookup}, - util::{ - constraint_builder::{ConstrainBuilderCommon, EVMConstraintBuilder}, - math_gadget::{IsZeroGadget, LtGadget}, - }, - }, - table::BlockContextFieldTag, - util::{Expr, Field}, -}; - -use eth_types::forks::{ - HardforkId, SCROLL_DEVNET_CHAIN_ID, SCROLL_MAINNET_CHAIN_ID, SCROLL_TESTNET_CHAIN_ID, -}; -use gadgets::util::not; -use halo2_proofs::{ - circuit::Value, - plonk::{Error, Expression}, -}; - -#[derive(Clone, Debug)] -pub(crate) struct CurieGadget { - chain_id: Cell, - /// Scroll chains have non-zero curie hard fork block number - is_scroll_chain: IsZeroGadget, - /// The block height at which curie hard fork happens - curie_fork_block_num: Cell, - pub(crate) is_before_curie: LtGadget, // block num is u64 -} - -impl CurieGadget { - pub(crate) fn construct(cb: &mut EVMConstraintBuilder, block_number: Expression) -> Self { - let chain_id = cb.query_cell(); - // Lookup block table with chain_id - cb.block_lookup( - BlockContextFieldTag::ChainId.expr(), - block_number.expr(), - chain_id.expr(), - ); - - // TODO: refactor - // is_scroll_chain means (chain_id - 534352) * (chain_id - 222222) == 0 - let is_scroll_chain = IsZeroGadget::construct( - cb, - (chain_id.expr() - SCROLL_MAINNET_CHAIN_ID.expr()) - * (chain_id.expr() - SCROLL_DEVNET_CHAIN_ID.expr()), - ); - - // For Scroll Networks (mainnet, testnet, devnet), - // curie_fork_block_num should be pre-defined. - // For other chain ids, it should be 0. - let curie_fork_block_num = cb.query_cell(); - cb.condition(is_scroll_chain.expr(), |cb| { - cb.add_lookup( - "Hardfork lookup", - Lookup::Fixed { - tag: FixedTableTag::ChainFork.expr(), - values: [ - (HardforkId::Curie as u64).expr(), - chain_id.expr(), - curie_fork_block_num.expr(), - ], - }, - ); - }); - cb.condition(not::expr(is_scroll_chain.expr()), |cb| { - cb.require_zero("enable curie since genesis", curie_fork_block_num.expr()); - }); - - let is_before_curie = LtGadget::construct( - cb, - cb.curr.state.block_number.expr(), - curie_fork_block_num.expr(), - ); - Self { - chain_id, - is_scroll_chain, - curie_fork_block_num, - is_before_curie, - } - } - - pub(crate) fn assign( - &self, - region: &mut CachedRegion<'_, '_, F>, - offset: usize, - chain_id: u64, - block_number: u64, - ) -> Result<(), Error> { - self.chain_id - .assign(region, offset, Value::known(F::from(chain_id)))?; - self.is_scroll_chain.assign( - region, - offset, - (F::from(chain_id) - F::from(SCROLL_MAINNET_CHAIN_ID)) - * (F::from(chain_id) - F::from(SCROLL_DEVNET_CHAIN_ID)), - )?; - let curie_fork_block_num = if chain_id == SCROLL_TESTNET_CHAIN_ID { - 0 - } else { - bus_mapping::circuit_input_builder::curie::get_curie_fork_block(chain_id) - }; - self.curie_fork_block_num.assign( - region, - offset, - Value::known(F::from(curie_fork_block_num)), - )?; - self.is_before_curie.assign( - region, - offset, - F::from(block_number), - F::from(curie_fork_block_num), - )?; - Ok(()) - } -} diff --git a/zkevm-circuits/src/evm_circuit/util/common_gadget/tx_l1_fee.rs b/zkevm-circuits/src/evm_circuit/util/common_gadget/tx_l1_fee.rs index eafc08bc63..091ed1c337 100644 --- a/zkevm-circuits/src/evm_circuit/util/common_gadget/tx_l1_fee.rs +++ b/zkevm-circuits/src/evm_circuit/util/common_gadget/tx_l1_fee.rs @@ -1,4 +1,11 @@ -use super::{CachedRegion, Cell}; +use bus_mapping::{ + circuit_input_builder::{TxL1Fee, TX_L1_FEE_PRECISION}, + l2_predeployed::l1_gas_price_oracle, +}; +use eth_types::{ToLittleEndian, U256}; +use gadgets::ToScalar; +use halo2_proofs::plonk::{Error, Expression}; + use crate::{ evm_circuit::{ param::N_BYTES_U64, @@ -11,14 +18,8 @@ use crate::{ }, util::{Expr, Field}, }; -use bus_mapping::{ - circuit_input_builder::{TxL1Fee, TX_L1_COMMIT_EXTRA_COST, TX_L1_FEE_PRECISION}, - l2_predeployed::l1_gas_price_oracle, -}; -use eth_types::{ToLittleEndian, U256}; -use gadgets::util::not; -use gadgets::ToScalar; -use halo2_proofs::plonk::{Error, Expression}; + +use super::{CachedRegion, Cell}; /// Transaction L1 fee gadget for L1GasPriceOracle contract #[derive(Clone, Debug)] @@ -60,12 +61,10 @@ pub(crate) struct TxL1FeeGadget { impl TxL1FeeGadget { pub(crate) fn construct( cb: &mut EVMConstraintBuilder, - is_curie: Expression, tx_id: Expression, - tx_data_gas_cost: Expression, tx_signed_length: Expression, ) -> Self { - let this = Self::raw_construct(cb, is_curie.expr(), tx_data_gas_cost, tx_signed_length); + let this = Self::raw_construct(cb, tx_signed_length); let l1_fee_address = Expression::Constant(l1_gas_price_oracle::ADDRESS.to_scalar().expect( "Unexpected address of l2 gasprice oracle contract -> Scalar conversion failure", @@ -112,34 +111,33 @@ impl TxL1FeeGadget { this.fee_scalar_committed.expr(), ); - cb.condition(is_curie.expr(), |cb| { - // Read l1blob_basefee_committed - cb.account_storage_read( - l1_fee_address.expr(), - l1_blob_basefee, - this.l1_blob_basefee_word.expr(), - tx_id.clone(), - this.l1_blob_basefee_committed.expr(), - ); + // Read l1blob_basefee_committed + cb.account_storage_read( + l1_fee_address.expr(), + l1_blob_basefee, + this.l1_blob_basefee_word.expr(), + tx_id.clone(), + this.l1_blob_basefee_committed.expr(), + ); - // Read L1 commit_scalar_committed - cb.account_storage_read( - l1_fee_address.expr(), - commit_scalar, - this.commit_scalar_word.expr(), - tx_id.expr(), - this.commit_scalar_committed.expr(), - ); + // Read L1 commit_scalar_committed + cb.account_storage_read( + l1_fee_address.expr(), + commit_scalar, + this.commit_scalar_word.expr(), + tx_id.expr(), + this.commit_scalar_committed.expr(), + ); + + // Read L1 blob_scalar_committed scalar + cb.account_storage_read( + l1_fee_address, + blob_scalar, + this.blob_scalar_word.expr(), + tx_id, + this.blob_scalar_committed.expr(), + ); - // Read L1 blob_scalar_committed scalar - cb.account_storage_read( - l1_fee_address, - blob_scalar, - this.blob_scalar_word.expr(), - tx_id, - this.blob_scalar_committed.expr(), - ); - }); this } @@ -176,8 +174,6 @@ impl TxL1FeeGadget { F::from(remainder), F::from(TX_L1_FEE_PRECISION), )?; - - // curie fields self.l1_blob_basefee_word.assign( region, offset, @@ -187,7 +183,6 @@ impl TxL1FeeGadget { .assign(region, offset, Some(l1_fee.commit_scalar.to_le_bytes()))?; self.blob_scalar_word .assign(region, offset, Some(l1_fee.blob_scalar.to_le_bytes()))?; - self.base_fee_committed.assign( region, offset, @@ -203,8 +198,6 @@ impl TxL1FeeGadget { offset, region.word_rlc(l1_fee_committed.fee_scalar.into()), )?; - - // curie fields self.l1_blob_basefee_committed.assign( region, offset, @@ -224,16 +217,15 @@ impl TxL1FeeGadget { Ok(()) } - pub(crate) fn rw_delta(&self, is_curie: Expression) -> Expression { + pub(crate) fn rw_delta(&self) -> Expression { // L1 base fee Read // L1 fee overhead Read // L1 fee scalar Read - // + curie fields // l1 blob baseFee // commit scalar // blob scalar // TODO: we could optimize the "overhead" and "scalar" for curie - 3.expr() + is_curie.expr() * 3.expr() + 6.expr() } pub(crate) fn tx_l1_fee(&self) -> Expression { @@ -244,31 +236,21 @@ impl TxL1FeeGadget { &self.tx_l1_fee_word } - fn raw_construct( - cb: &mut EVMConstraintBuilder, - is_curie: Expression, - tx_data_gas_cost: Expression, - tx_signed_length: Expression, - ) -> Self { + fn raw_construct(cb: &mut EVMConstraintBuilder, tx_signed_length: Expression) -> Self { let tx_l1_fee_word = cb.query_word_rlc(); let remainder_word = cb.query_word_rlc(); let base_fee_word = cb.query_word_rlc(); let fee_overhead_word = cb.query_word_rlc(); let fee_scalar_word = cb.query_word_rlc(); - // curie fields + let l1_blob_basefee_word = cb.query_word_rlc(); let commit_scalar_word = cb.query_word_rlc(); let blob_scalar_word = cb.query_word_rlc(); let tx_l1_fee = from_bytes::expr(&tx_l1_fee_word.cells[..N_BYTES_U64]); - let [remainder, base_fee, fee_overhead, fee_scalar] = [ - &remainder_word, - &base_fee_word, - &fee_overhead_word, - &fee_scalar_word, - ] - .map(|word| from_bytes::expr(&word.cells[..N_BYTES_U64])); + let [remainder, base_fee] = [&remainder_word, &base_fee_word] + .map(|word| from_bytes::expr(&word.cells[..N_BYTES_U64])); let remainder_range = LtGadget::construct(cb, remainder.expr(), TX_L1_FEE_PRECISION.expr()); cb.require_equal( @@ -284,29 +266,16 @@ impl TxL1FeeGadget { ] .map(|word| from_bytes::expr(&word.cells[..N_BYTES_U64])); - // For curie and non-curie, see l2geth side implementation: - // - cb.condition(is_curie.expr(), |cb| { - cb.require_equal( - "commitScalar * l1BaseFee + blobScalar * _data.length * l1BlobBaseFee == tx_l1_fee * 10e9 + remainder", - commit_scalar * base_fee.clone() + blob_scalar * tx_signed_length * l1_blob_basefee, - tx_l1_fee.clone() * TX_L1_FEE_PRECISION.expr() + remainder.clone(), - ); - }); - - cb.condition(not::expr(is_curie.expr()), |cb| { - let tx_l1_gas = tx_data_gas_cost + TX_L1_COMMIT_EXTRA_COST.expr() + fee_overhead; - cb.require_equal( - "fee_scalar * base_fee * tx_l1_gas == tx_l1_fee * 10e9 + remainder", - fee_scalar * base_fee * tx_l1_gas, - tx_l1_fee * TX_L1_FEE_PRECISION.expr() + remainder, - ); - }); + // refer: + cb.require_equal( + "commitScalar * l1BaseFee + blobScalar * _data.length * l1BlobBaseFee == tx_l1_fee * 10e9 + remainder", + commit_scalar * base_fee + blob_scalar * tx_signed_length * l1_blob_basefee, + tx_l1_fee * TX_L1_FEE_PRECISION.expr() + remainder, + ); let base_fee_committed = cb.query_cell_phase2(); let fee_overhead_committed = cb.query_cell_phase2(); let fee_scalar_committed = cb.query_cell_phase2(); - // curie fields let l1_blob_basefee_committed = cb.query_cell_phase2(); let commit_scalar_committed = cb.query_cell_phase2(); let blob_scalar_committed = cb.query_cell_phase2(); @@ -333,122 +302,93 @@ impl TxL1FeeGadget { #[cfg(test)] mod tests { - use super::*; + use eth_types::U256; + use halo2_proofs::{circuit::Value, halo2curves::bn256::Fr}; + use crate::evm_circuit::util::{ constraint_builder::ConstrainBuilderCommon, math_gadget::test_util::{test_math_gadget_container, try_test, MathGadgetContainer}, }; - use eth_types::U256; - use halo2_proofs::{circuit::Value, halo2curves::bn256::Fr}; - //refer to test in - const TEST_BASE_FEE_BEFORE_CURIE: u64 = 15_000_000; + use super::*; + + //refer: const TEST_FEE_OVERHEAD: u64 = 100; const TEST_FEE_SCALAR: u64 = 10; const TEST_TX_DATA_GAS_COST: u64 = 40; // 2 (zeros) * 4 + 2 (non-zeros) * 16 - const TEST_TX_L1_FEE_BEFORE_CURIE: u128 = 30; - const TEST_AFTER_CURIE: u64 = 1; - const TEST_BEFORE_CURIE: u64 = 0; - - // refer to test in https://github.com/scroll-tech/go-ethereum/blob/develop/rollup/fees/rollup_fee_test.go#L22 - const TEST_BASE_FEE_AFTER_CURIE: u64 = 1_500_000_000; + // refer: + const TEST_BASE_FEE: u64 = 1_500_000_000; const L1_BLOB_BASEFEE: u64 = 150_000_000; const COMMIT_SCALAR: u64 = 10; const BLOB_SCALAR: u64 = 10; - const TEST_TX_RLP_SIGNED_LENGTH: u128 = 4; - const TEST_TX_L1_FEE_AFTER_CURIE: u128 = 21; + const TEST_TX_RLP_SIGNED_LENGTH: u64 = 4; + const TEST_TX_L1_FEE: u64 = 21; #[test] - fn test_tx_l1_fee_with_right_values() { - // test both before & after curie upgrade - for is_curie in [TEST_BEFORE_CURIE, TEST_AFTER_CURIE] { - let witnesses = [ - is_curie.into(), - TEST_BASE_FEE_BEFORE_CURIE.into(), - TEST_FEE_OVERHEAD.into(), - TEST_FEE_SCALAR.into(), - TEST_TX_DATA_GAS_COST.into(), - TEST_TX_L1_FEE_BEFORE_CURIE, - // Curie fields - TEST_BASE_FEE_AFTER_CURIE.into(), - L1_BLOB_BASEFEE.into(), - COMMIT_SCALAR.into(), - BLOB_SCALAR.into(), - TEST_TX_RLP_SIGNED_LENGTH, - TEST_TX_L1_FEE_AFTER_CURIE, - ] - .map(U256::from); - - try_test!(TxL1FeeGadgetTestContainer, witnesses, true); - } + fn test_tx_l1_fee_success() { + let witnesses = [ + TEST_FEE_OVERHEAD, + TEST_FEE_SCALAR, + TEST_TX_DATA_GAS_COST, + TEST_BASE_FEE, + L1_BLOB_BASEFEE, + COMMIT_SCALAR, + BLOB_SCALAR, + TEST_TX_RLP_SIGNED_LENGTH, + TEST_TX_L1_FEE, + ] + .map(U256::from); + + try_test!(TxL1FeeGadgetTestContainer, witnesses, true); } #[test] - fn test_tx_l1_fee_with_wrong_values() { - // test both before & after curie upgrade - for is_curie in [TEST_BEFORE_CURIE, TEST_AFTER_CURIE] { - let witnesses = [ - is_curie.into(), - TEST_BASE_FEE_BEFORE_CURIE.into(), - TEST_FEE_OVERHEAD.into(), - TEST_FEE_SCALAR.into(), - TEST_TX_DATA_GAS_COST.into(), - // set wrong l1 fee - TEST_TX_L1_FEE_BEFORE_CURIE + 1, - // Curie fields - TEST_BASE_FEE_AFTER_CURIE.into(), - L1_BLOB_BASEFEE.into(), - COMMIT_SCALAR.into(), - BLOB_SCALAR.into(), - TEST_TX_RLP_SIGNED_LENGTH, - // set wrong l1 fee - TEST_TX_L1_FEE_AFTER_CURIE + 1, - ] - .map(U256::from); - - try_test!(TxL1FeeGadgetTestContainer, witnesses, false); - } + fn test_tx_l1_fee_failure() { + let witnesses = [ + TEST_FEE_OVERHEAD, + TEST_FEE_SCALAR, + TEST_TX_DATA_GAS_COST, + // set wrong base fee + TEST_BASE_FEE + 1, + TEST_BASE_FEE, + L1_BLOB_BASEFEE, + COMMIT_SCALAR, + BLOB_SCALAR, + TEST_TX_RLP_SIGNED_LENGTH, + // set wrong l1 fee + TEST_TX_L1_FEE + 1, + ] + .map(U256::from); + + try_test!(TxL1FeeGadgetTestContainer, witnesses, false); } #[derive(Clone)] struct TxL1FeeGadgetTestContainer { - is_curie: Cell, - gadget: TxL1FeeGadget, - tx_data_gas_cost: Cell, + tx_l1_fee: TxL1FeeGadget, tx_signed_length: Cell, expected_tx_l1_fee: Cell, } impl MathGadgetContainer for TxL1FeeGadgetTestContainer { fn configure_gadget_container(cb: &mut EVMConstraintBuilder) -> Self { - let tx_data_gas_cost: Cell = cb.query_cell(); let tx_signed_length = cb.query_cell(); let expected_tx_l1_fee = cb.query_cell(); - let is_curie = cb.query_cell(); - - cb.require_boolean("is_curie is bool", is_curie.expr()); - // l1 fee for both before and after Curie upgrade - let gadget = TxL1FeeGadget::::raw_construct( - cb, - is_curie.expr(), - tx_data_gas_cost.expr(), - tx_signed_length.expr(), - ); + + let tx_l1_fee = TxL1FeeGadget::::raw_construct(cb, tx_signed_length.expr()); cb.require_equal( "tx_l1_fee must be correct", - gadget.tx_l1_fee(), + tx_l1_fee.tx_l1_fee(), expected_tx_l1_fee.expr(), ); TxL1FeeGadgetTestContainer { - is_curie, - gadget, - tx_data_gas_cost, + tx_l1_fee, tx_signed_length, expected_tx_l1_fee, } @@ -459,28 +399,20 @@ mod tests { witnesses: &[U256], region: &mut CachedRegion<'_, '_, F>, ) -> Result<(), Error> { - let [is_curie, base_fee_before_curie, fee_overhead, fee_scalar, tx_data_gas_cost, tx_l1_fee_before_curie] = - [0, 1, 2, 3, 4, 5].map(|i| witnesses[i].as_u64()); - - let [base_fee_after_curie, l1_blob_basefee, commit_scalar, blob_scalar, tx_signed_length, tx_l1_fee_after_curie] = - [6, 7, 8, 9, 10, 11].map(|i| witnesses[i].as_u64()); + let [fee_overhead, fee_scalar, tx_data_gas_cost, base_fee, l1_blob_basefee, commit_scalar, blob_scalar, tx_signed_length, tx_l1_fee] = + [0, 1, 2, 3, 4, 5, 6, 7, 8].map(|i| witnesses[i].as_u64()); let l1_fee = TxL1Fee { chain_id: eth_types::forks::SCROLL_DEVNET_CHAIN_ID, - // block_number 5 is starting number for curie in test devnet. - block_number: if is_curie == 1 { 5 + 1 } else { 1 }, - base_fee: if is_curie == 1 { - base_fee_after_curie - } else { - base_fee_before_curie - }, + block_number: 6, + base_fee, fee_overhead, fee_scalar, l1_blob_basefee, commit_scalar, blob_scalar, }; - self.gadget.assign( + self.tx_l1_fee.assign( region, 0, l1_fee, @@ -488,26 +420,10 @@ mod tests { tx_data_gas_cost, tx_signed_length, )?; - - self.tx_data_gas_cost.assign( - region, - 0, - Value::known(tx_data_gas_cost.to_scalar().unwrap()), - )?; - - self.is_curie - .assign(region, 0, Value::known(F::from(is_curie)))?; - let rlp_signed_len = if is_curie == 1 { tx_signed_length } else { 0 }; self.tx_signed_length - .assign(region, 0, Value::known(F::from(rlp_signed_len)))?; - - let expected_tx_l1_fee = if is_curie == 1 { - tx_l1_fee_after_curie - } else { - tx_l1_fee_before_curie - }; + .assign(region, 0, Value::known(F::from(tx_signed_length)))?; self.expected_tx_l1_fee - .assign(region, 0, Value::known(F::from(expected_tx_l1_fee)))?; + .assign(region, 0, Value::known(F::from(tx_l1_fee)))?; Ok(()) }