chore: remove TransactionSignedNoHash (#13120)

This commit is contained in:
Arsenii Kulikov
2024-12-04 11:59:01 +04:00
committed by GitHub
parent 4e73bb04c2
commit 5c07669ef6
12 changed files with 96 additions and 246 deletions

View File

@ -22,7 +22,7 @@ use reth_db::{
}; };
use reth_fs_util as fs; use reth_fs_util as fs;
use reth_primitives::{ use reth_primitives::{
Account, Log, LogData, Receipt, StorageEntry, Transaction, TransactionSignedNoHash, TxType, Account, Log, LogData, Receipt, StorageEntry, Transaction, TransactionSigned, TxType,
}; };
use reth_prune_types::{PruneCheckpoint, PruneMode}; use reth_prune_types::{PruneCheckpoint, PruneMode};
use reth_stages_types::{ use reth_stages_types::{
@ -111,7 +111,7 @@ compact_types!(
StoredBlockBodyIndices, StoredBlockBodyIndices,
StoredBlockWithdrawals, StoredBlockWithdrawals,
// Manual implementations // Manual implementations
TransactionSignedNoHash, TransactionSigned,
// Bytecode, // todo revm arbitrary // Bytecode, // todo revm arbitrary
StorageEntry, StorageEntry,
// MerkleCheckpoint, // todo storedsubnode -> branchnodecompact arbitrary // MerkleCheckpoint, // todo storedsubnode -> branchnodecompact arbitrary

View File

@ -11,7 +11,7 @@ use proptest_arbitrary_interop::arb;
use reth_db::tables; use reth_db::tables;
use reth_db_api::table::{DupSort, Table, TableRow}; use reth_db_api::table::{DupSort, Table, TableRow};
use reth_fs_util as fs; use reth_fs_util as fs;
use reth_primitives::TransactionSignedNoHash; use reth_primitives::TransactionSigned;
use std::collections::HashSet; use std::collections::HashSet;
use tracing::error; use tracing::error;
@ -74,7 +74,7 @@ pub fn generate_vectors(mut tables: Vec<String>) -> Result<()> {
(BlockBodyIndices, PER_TABLE, TABLE), (BlockBodyIndices, PER_TABLE, TABLE),
(BlockOmmers<Header>, 100, TABLE), (BlockOmmers<Header>, 100, TABLE),
(TransactionHashNumbers, PER_TABLE, TABLE), (TransactionHashNumbers, PER_TABLE, TABLE),
(Transactions<TransactionSignedNoHash>, 100, TABLE), (Transactions<TransactionSigned>, 100, TABLE),
(PlainStorageState, PER_TABLE, DUPSORT), (PlainStorageState, PER_TABLE, DUPSORT),
(PlainAccountState, PER_TABLE, TABLE) (PlainAccountState, PER_TABLE, TABLE)
]); ]);

View File

@ -53,7 +53,7 @@ pub use transaction::{
util::secp256k1::{public_key_to_address, recover_signer_unchecked, sign_message}, util::secp256k1::{public_key_to_address, recover_signer_unchecked, sign_message},
BlobTransaction, InvalidTransactionError, PooledTransactionsElement, BlobTransaction, InvalidTransactionError, PooledTransactionsElement,
PooledTransactionsElementEcRecovered, RecoveredTx, Transaction, TransactionMeta, PooledTransactionsElementEcRecovered, RecoveredTx, Transaction, TransactionMeta,
TransactionSigned, TransactionSignedEcRecovered, TransactionSignedNoHash, TxType, TransactionSigned, TransactionSignedEcRecovered, TxType,
}; };
// Re-exports // Re-exports

View File

@ -780,230 +780,6 @@ impl From<TypedTransaction> for Transaction {
} }
} }
/// Signed transaction without its Hash. Used type for inserting into the DB.
///
/// This can by converted to [`TransactionSigned`] by calling [`TransactionSignedNoHash::hash`].
#[derive(Debug, Clone, PartialEq, Eq, Hash, AsRef, Deref, Serialize, Deserialize)]
#[cfg_attr(any(test, feature = "reth-codec"), reth_codecs::add_arbitrary_tests(compact))]
pub struct TransactionSignedNoHash {
/// The transaction signature values
pub signature: Signature,
/// Raw transaction info
#[deref]
#[as_ref]
pub transaction: Transaction,
}
impl TransactionSignedNoHash {
/// Calculates the transaction hash. If used more than once, it's better to convert it to
/// [`TransactionSigned`] first.
pub fn hash(&self) -> B256 {
// pre-allocate buffer for the transaction
let mut buf = Vec::with_capacity(128 + self.transaction.input().len());
self.transaction.eip2718_encode(&self.signature, &mut buf);
keccak256(&buf)
}
/// Recover signer from signature and hash.
///
/// Returns `None` if the transaction's signature is invalid, see also [`Self::recover_signer`].
pub fn recover_signer(&self) -> Option<Address> {
// Optimism's Deposit transaction does not have a signature. Directly return the
// `from` address.
#[cfg(feature = "optimism")]
if let Transaction::Deposit(TxDeposit { from, .. }) = self.transaction {
return Some(from)
}
let signature_hash = self.signature_hash();
recover_signer(&self.signature, signature_hash)
}
/// Recover signer from signature and hash _without ensuring that the signature has a low `s`
/// value_.
///
/// Reuses a given buffer to avoid numerous reallocations when recovering batches. **Clears the
/// buffer before use.**
///
/// Returns `None` if the transaction's signature is invalid, see also
/// [`recover_signer_unchecked`].
///
/// # Optimism
///
/// For optimism this will return [`Address::ZERO`] if the Signature is empty, this is because pre bedrock (on OP mainnet), relay messages to the L2 Cross Domain Messenger were sent as legacy transactions from the zero address with an empty signature, e.g.: <https://optimistic.etherscan.io/tx/0x1bb352ff9215efe5a4c102f45d730bae323c3288d2636672eb61543ddd47abad>
/// This makes it possible to import pre bedrock transactions via the sender recovery stage.
pub fn encode_and_recover_unchecked(&self, buffer: &mut Vec<u8>) -> Option<Address> {
buffer.clear();
self.transaction.encode_for_signing(buffer);
// Optimism's Deposit transaction does not have a signature. Directly return the
// `from` address.
#[cfg(feature = "optimism")]
{
if let Transaction::Deposit(TxDeposit { from, .. }) = self.transaction {
return Some(from)
}
// pre bedrock system transactions were sent from the zero address as legacy
// transactions with an empty signature
//
// NOTE: this is very hacky and only relevant for op-mainnet pre bedrock
if self.is_legacy() && self.signature == TxDeposit::signature() {
return Some(Address::ZERO)
}
}
recover_signer_unchecked(&self.signature, keccak256(buffer))
}
/// Converts into a transaction type with its hash: [`TransactionSigned`].
///
/// Note: This will recalculate the hash of the transaction.
#[inline]
pub fn with_hash(self) -> TransactionSigned {
let Self { signature, transaction } = self;
TransactionSigned::new_unhashed(transaction, signature)
}
/// Recovers a list of signers from a transaction list iterator
///
/// Returns `None`, if some transaction's signature is invalid, see also
/// [`Self::recover_signer`].
pub fn recover_signers<'a, T>(txes: T, num_txes: usize) -> Option<Vec<Address>>
where
T: IntoParallelIterator<Item = &'a Self> + IntoIterator<Item = &'a Self> + Send,
{
if num_txes < *PARALLEL_SENDER_RECOVERY_THRESHOLD {
txes.into_iter().map(|tx| tx.recover_signer()).collect()
} else {
txes.into_par_iter().map(|tx| tx.recover_signer()).collect()
}
}
}
impl Default for TransactionSignedNoHash {
fn default() -> Self {
Self { signature: Signature::test_signature(), transaction: Default::default() }
}
}
#[cfg(any(test, feature = "arbitrary"))]
impl<'a> arbitrary::Arbitrary<'a> for TransactionSignedNoHash {
fn arbitrary(u: &mut arbitrary::Unstructured<'a>) -> arbitrary::Result<Self> {
let tx_signed = TransactionSigned::arbitrary(u)?;
Ok(Self { signature: tx_signed.signature, transaction: tx_signed.transaction })
}
}
#[cfg(any(test, feature = "reth-codec"))]
impl reth_codecs::Compact for TransactionSignedNoHash {
fn to_compact<B>(&self, buf: &mut B) -> usize
where
B: bytes::BufMut + AsMut<[u8]>,
{
let start = buf.as_mut().len();
// Placeholder for bitflags.
// The first byte uses 4 bits as flags: IsCompressed[1bit], TxType[2bits], Signature[1bit]
buf.put_u8(0);
let sig_bit = self.signature.to_compact(buf) as u8;
let zstd_bit = self.transaction.input().len() >= 32;
let tx_bits = if zstd_bit {
let mut tmp = Vec::with_capacity(256);
if cfg!(feature = "std") {
crate::compression::TRANSACTION_COMPRESSOR.with(|compressor| {
let mut compressor = compressor.borrow_mut();
let tx_bits = self.transaction.to_compact(&mut tmp);
buf.put_slice(&compressor.compress(&tmp).expect("Failed to compress"));
tx_bits as u8
})
} else {
let mut compressor = crate::compression::create_tx_compressor();
let tx_bits = self.transaction.to_compact(&mut tmp);
buf.put_slice(&compressor.compress(&tmp).expect("Failed to compress"));
tx_bits as u8
}
} else {
self.transaction.to_compact(buf) as u8
};
// Replace bitflags with the actual values
buf.as_mut()[start] = sig_bit | (tx_bits << 1) | ((zstd_bit as u8) << 3);
buf.as_mut().len() - start
}
fn from_compact(mut buf: &[u8], _len: usize) -> (Self, &[u8]) {
use bytes::Buf;
// The first byte uses 4 bits as flags: IsCompressed[1], TxType[2], Signature[1]
let bitflags = buf.get_u8() as usize;
let sig_bit = bitflags & 1;
let (signature, buf) = Signature::from_compact(buf, sig_bit);
let zstd_bit = bitflags >> 3;
let (transaction, buf) = if zstd_bit != 0 {
if cfg!(feature = "std") {
crate::compression::TRANSACTION_DECOMPRESSOR.with(|decompressor| {
let mut decompressor = decompressor.borrow_mut();
// TODO: enforce that zstd is only present at a "top" level type
let transaction_type = (bitflags & 0b110) >> 1;
let (transaction, _) =
Transaction::from_compact(decompressor.decompress(buf), transaction_type);
(transaction, buf)
})
} else {
let mut decompressor = crate::compression::create_tx_decompressor();
let transaction_type = (bitflags & 0b110) >> 1;
let (transaction, _) =
Transaction::from_compact(decompressor.decompress(buf), transaction_type);
(transaction, buf)
}
} else {
let transaction_type = bitflags >> 1;
Transaction::from_compact(buf, transaction_type)
};
(Self { signature, transaction }, buf)
}
}
#[cfg(any(test, feature = "reth-codec"))]
impl reth_codecs::Compact for TransactionSigned {
fn to_compact<B>(&self, buf: &mut B) -> usize
where
B: bytes::BufMut + AsMut<[u8]>,
{
let tx: TransactionSignedNoHash = self.clone().into();
tx.to_compact(buf)
}
fn from_compact(buf: &[u8], len: usize) -> (Self, &[u8]) {
let (tx, buf) = TransactionSignedNoHash::from_compact(buf, len);
(tx.into(), buf)
}
}
impl From<TransactionSignedNoHash> for TransactionSigned {
fn from(tx: TransactionSignedNoHash) -> Self {
tx.with_hash()
}
}
impl From<TransactionSigned> for TransactionSignedNoHash {
fn from(tx: TransactionSigned) -> Self {
Self { signature: tx.signature, transaction: tx.transaction }
}
}
/// Signed transaction. /// Signed transaction.
#[cfg_attr(any(test, feature = "reth-codec"), reth_codecs::add_arbitrary_tests(rlp))] #[cfg_attr(any(test, feature = "reth-codec"), reth_codecs::add_arbitrary_tests(rlp))]
#[derive(Debug, Clone, Eq, AsRef, Deref, Serialize, Deserialize)] #[derive(Debug, Clone, Eq, AsRef, Deref, Serialize, Deserialize)]
@ -1543,6 +1319,86 @@ impl Decodable2718 for TransactionSigned {
} }
} }
#[cfg(any(test, feature = "reth-codec"))]
impl reth_codecs::Compact for TransactionSigned {
fn to_compact<B>(&self, buf: &mut B) -> usize
where
B: bytes::BufMut + AsMut<[u8]>,
{
let start = buf.as_mut().len();
// Placeholder for bitflags.
// The first byte uses 4 bits as flags: IsCompressed[1bit], TxType[2bits], Signature[1bit]
buf.put_u8(0);
let sig_bit = self.signature.to_compact(buf) as u8;
let zstd_bit = self.transaction.input().len() >= 32;
let tx_bits = if zstd_bit {
let mut tmp = Vec::with_capacity(256);
if cfg!(feature = "std") {
crate::compression::TRANSACTION_COMPRESSOR.with(|compressor| {
let mut compressor = compressor.borrow_mut();
let tx_bits = self.transaction.to_compact(&mut tmp);
buf.put_slice(&compressor.compress(&tmp).expect("Failed to compress"));
tx_bits as u8
})
} else {
let mut compressor = crate::compression::create_tx_compressor();
let tx_bits = self.transaction.to_compact(&mut tmp);
buf.put_slice(&compressor.compress(&tmp).expect("Failed to compress"));
tx_bits as u8
}
} else {
self.transaction.to_compact(buf) as u8
};
// Replace bitflags with the actual values
buf.as_mut()[start] = sig_bit | (tx_bits << 1) | ((zstd_bit as u8) << 3);
buf.as_mut().len() - start
}
fn from_compact(mut buf: &[u8], _len: usize) -> (Self, &[u8]) {
use bytes::Buf;
// The first byte uses 4 bits as flags: IsCompressed[1], TxType[2], Signature[1]
let bitflags = buf.get_u8() as usize;
let sig_bit = bitflags & 1;
let (signature, buf) = Signature::from_compact(buf, sig_bit);
let zstd_bit = bitflags >> 3;
let (transaction, buf) = if zstd_bit != 0 {
if cfg!(feature = "std") {
crate::compression::TRANSACTION_DECOMPRESSOR.with(|decompressor| {
let mut decompressor = decompressor.borrow_mut();
// TODO: enforce that zstd is only present at a "top" level type
let transaction_type = (bitflags & 0b110) >> 1;
let (transaction, _) =
Transaction::from_compact(decompressor.decompress(buf), transaction_type);
(transaction, buf)
})
} else {
let mut decompressor = crate::compression::create_tx_decompressor();
let transaction_type = (bitflags & 0b110) >> 1;
let (transaction, _) =
Transaction::from_compact(decompressor.decompress(buf), transaction_type);
(transaction, buf)
}
} else {
let transaction_type = bitflags >> 1;
Transaction::from_compact(buf, transaction_type)
};
(Self { signature, transaction, hash: Default::default() }, buf)
}
}
macro_rules! impl_from_signed { macro_rules! impl_from_signed {
($($tx:ident),*) => { ($($tx:ident),*) => {
$( $(

View File

@ -29,7 +29,7 @@ pub const COMPACT_EXTENDED_IDENTIFIER_FLAG: usize = 3;
/// Transaction Type /// Transaction Type
/// ///
/// Currently being used as 2-bit type when encoding it to `reth_codecs::Compact` on /// Currently being used as 2-bit type when encoding it to `reth_codecs::Compact` on
/// [`crate::TransactionSignedNoHash`]. Adding more transaction types will break the codec and /// [`crate::TransactionSigned`]. Adding more transaction types will break the codec and
/// database format. /// database format.
/// ///
/// Other required changes when adding a new type can be seen on [PR#3953](https://github.com/paradigmxyz/reth/pull/3953/files). /// Other required changes when adding a new type can be seen on [PR#3953](https://github.com/paradigmxyz/reth/pull/3953/files).

View File

@ -359,10 +359,7 @@ mod tests {
transaction.hash(), transaction.hash(),
next_tx_num, next_tx_num,
)?; )?;
tx.put::<tables::Transactions>( tx.put::<tables::Transactions>(next_tx_num, transaction.clone())?;
next_tx_num,
transaction.clone().into(),
)?;
let (addr, _) = let (addr, _) =
accounts.get_mut(rng.gen::<usize>() % n_accounts as usize).unwrap(); accounts.get_mut(rng.gen::<usize>() % n_accounts as usize).unwrap();

View File

@ -267,7 +267,7 @@ impl TestStageDB {
if let Some(txs_writer) = &mut txs_writer { if let Some(txs_writer) = &mut txs_writer {
txs_writer.append_transaction(next_tx_num, body_tx)?; txs_writer.append_transaction(next_tx_num, body_tx)?;
} else { } else {
tx.put::<tables::Transactions>(next_tx_num, body_tx.clone().into())? tx.put::<tables::Transactions>(next_tx_num, body_tx.clone())?
} }
next_tx_num += 1; next_tx_num += 1;
Ok::<(), ProviderError>(()) Ok::<(), ProviderError>(())

View File

@ -8,7 +8,7 @@ use alloy_consensus::Header;
use alloy_genesis::GenesisAccount; use alloy_genesis::GenesisAccount;
use alloy_primitives::{Address, Bytes, Log, B256, U256}; use alloy_primitives::{Address, Bytes, Log, B256, U256};
use reth_codecs::{add_arbitrary_tests, Compact}; use reth_codecs::{add_arbitrary_tests, Compact};
use reth_primitives::{Receipt, StorageEntry, TransactionSigned, TransactionSignedNoHash, TxType}; use reth_primitives::{Receipt, StorageEntry, TransactionSigned, TxType};
use reth_primitives_traits::{Account, Bytecode}; use reth_primitives_traits::{Account, Bytecode};
use reth_prune_types::{PruneCheckpoint, PruneSegment}; use reth_prune_types::{PruneCheckpoint, PruneSegment};
use reth_stages_types::StageCheckpoint; use reth_stages_types::StageCheckpoint;
@ -226,7 +226,6 @@ impl_compression_for_compact!(
StoredBlockWithdrawals, StoredBlockWithdrawals,
Bytecode, Bytecode,
AccountBeforeTx, AccountBeforeTx,
TransactionSignedNoHash,
TransactionSigned, TransactionSigned,
CompactU256, CompactU256,
StageCheckpoint, StageCheckpoint,

View File

@ -31,7 +31,7 @@ use reth_db_api::{
}, },
table::{Decode, DupSort, Encode, Table}, table::{Decode, DupSort, Encode, Table},
}; };
use reth_primitives::{Receipt, StorageEntry, TransactionSignedNoHash}; use reth_primitives::{Receipt, StorageEntry, TransactionSigned};
use reth_primitives_traits::{Account, Bytecode}; use reth_primitives_traits::{Account, Bytecode};
use reth_prune_types::{PruneCheckpoint, PruneSegment}; use reth_prune_types::{PruneCheckpoint, PruneSegment};
use reth_stages_types::StageCheckpoint; use reth_stages_types::StageCheckpoint;
@ -348,7 +348,7 @@ tables! {
} }
/// Canonical only Stores the transaction body for canonical transactions. /// Canonical only Stores the transaction body for canonical transactions.
table Transactions<T = TransactionSignedNoHash> { table Transactions<T = TransactionSigned> {
type Key = TxNumber; type Key = TxNumber;
type Value = T; type Value = T;
} }

View File

@ -29,7 +29,7 @@ use reth_node_types::{BlockTy, HeaderTy, NodeTypesWithDB, ReceiptTy, TxTy};
use reth_primitives::{ use reth_primitives::{
Account, Block, BlockWithSenders, EthPrimitives, NodePrimitives, Receipt, SealedBlock, Account, Block, BlockWithSenders, EthPrimitives, NodePrimitives, Receipt, SealedBlock,
SealedBlockFor, SealedBlockWithSenders, SealedHeader, StorageEntry, TransactionMeta, SealedBlockFor, SealedBlockWithSenders, SealedHeader, StorageEntry, TransactionMeta,
TransactionSigned, TransactionSignedNoHash, TransactionSigned,
}; };
use reth_primitives_traits::BlockBody as _; use reth_primitives_traits::BlockBody as _;
use reth_prune_types::{PruneCheckpoint, PruneSegment}; use reth_prune_types::{PruneCheckpoint, PruneSegment};
@ -850,9 +850,7 @@ mod tests {
use reth_db_api::{cursor::DbCursorRO, transaction::DbTx}; use reth_db_api::{cursor::DbCursorRO, transaction::DbTx};
use reth_errors::ProviderError; use reth_errors::ProviderError;
use reth_execution_types::{Chain, ExecutionOutcome}; use reth_execution_types::{Chain, ExecutionOutcome};
use reth_primitives::{ use reth_primitives::{BlockExt, Receipt, SealedBlock, StaticFileSegment};
BlockExt, Receipt, SealedBlock, StaticFileSegment, TransactionSignedNoHash,
};
use reth_primitives_traits::{BlockBody as _, SignedTransaction}; use reth_primitives_traits::{BlockBody as _, SignedTransaction};
use reth_storage_api::{ use reth_storage_api::{
BlockHashReader, BlockIdReader, BlockNumReader, BlockReader, BlockReaderIdExt, BlockSource, BlockHashReader, BlockIdReader, BlockNumReader, BlockReader, BlockReaderIdExt, BlockSource,

View File

@ -39,7 +39,7 @@ use reth_primitives::{
}, },
transaction::recover_signers, transaction::recover_signers,
BlockWithSenders, Receipt, SealedBlockFor, SealedBlockWithSenders, SealedHeader, BlockWithSenders, Receipt, SealedBlockFor, SealedBlockWithSenders, SealedHeader,
StaticFileSegment, TransactionMeta, TransactionSignedNoHash, StaticFileSegment, TransactionMeta, TransactionSigned,
}; };
use reth_primitives_traits::SignedTransaction; use reth_primitives_traits::SignedTransaction;
use reth_stages_types::{PipelineTarget, StageId}; use reth_stages_types::{PipelineTarget, StageId};
@ -1706,7 +1706,7 @@ impl<N: NodePrimitives> StatsReader for StaticFileProvider<N> {
.get_highest_static_file_tx(StaticFileSegment::Receipts) .get_highest_static_file_tx(StaticFileSegment::Receipts)
.map(|receipts| receipts + 1) .map(|receipts| receipts + 1)
.unwrap_or_default() as usize), .unwrap_or_default() as usize),
tables::Transactions::<TransactionSignedNoHash>::NAME => Ok(self tables::Transactions::<TransactionSigned>::NAME => Ok(self
.get_highest_static_file_tx(StaticFileSegment::Transactions) .get_highest_static_file_tx(StaticFileSegment::Transactions)
.map(|txs| txs + 1) .map(|txs| txs + 1)
.unwrap_or_default() .unwrap_or_default()

View File

@ -56,7 +56,7 @@ BlockWithdrawals {
} }
Transactions { Transactions {
u64 TxNumber "PK" u64 TxNumber "PK"
TransactionSignedNoHash Data TransactionSigned Data
} }
TransactionHashNumbers { TransactionHashNumbers {
B256 TxHash "PK" B256 TxHash "PK"