Removed PowBlock struct that never got used (#2813)

This commit is contained in:
ethDreamer 2021-11-16 23:51:11 -06:00 committed by Paul Hauner
parent 5f0fef2d1e
commit f6748537db
No known key found for this signature in database
GPG Key ID: 5E2CFF9B75FA63DF
6 changed files with 35 additions and 129 deletions

View File

@ -15,8 +15,8 @@ use std::time::{SystemTime, UNIX_EPOCH};
use store::{DBColumn, Error as StoreError, StoreItem}; use store::{DBColumn, Error as StoreError, StoreItem};
use task_executor::TaskExecutor; use task_executor::TaskExecutor;
use types::{ use types::{
BeaconState, BeaconStateError, ChainSpec, Deposit, Eth1Data, EthSpec, ExecutionPayload, BeaconState, BeaconStateError, ChainSpec, Deposit, Eth1Data, EthSpec, Hash256, Slot, Unsigned,
Hash256, Slot, Unsigned, DEPOSIT_TREE_DEPTH, DEPOSIT_TREE_DEPTH,
}; };
type BlockNumber = u64; type BlockNumber = u64;
@ -53,8 +53,6 @@ pub enum Error {
UnknownPreviousEth1BlockHash, UnknownPreviousEth1BlockHash,
/// An arithmetic error occurred. /// An arithmetic error occurred.
ArithError(safe_arith::ArithError), ArithError(safe_arith::ArithError),
/// Unable to execute payload
UnableToExecutePayload(String),
} }
impl From<safe_arith::ArithError> for Error { impl From<safe_arith::ArithError> for Error {
@ -281,15 +279,6 @@ where
) )
} }
pub fn on_payload(&self, execution_payload: &ExecutionPayload<E>) -> Result<bool, Error> {
if self.use_dummy_backend {
let dummy_backend: DummyEth1ChainBackend<E> = DummyEth1ChainBackend::default();
dummy_backend.on_payload(execution_payload)
} else {
self.backend.on_payload(execution_payload)
}
}
/// Instantiate `Eth1Chain` from a persisted `SszEth1`. /// Instantiate `Eth1Chain` from a persisted `SszEth1`.
/// ///
/// The `Eth1Chain` will have the same caches as the persisted `SszEth1`. /// The `Eth1Chain` will have the same caches as the persisted `SszEth1`.
@ -350,9 +339,6 @@ pub trait Eth1ChainBackend<T: EthSpec>: Sized + Send + Sync {
/// an idea of how up-to-date the remote eth1 node is. /// an idea of how up-to-date the remote eth1 node is.
fn head_block(&self) -> Option<Eth1Block>; fn head_block(&self) -> Option<Eth1Block>;
/// Verifies the execution payload
fn on_payload(&self, execution_payload: &ExecutionPayload<T>) -> Result<bool, Error>;
/// Encode the `Eth1ChainBackend` instance to bytes. /// Encode the `Eth1ChainBackend` instance to bytes.
fn as_bytes(&self) -> Vec<u8>; fn as_bytes(&self) -> Vec<u8>;
@ -407,10 +393,6 @@ impl<T: EthSpec> Eth1ChainBackend<T> for DummyEth1ChainBackend<T> {
None None
} }
fn on_payload(&self, _execution_payload: &ExecutionPayload<T>) -> Result<bool, Error> {
Ok(true)
}
/// Return empty Vec<u8> for dummy backend. /// Return empty Vec<u8> for dummy backend.
fn as_bytes(&self) -> Vec<u8> { fn as_bytes(&self) -> Vec<u8> {
Vec::new() Vec::new()
@ -579,15 +561,6 @@ impl<T: EthSpec> Eth1ChainBackend<T> for CachingEth1Backend<T> {
self.core.head_block() self.core.head_block()
} }
fn on_payload(&self, execution_payload: &ExecutionPayload<T>) -> Result<bool, Error> {
futures::executor::block_on(async move {
self.core
.on_payload(execution_payload.clone())
.await
.map_err(|e| Error::UnableToExecutePayload(format!("{:?}", e)))
})
}
/// Return encoded byte representation of the block and deposit caches. /// Return encoded byte representation of the block and deposit caches.
fn as_bytes(&self) -> Vec<u8> { fn as_bytes(&self) -> Vec<u8> {
self.core.as_bytes() self.core.as_bytes()

View File

@ -19,7 +19,7 @@ use std::fmt;
use std::ops::Range; use std::ops::Range;
use std::str::FromStr; use std::str::FromStr;
use std::time::Duration; use std::time::Duration;
use types::{Hash256, PowBlock, Uint256}; use types::Hash256;
/// `keccak("DepositEvent(bytes,bytes,bytes,bytes,bytes)")` /// `keccak("DepositEvent(bytes,bytes,bytes,bytes,bytes)")`
pub const DEPOSIT_EVENT_TOPIC: &str = pub const DEPOSIT_EVENT_TOPIC: &str =
@ -49,7 +49,6 @@ pub enum Eth1Id {
#[derive(Clone, Copy)] #[derive(Clone, Copy)]
pub enum BlockQuery { pub enum BlockQuery {
Number(u64), Number(u64),
Hash(Hash256),
Latest, Latest,
} }
@ -136,6 +135,13 @@ pub async fn get_chain_id(endpoint: &SensitiveUrl, timeout: Duration) -> Result<
} }
} }
#[derive(Debug, PartialEq, Clone)]
pub struct Block {
pub hash: Hash256,
pub timestamp: u64,
pub number: u64,
}
/// Returns the current block number. /// Returns the current block number.
/// ///
/// Uses HTTP JSON RPC at `endpoint`. E.g., `http://localhost:8545`. /// Uses HTTP JSON RPC at `endpoint`. E.g., `http://localhost:8545`.
@ -150,74 +156,40 @@ pub async fn get_block_number(endpoint: &SensitiveUrl, timeout: Duration) -> Res
.map_err(|e| format!("Failed to get block number: {}", e)) .map_err(|e| format!("Failed to get block number: {}", e))
} }
/// Gets a block by hash or block number. /// Gets a block hash by block number.
/// ///
/// Uses HTTP JSON RPC at `endpoint`. E.g., `http://localhost:8545`. /// Uses HTTP JSON RPC at `endpoint`. E.g., `http://localhost:8545`.
pub async fn get_block( pub async fn get_block(
endpoint: &SensitiveUrl, endpoint: &SensitiveUrl,
query: BlockQuery, query: BlockQuery,
timeout: Duration, timeout: Duration,
) -> Result<PowBlock, String> { ) -> Result<Block, String> {
let query_param = match query { let query_param = match query {
BlockQuery::Number(block_number) => format!("0x{:x}", block_number), BlockQuery::Number(block_number) => format!("0x{:x}", block_number),
BlockQuery::Hash(hash) => format!("{:?}", hash), // debug formatting ensures output not truncated
BlockQuery::Latest => "latest".to_string(), BlockQuery::Latest => "latest".to_string(),
}; };
let rpc_method = match query {
BlockQuery::Number(_) | BlockQuery::Latest => "eth_getBlockByNumber",
BlockQuery::Hash(_) => "eth_getBlockByHash",
};
let params = json!([ let params = json!([
query_param, query_param,
false // do not return full tx objects. false // do not return full tx objects.
]); ]);
let response_body = send_rpc_request(endpoint, rpc_method, params, timeout).await?; let response_body = send_rpc_request(endpoint, "eth_getBlockByNumber", params, timeout).await?;
let response = response_result_or_error(&response_body) let response = response_result_or_error(&response_body)
.map_err(|e| format!("{} failed: {}", rpc_method, e))?; .map_err(|e| format!("eth_getBlockByNumber failed: {}", e))?;
let block_hash: Vec<u8> = hex_to_bytes( let hash: Vec<u8> = hex_to_bytes(
response response
.get("hash") .get("hash")
.ok_or("No hash for block")? .ok_or("No hash for block")?
.as_str() .as_str()
.ok_or("Block hash was not string")?, .ok_or("Block hash was not string")?,
)?; )?;
let block_hash: Hash256 = if block_hash.len() == 32 { let hash: Hash256 = if hash.len() == 32 {
Hash256::from_slice(&block_hash) Hash256::from_slice(&hash)
} else { } else {
return Err(format!("Block hash was not 32 bytes: {:?}", block_hash)); return Err(format!("Block has was not 32 bytes: {:?}", hash));
}; };
let parent_hash: Vec<u8> = hex_to_bytes(
response
.get("parentHash")
.ok_or("No parent hash for block")?
.as_str()
.ok_or("Parent hash was not string")?,
)?;
let parent_hash: Hash256 = if parent_hash.len() == 32 {
Hash256::from_slice(&parent_hash)
} else {
return Err(format!("parent hash was not 32 bytes: {:?}", parent_hash));
};
let total_difficulty_str = response
.get("totalDifficulty")
.ok_or("No total difficulty for block")?
.as_str()
.ok_or("Total difficulty was not a string")?;
let total_difficulty = Uint256::from_str(total_difficulty_str)
.map_err(|e| format!("total_difficulty from_str {:?}", e))?;
let difficulty_str = response
.get("difficulty")
.ok_or("No difficulty for block")?
.as_str()
.ok_or("Difficulty was not a string")?;
let difficulty =
Uint256::from_str(difficulty_str).map_err(|e| format!("difficulty from_str {:?}", e))?;
let timestamp = hex_to_u64_be( let timestamp = hex_to_u64_be(
response response
.get("timestamp") .get("timestamp")
@ -226,7 +198,7 @@ pub async fn get_block(
.ok_or("Block timestamp was not string")?, .ok_or("Block timestamp was not string")?,
)?; )?;
let block_number = hex_to_u64_be( let number = hex_to_u64_be(
response response
.get("number") .get("number")
.ok_or("No number for block")? .ok_or("No number for block")?
@ -234,20 +206,14 @@ pub async fn get_block(
.ok_or("Block number was not string")?, .ok_or("Block number was not string")?,
)?; )?;
if block_number <= usize::max_value() as u64 { if number <= usize::max_value() as u64 {
Ok(PowBlock { Ok(Block {
block_hash, hash,
parent_hash,
total_difficulty,
difficulty,
timestamp, timestamp,
block_number, number,
}) })
} else { } else {
Err(format!( Err(format!("Block number {} is larger than a usize", number))
"Block number {} is larger than a usize",
block_number
))
} }
.map_err(|e| format!("Failed to get block number: {}", e)) .map_err(|e| format!("Failed to get block number: {}", e))
} }
@ -479,7 +445,7 @@ pub async fn send_rpc_request(
} }
/// Accepts an entire HTTP body (as a string) and returns either the `result` field or the `error['message']` field, as a serde `Value`. /// Accepts an entire HTTP body (as a string) and returns either the `result` field or the `error['message']` field, as a serde `Value`.
pub fn response_result_or_error(response: &str) -> Result<Value, RpcError> { fn response_result_or_error(response: &str) -> Result<Value, RpcError> {
let json = serde_json::from_str::<Value>(response) let json = serde_json::from_str::<Value>(response)
.map_err(|e| RpcError::InvalidJson(e.to_string()))?; .map_err(|e| RpcError::InvalidJson(e.to_string()))?;
@ -501,7 +467,7 @@ pub fn response_result_or_error(response: &str) -> Result<Value, RpcError> {
/// Therefore, this function is only useful for numbers encoded by the JSON RPC. /// Therefore, this function is only useful for numbers encoded by the JSON RPC.
/// ///
/// E.g., `0x01 == 1` /// E.g., `0x01 == 1`
pub fn hex_to_u64_be(hex: &str) -> Result<u64, String> { fn hex_to_u64_be(hex: &str) -> Result<u64, String> {
u64::from_str_radix(strip_prefix(hex)?, 16) u64::from_str_radix(strip_prefix(hex)?, 16)
.map_err(|e| format!("Failed to parse hex as u64: {:?}", e)) .map_err(|e| format!("Failed to parse hex as u64: {:?}", e))
} }

View File

@ -21,7 +21,7 @@ use std::sync::Arc;
use std::time::{SystemTime, UNIX_EPOCH}; use std::time::{SystemTime, UNIX_EPOCH};
use tokio::sync::RwLock as TRwLock; use tokio::sync::RwLock as TRwLock;
use tokio::time::{interval_at, Duration, Instant}; use tokio::time::{interval_at, Duration, Instant};
use types::{ChainSpec, EthSpec, ExecutionPayload, Unsigned}; use types::{ChainSpec, EthSpec, Unsigned};
/// Indicates the default eth1 network id we use for the deposit contract. /// Indicates the default eth1 network id we use for the deposit contract.
pub const DEFAULT_NETWORK_ID: Eth1Id = Eth1Id::Goerli; pub const DEFAULT_NETWORK_ID: Eth1Id = Eth1Id::Goerli;
@ -331,8 +331,6 @@ pub enum SingleEndpointError {
GetDepositCountFailed(String), GetDepositCountFailed(String),
/// Failed to read the deposit contract root from the eth1 node. /// Failed to read the deposit contract root from the eth1 node.
GetDepositLogsFailed(String), GetDepositLogsFailed(String),
/// Failed to run engine_ExecutePayload
EngineExecutePayloadFailed,
} }
#[derive(Debug, PartialEq)] #[derive(Debug, PartialEq)]
@ -671,21 +669,6 @@ impl Service {
} }
} }
/// This is were we call out to engine_executePayload to determine if payload is valid
pub async fn on_payload<T: EthSpec>(
&self,
_execution_payload: ExecutionPayload<T>,
) -> Result<bool, Error> {
let endpoints = self.init_endpoints();
// TODO: call engine_executePayload and figure out how backup endpoint works..
endpoints
.first_success(|_e| async move { Ok(true) })
.await
.map(|(res, _)| res)
.map_err(Error::FallbackError)
}
/// Update the deposit and block cache, returning an error if either fail. /// Update the deposit and block cache, returning an error if either fail.
/// ///
/// ## Returns /// ## Returns
@ -1259,7 +1242,7 @@ async fn download_eth1_block(
}); });
// Performs a `get_blockByNumber` call to an eth1 node. // Performs a `get_blockByNumber` call to an eth1 node.
let pow_block = get_block( let http_block = get_block(
endpoint, endpoint,
block_number_opt block_number_opt
.map(BlockQuery::Number) .map(BlockQuery::Number)
@ -1270,9 +1253,9 @@ async fn download_eth1_block(
.await?; .await?;
Ok(Eth1Block { Ok(Eth1Block {
hash: pow_block.block_hash, hash: http_block.hash,
number: pow_block.block_number, number: http_block.number,
timestamp: pow_block.timestamp, timestamp: http_block.timestamp,
deposit_root, deposit_root,
deposit_count, deposit_count,
}) })

View File

@ -1,6 +1,6 @@
#![cfg(test)] #![cfg(test)]
use environment::{Environment, EnvironmentBuilder}; use environment::{Environment, EnvironmentBuilder};
use eth1::http::{get_deposit_count, get_deposit_logs_in_range, get_deposit_root, Log}; use eth1::http::{get_deposit_count, get_deposit_logs_in_range, get_deposit_root, Block, Log};
use eth1::{Config, Service}; use eth1::{Config, Service};
use eth1::{DepositCache, DEFAULT_CHAIN_ID, DEFAULT_NETWORK_ID}; use eth1::{DepositCache, DEFAULT_CHAIN_ID, DEFAULT_NETWORK_ID};
use eth1_test_rig::GanacheEth1Instance; use eth1_test_rig::GanacheEth1Instance;
@ -571,9 +571,8 @@ mod deposit_tree {
mod http { mod http {
use super::*; use super::*;
use eth1::http::BlockQuery; use eth1::http::BlockQuery;
use types::PowBlock;
async fn get_block(eth1: &GanacheEth1Instance, block_number: u64) -> PowBlock { async fn get_block(eth1: &GanacheEth1Instance, block_number: u64) -> Block {
eth1::http::get_block( eth1::http::get_block(
&SensitiveUrl::parse(eth1.endpoint().as_str()).unwrap(), &SensitiveUrl::parse(eth1.endpoint().as_str()).unwrap(),
BlockQuery::Number(block_number), BlockQuery::Number(block_number),
@ -640,7 +639,7 @@ mod http {
// Check the block hash. // Check the block hash.
let new_block = get_block(&eth1, block_number).await; let new_block = get_block(&eth1, block_number).await;
assert_ne!( assert_ne!(
new_block.block_hash, old_block.block_hash, new_block.hash, old_block.hash,
"block hash should change with each deposit" "block hash should change with each deposit"
); );
@ -662,7 +661,7 @@ mod http {
// Check to ensure the block root is changing // Check to ensure the block root is changing
assert_ne!( assert_ne!(
new_root, new_root,
Some(new_block.block_hash), Some(new_block.hash),
"the deposit root should be different to the block hash" "the deposit root should be different to the block hash"
); );
} }

View File

@ -47,7 +47,6 @@ pub mod graffiti;
pub mod historical_batch; pub mod historical_batch;
pub mod indexed_attestation; pub mod indexed_attestation;
pub mod pending_attestation; pub mod pending_attestation;
pub mod pow_block;
pub mod proposer_slashing; pub mod proposer_slashing;
pub mod relative_epoch; pub mod relative_epoch;
pub mod selection_proof; pub mod selection_proof;
@ -126,7 +125,6 @@ pub use crate::indexed_attestation::IndexedAttestation;
pub use crate::participation_flags::ParticipationFlags; pub use crate::participation_flags::ParticipationFlags;
pub use crate::participation_list::ParticipationList; pub use crate::participation_list::ParticipationList;
pub use crate::pending_attestation::PendingAttestation; pub use crate::pending_attestation::PendingAttestation;
pub use crate::pow_block::PowBlock;
pub use crate::preset::{AltairPreset, BasePreset}; pub use crate::preset::{AltairPreset, BasePreset};
pub use crate::proposer_slashing::ProposerSlashing; pub use crate::proposer_slashing::ProposerSlashing;
pub use crate::relative_epoch::{Error as RelativeEpochError, RelativeEpoch}; pub use crate::relative_epoch::{Error as RelativeEpochError, RelativeEpoch};

View File

@ -1,13 +0,0 @@
use crate::*;
#[cfg_attr(feature = "arbitrary-fuzz", derive(arbitrary::Arbitrary))]
#[derive(Default, Debug, PartialEq, Clone)]
pub struct PowBlock {
pub block_hash: Hash256,
pub parent_hash: Hash256,
pub total_difficulty: Uint256,
pub difficulty: Uint256,
// needed to unify with other parts of codebase
pub timestamp: u64,
pub block_number: u64,
}