{
/// Remove any block components from the *processing cache* if we no longer require them. If the
/// block was imported full or erred, we no longer require them.
- fn remove_notified_custody_columns(
+ fn remove_notified_custody_columns(
&self,
block_root: &Hash256,
- r: Result>,
- ) -> Result> {
- let has_missing_components =
- matches!(r, Ok(AvailabilityProcessingStatus::MissingComponents(_, _)));
+ r: Result<(AvailabilityProcessingStatus, P), BlockError>,
+ ) -> Result<(AvailabilityProcessingStatus, P), BlockError> {
+ let has_missing_components = matches!(
+ r,
+ Ok((AvailabilityProcessingStatus::MissingComponents(_, _), _))
+ );
if !has_missing_components {
self.reqresp_pre_import_cache.write().remove(block_root);
}
@@ -3058,7 +3168,7 @@ impl BeaconChain {
unverified_block: B,
block_source: BlockImportSource,
notify_execution_layer: NotifyExecutionLayer,
- ) -> Result> {
+ ) -> Result {
self.reqresp_pre_import_cache
.write()
.insert(block_root, unverified_block.block_cloned());
@@ -3094,8 +3204,8 @@ impl BeaconChain {
unverified_block: B,
notify_execution_layer: NotifyExecutionLayer,
block_source: BlockImportSource,
- publish_fn: impl FnOnce() -> Result<(), BlockError> + Send + 'static,
- ) -> Result> {
+ publish_fn: impl FnOnce() -> Result<(), BlockError> + Send + 'static,
+ ) -> Result {
// Start the Prometheus timer.
let _full_timer = metrics::start_timer(&metrics::BLOCK_PROCESSING_TIMES);
@@ -3216,7 +3326,7 @@ impl BeaconChain {
pub async fn into_executed_block(
self: Arc,
execution_pending_block: ExecutionPendingBlock,
- ) -> Result, BlockError> {
+ ) -> Result, BlockError> {
let ExecutionPendingBlock {
block,
import_data,
@@ -3271,7 +3381,7 @@ impl BeaconChain {
async fn check_block_availability_and_import(
self: &Arc,
block: AvailabilityPendingExecutedBlock,
- ) -> Result> {
+ ) -> Result {
let slot = block.block.slot();
let availability = self
.data_availability_checker
@@ -3284,7 +3394,7 @@ impl BeaconChain {
async fn check_gossip_blob_availability_and_import(
self: &Arc,
blob: GossipVerifiedBlob,
- ) -> Result> {
+ ) -> Result {
let slot = blob.slot();
if let Some(slasher) = self.slasher.as_ref() {
slasher.accept_block_header(blob.signed_block_header());
@@ -3301,20 +3411,26 @@ impl BeaconChain {
slot: Slot,
block_root: Hash256,
data_columns: Vec>,
- ) -> Result> {
+ ) -> Result<
+ (
+ AvailabilityProcessingStatus,
+ DataColumnsToPublish,
+ ),
+ BlockError,
+ > {
if let Some(slasher) = self.slasher.as_ref() {
for data_colum in &data_columns {
slasher.accept_block_header(data_colum.signed_block_header());
}
}
- let availability = self.data_availability_checker.put_gossip_data_columns(
- slot,
- block_root,
- data_columns,
- )?;
+ let (availability, data_columns_to_publish) = self
+ .data_availability_checker
+ .put_gossip_data_columns(slot, block_root, data_columns)?;
- self.process_availability(slot, availability).await
+ self.process_availability(slot, availability)
+ .await
+ .map(|result| (result, data_columns_to_publish))
}
/// Checks if the provided blobs can make any cached blocks available, and imports immediately
@@ -3324,17 +3440,17 @@ impl BeaconChain {
slot: Slot,
block_root: Hash256,
blobs: FixedBlobSidecarList,
- ) -> Result> {
+ ) -> Result {
// Need to scope this to ensure the lock is dropped before calling `process_availability`
// Even an explicit drop is not enough to convince the borrow checker.
{
let mut slashable_cache = self.observed_slashable.write();
for header in blobs
- .into_iter()
+ .iter()
.filter_map(|b| b.as_ref().map(|b| b.signed_block_header.clone()))
.unique()
{
- if verify_header_signature::>(self, &header).is_ok() {
+ if verify_header_signature::(self, &header).is_ok() {
slashable_cache
.observe_slashable(
header.message.slot,
@@ -3356,6 +3472,56 @@ impl BeaconChain {
self.process_availability(slot, availability).await
}
+ /// Checks if the provided columns can make any cached blocks available, and imports immediately
+ /// if so, otherwise caches the columns in the data availability checker.
+ async fn check_rpc_custody_columns_availability_and_import(
+ self: &Arc,
+ slot: Slot,
+ block_root: Hash256,
+ custody_columns: DataColumnSidecarList,
+ ) -> Result<
+ (
+ AvailabilityProcessingStatus,
+ DataColumnsToPublish,
+ ),
+ BlockError,
+ > {
+ // Need to scope this to ensure the lock is dropped before calling `process_availability`
+ // Even an explicit drop is not enough to convince the borrow checker.
+ {
+ let mut slashable_cache = self.observed_slashable.write();
+ // Assumes all items in custody_columns are for the same block_root
+ if let Some(column) = custody_columns.first() {
+ let header = &column.signed_block_header;
+ if verify_header_signature::(self, header).is_ok() {
+ slashable_cache
+ .observe_slashable(
+ header.message.slot,
+ header.message.proposer_index,
+ block_root,
+ )
+ .map_err(|e| BlockError::BeaconChainError(e.into()))?;
+ if let Some(slasher) = self.slasher.as_ref() {
+ slasher.accept_block_header(header.clone());
+ }
+ }
+ }
+ }
+
+ // This slot value is purely informative for the consumers of
+ // `AvailabilityProcessingStatus::MissingComponents` to log an error with a slot.
+ let (availability, data_columns_to_publish) =
+ self.data_availability_checker.put_rpc_custody_columns(
+ block_root,
+ slot.epoch(T::EthSpec::slots_per_epoch()),
+ custody_columns,
+ )?;
+
+ self.process_availability(slot, availability)
+ .await
+ .map(|result| (result, data_columns_to_publish))
+ }
+
/// Imports a fully available block. Otherwise, returns `AvailabilityProcessingStatus::MissingComponents`
///
/// An error is returned if the block was unable to be imported. It may be partially imported
@@ -3364,7 +3530,7 @@ impl BeaconChain {
self: &Arc,
slot: Slot,
availability: Availability,
- ) -> Result> {
+ ) -> Result {
match availability {
Availability::Available(block) => {
// Block is fully available, import into fork choice
@@ -3379,7 +3545,7 @@ impl BeaconChain {
pub async fn import_available_block(
self: &Arc,
block: Box>,
- ) -> Result> {
+ ) -> Result {
let AvailableExecutedBlock {
block,
import_data,
@@ -3404,6 +3570,8 @@ impl BeaconChain {
);
}
+ // TODO(das) record custody column available timestamp
+
// import
let chain = self.clone();
let block_root = self
@@ -3456,7 +3624,7 @@ impl BeaconChain {
parent_block: SignedBlindedBeaconBlock,
parent_eth1_finalization_data: Eth1FinalizationData,
mut consensus_context: ConsensusContext,
- ) -> Result> {
+ ) -> Result {
// ----------------------------- BLOCK NOT YET ATTESTABLE ----------------------------------
// Everything in this initial section is on the hot path between processing the block and
// being able to attest to it. DO NOT add any extra processing in this initial section
@@ -3514,8 +3682,6 @@ impl BeaconChain {
// Register the new block with the fork choice service.
{
- let _fork_choice_block_timer =
- metrics::start_timer(&metrics::FORK_CHOICE_PROCESS_BLOCK_TIMES);
let block_delay = self
.slot_clock
.seconds_from_current_slot_start()
@@ -3648,16 +3814,15 @@ impl BeaconChain {
}
}
- if let Some(_data_columns) = data_columns {
- // TODO(das): depends on https://github.com/sigp/lighthouse/pull/6073
- // if !data_columns.is_empty() {
- // debug!(
- // self.log, "Writing data_columns to store";
- // "block_root" => %block_root,
- // "count" => data_columns.len(),
- // );
- // ops.push(StoreOp::PutDataColumns(block_root, data_columns));
- // }
+ if let Some(data_columns) = data_columns {
+ if !data_columns.is_empty() {
+ debug!(
+ self.log, "Writing data_columns to store";
+ "block_root" => %block_root,
+ "count" => data_columns.len(),
+ );
+ ops.push(StoreOp::PutDataColumns(block_root, data_columns));
+ }
}
let txn_lock = self.store.hot_db.begin_rw_transaction();
@@ -3769,7 +3934,7 @@ impl BeaconChain {
block: BeaconBlockRef,
block_root: Hash256,
state: &BeaconState,
- ) -> Result<(), BlockError> {
+ ) -> Result<(), BlockError> {
// Only perform the weak subjectivity check if it was configured.
let Some(wss_checkpoint) = self.config.weak_subjectivity_checkpoint else {
return Ok(());
@@ -4104,7 +4269,7 @@ impl BeaconChain {
&self,
block_root: Hash256,
state: &mut BeaconState,
- ) -> Result<(), BlockError> {
+ ) -> Result<(), BlockError> {
for relative_epoch in [RelativeEpoch::Current, RelativeEpoch::Next] {
let shuffling_id = AttestationShufflingId::new(block_root, state, relative_epoch)?;
@@ -5260,7 +5425,7 @@ impl BeaconChain {
},
}),
None,
- Uint256::zero(),
+ Uint256::ZERO,
),
BeaconState::Altair(_) => (
BeaconBlock::Altair(BeaconBlockAltair {
@@ -5283,7 +5448,7 @@ impl BeaconChain {
},
}),
None,
- Uint256::zero(),
+ Uint256::ZERO,
),
BeaconState::Bellatrix(_) => {
let block_proposal_contents =
@@ -5416,8 +5581,6 @@ impl BeaconChain {
bls_to_execution_changes: bls_to_execution_changes.into(),
blob_kzg_commitments: kzg_commitments
.ok_or(BlockProductionError::InvalidPayloadFork)?,
- // TODO(electra): finish consolidations when they're more spec'd out
- consolidations: Vec::new().into(),
},
}),
maybe_blobs_and_proofs,
@@ -6764,6 +6927,33 @@ impl BeaconChain {
self.data_availability_checker.data_availability_boundary()
}
+ /// Returns true if epoch is within the data availability boundary
+ pub fn da_check_required_for_epoch(&self, epoch: Epoch) -> bool {
+ self.data_availability_checker
+ .da_check_required_for_epoch(epoch)
+ }
+
+ /// Returns true if we should fetch blobs for this block
+ pub fn should_fetch_blobs(&self, block_epoch: Epoch) -> bool {
+ self.da_check_required_for_epoch(block_epoch)
+ && !self.spec.is_peer_das_enabled_for_epoch(block_epoch)
+ }
+
+ /// Returns true if we should fetch custody columns for this block
+ pub fn should_fetch_custody_columns(&self, block_epoch: Epoch) -> bool {
+ self.da_check_required_for_epoch(block_epoch)
+ && self.spec.is_peer_das_enabled_for_epoch(block_epoch)
+ }
+
+ /// Returns true if we should issue a sampling request for this block
+ /// TODO(das): check if the block is still within the da_window
+ pub fn should_sample_slot(&self, slot: Slot) -> bool {
+ self.config.enable_sampling
+ && self
+ .spec
+ .is_peer_das_enabled_for_epoch(slot.epoch(T::EthSpec::slots_per_epoch()))
+ }
+
pub fn logger(&self) -> &Logger {
&self.log
}
@@ -6852,8 +7042,8 @@ impl From for Error {
}
}
-impl ChainSegmentResult {
- pub fn into_block_error(self) -> Result<(), BlockError> {
+impl ChainSegmentResult {
+ pub fn into_block_error(self) -> Result<(), BlockError> {
match self {
ChainSegmentResult::Failed { error, .. } => Err(error),
ChainSegmentResult::Successful { .. } => Ok(()),
diff --git a/beacon_node/beacon_chain/src/beacon_fork_choice_store.rs b/beacon_node/beacon_chain/src/beacon_fork_choice_store.rs
index f746b68996f..a6aedda19d0 100644
--- a/beacon_node/beacon_chain/src/beacon_fork_choice_store.rs
+++ b/beacon_node/beacon_chain/src/beacon_fork_choice_store.rs
@@ -17,7 +17,7 @@ use store::{Error as StoreError, HotColdDB, ItemStore};
use superstruct::superstruct;
use types::{
AbstractExecPayload, BeaconBlockRef, BeaconState, BeaconStateError, Checkpoint, Epoch, EthSpec,
- Hash256, Slot,
+ FixedBytesExtended, Hash256, Slot,
};
#[derive(Debug)]
diff --git a/beacon_node/beacon_chain/src/bellatrix_readiness.rs b/beacon_node/beacon_chain/src/bellatrix_readiness.rs
index 60b1abaf098..c2e387c422f 100644
--- a/beacon_node/beacon_chain/src/bellatrix_readiness.rs
+++ b/beacon_node/beacon_chain/src/bellatrix_readiness.rs
@@ -64,7 +64,7 @@ impl MergeConfig {
/// Instantiate `self` from the values in a `ChainSpec`.
pub fn from_chainspec(spec: &ChainSpec) -> Self {
let mut params = MergeConfig::default();
- if spec.terminal_total_difficulty != Uint256::max_value() {
+ if spec.terminal_total_difficulty != Uint256::MAX {
params.terminal_total_difficulty = Some(spec.terminal_total_difficulty);
}
if spec.terminal_block_hash != ExecutionBlockHash::zero() {
diff --git a/beacon_node/beacon_chain/src/blob_verification.rs b/beacon_node/beacon_chain/src/blob_verification.rs
index bba58675933..e4646d62882 100644
--- a/beacon_node/beacon_chain/src/blob_verification.rs
+++ b/beacon_node/beacon_chain/src/blob_verification.rs
@@ -22,7 +22,7 @@ use types::{
/// An error occurred while validating a gossip blob.
#[derive(Debug)]
-pub enum GossipBlobError {
+pub enum GossipBlobError {
/// The blob sidecar is from a slot that is later than the current slot (with respect to the
/// gossip clock disparity).
///
@@ -95,7 +95,7 @@ pub enum GossipBlobError {
/// ## Peer scoring
///
/// We cannot process the blob without validating its parent, the peer isn't necessarily faulty.
- BlobParentUnknown(Arc>),
+ BlobParentUnknown { parent_root: Hash256 },
/// Invalid kzg commitment inclusion proof
/// ## Peer scoring
@@ -145,28 +145,19 @@ pub enum GossipBlobError {
NotFinalizedDescendant { block_parent_root: Hash256 },
}
-impl std::fmt::Display for GossipBlobError {
+impl std::fmt::Display for GossipBlobError {
fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
- match self {
- GossipBlobError::BlobParentUnknown(blob_sidecar) => {
- write!(
- f,
- "BlobParentUnknown(parent_root:{})",
- blob_sidecar.block_parent_root()
- )
- }
- other => write!(f, "{:?}", other),
- }
+ write!(f, "{:?}", self)
}
}
-impl From for GossipBlobError {
+impl From for GossipBlobError {
fn from(e: BeaconChainError) -> Self {
GossipBlobError::BeaconChainError(e)
}
}
-impl From for GossipBlobError {
+impl From for GossipBlobError {
fn from(e: BeaconStateError) -> Self {
GossipBlobError::BeaconChainError(BeaconChainError::BeaconStateError(e))
}
@@ -190,12 +181,12 @@ impl GossipVerifiedBlob {
blob: Arc>,
subnet_id: u64,
chain: &BeaconChain,
- ) -> Result> {
+ ) -> Result {
let header = blob.signed_block_header.clone();
// We only process slashing info if the gossip verification failed
// since we do not process the blob any further in that case.
validate_blob_sidecar_for_gossip(blob, subnet_id, chain).map_err(|e| {
- process_block_slash_info::<_, GossipBlobError>(
+ process_block_slash_info::<_, GossipBlobError>(
chain,
BlockSlashInfo::from_early_error_blob(header, e),
)
@@ -339,16 +330,16 @@ impl KzgVerifiedBlobList {
kzg: &Kzg,
seen_timestamp: Duration,
) -> Result {
- let blobs = blob_list.into_iter().collect::>();
- verify_kzg_for_blob_list(blobs.iter(), kzg)?;
+ let blobs = blob_list
+ .into_iter()
+ .map(|blob| KzgVerifiedBlob {
+ blob,
+ seen_timestamp,
+ })
+ .collect::>();
+ verify_kzg_for_blob_list(blobs.iter().map(|b| &b.blob), kzg)?;
Ok(Self {
- verified_blobs: blobs
- .into_iter()
- .map(|blob| KzgVerifiedBlob {
- blob,
- seen_timestamp,
- })
- .collect(),
+ verified_blobs: blobs,
})
}
}
@@ -384,7 +375,7 @@ pub fn validate_blob_sidecar_for_gossip(
blob_sidecar: Arc>,
subnet: u64,
chain: &BeaconChain,
-) -> Result, GossipBlobError> {
+) -> Result, GossipBlobError> {
let blob_slot = blob_sidecar.slot();
let blob_index = blob_sidecar.index;
let block_parent_root = blob_sidecar.block_parent_root();
@@ -409,8 +400,8 @@ pub fn validate_blob_sidecar_for_gossip(
// Verify that the blob_sidecar was received on the correct subnet.
if blob_index != subnet {
return Err(GossipBlobError::InvalidSubnet {
- expected: blob_index,
- received: subnet,
+ expected: subnet,
+ received: blob_index,
});
}
@@ -466,7 +457,9 @@ pub fn validate_blob_sidecar_for_gossip(
// We have already verified that the blob is past finalization, so we can
// just check fork choice for the block's parent.
let Some(parent_block) = fork_choice.get_block(&block_parent_root) else {
- return Err(GossipBlobError::BlobParentUnknown(blob_sidecar));
+ return Err(GossipBlobError::BlobParentUnknown {
+ parent_root: block_parent_root,
+ });
};
// Do not process a blob that does not descend from the finalized root.
@@ -516,7 +509,7 @@ pub fn validate_blob_sidecar_for_gossip(
))
})?;
- let state = cheap_state_advance_to_obtain_committees::<_, GossipBlobError>(
+ let state = cheap_state_advance_to_obtain_committees::<_, GossipBlobError>(
&mut parent_state,
Some(parent_state_root),
blob_slot,
@@ -570,8 +563,9 @@ pub fn validate_blob_sidecar_for_gossip(
.kzg
.as_ref()
.ok_or(GossipBlobError::KzgNotInitialized)?;
- let kzg_verified_blob = KzgVerifiedBlob::new(blob_sidecar.clone(), kzg, seen_timestamp)
+ let kzg_verified_blob = KzgVerifiedBlob::new(blob_sidecar, kzg, seen_timestamp)
.map_err(GossipBlobError::KzgError)?;
+ let blob_sidecar = &kzg_verified_blob.blob;
chain
.observed_slashable
@@ -597,7 +591,7 @@ pub fn validate_blob_sidecar_for_gossip(
if chain
.observed_blob_sidecars
.write()
- .observe_sidecar(&blob_sidecar)
+ .observe_sidecar(blob_sidecar)
.map_err(|e| GossipBlobError::BeaconChainError(e.into()))?
{
return Err(GossipBlobError::RepeatBlob {
diff --git a/beacon_node/beacon_chain/src/block_times_cache.rs b/beacon_node/beacon_chain/src/block_times_cache.rs
index 3b75046f3a4..af122ccdc06 100644
--- a/beacon_node/beacon_chain/src/block_times_cache.rs
+++ b/beacon_node/beacon_chain/src/block_times_cache.rs
@@ -294,6 +294,7 @@ impl BlockTimesCache {
#[cfg(test)]
mod test {
use super::*;
+ use types::FixedBytesExtended;
#[test]
fn observed_time_uses_minimum() {
diff --git a/beacon_node/beacon_chain/src/block_verification.rs b/beacon_node/beacon_chain/src/block_verification.rs
index 832eaccc803..8bd93a3753c 100644
--- a/beacon_node/beacon_chain/src/block_verification.rs
+++ b/beacon_node/beacon_chain/src/block_verification.rs
@@ -49,17 +49,20 @@
#![allow(clippy::result_large_err)]
use crate::beacon_snapshot::PreProcessingSnapshot;
-use crate::blob_verification::{GossipBlobError, GossipVerifiedBlob};
+use crate::blob_verification::{GossipBlobError, GossipVerifiedBlob, GossipVerifiedBlobList};
use crate::block_verification_types::{
AsBlock, BlockContentsError, BlockImportData, GossipVerifiedBlockContents, RpcBlock,
};
use crate::data_availability_checker::{AvailabilityCheckError, MaybeAvailableBlock};
-use crate::data_column_verification::GossipDataColumnError;
+use crate::data_column_verification::{
+ GossipDataColumnError, GossipVerifiedDataColumn, GossipVerifiedDataColumnList,
+};
use crate::eth1_finalization_cache::Eth1FinalizationData;
use crate::execution_payload::{
is_optimistic_candidate_block, validate_execution_payload_for_gossip, validate_merge_block,
AllowOptimisticImport, NotifyExecutionLayer, PayloadNotifier,
};
+use crate::kzg_utils::blobs_to_data_column_sidecars;
use crate::observed_block_producers::SeenBlock;
use crate::validator_monitor::HISTORIC_EPOCHS as VALIDATOR_MONITOR_HISTORIC_EPOCHS;
use crate::validator_pubkey_cache::ValidatorPubkeyCache;
@@ -94,10 +97,12 @@ use std::io::Write;
use std::sync::Arc;
use store::{Error as DBError, HotStateSummary, KeyValueStore, StoreOp};
use task_executor::JoinHandle;
+use types::data_column_sidecar::DataColumnSidecarError;
use types::{
- BeaconBlockRef, BeaconState, BeaconStateError, ChainSpec, Epoch, EthSpec, ExecutionBlockHash,
- Hash256, InconsistentFork, PublicKey, PublicKeyBytes, RelativeEpoch, SignedBeaconBlock,
- SignedBeaconBlockHeader, Slot,
+ BeaconBlockRef, BeaconState, BeaconStateError, BlobsList, ChainSpec, DataColumnSubnetId, Epoch,
+ EthSpec, ExecutionBlockHash, FullPayload, Hash256, InconsistentFork, KzgProofs, PublicKey,
+ PublicKeyBytes, RelativeEpoch, RuntimeVariableList, SignedBeaconBlock, SignedBeaconBlockHeader,
+ Slot,
};
use types::{BlobSidecar, ExecPayload};
@@ -139,14 +144,14 @@ const WRITE_BLOCK_PROCESSING_SSZ: bool = cfg!(feature = "write_ssz_files");
/// - The block is malformed/invalid (indicated by all results other than `BeaconChainError`.
/// - We encountered an error whilst trying to verify the block (a `BeaconChainError`).
#[derive(Debug)]
-pub enum BlockError {
+pub enum BlockError {
/// The parent block was unknown.
///
/// ## Peer scoring
///
/// It's unclear if this block is valid, but it cannot be processed without already knowing
/// its parent.
- ParentUnknown(RpcBlock),
+ ParentUnknown { parent_root: Hash256 },
/// The block slot is greater than the present slot.
///
/// ## Peer scoring
@@ -306,6 +311,14 @@ pub enum BlockError {
/// TODO: We may need to penalize the peer that gave us a potentially invalid rpc blob.
/// https://github.com/sigp/lighthouse/issues/4546
AvailabilityCheck(AvailabilityCheckError),
+ /// A Blob with a slot after PeerDAS is received and is not required to be imported.
+ /// This can happen because we stay subscribed to the blob subnet after 2 epochs, as we could
+ /// still receive valid blobs from a Deneb epoch after PeerDAS is activated.
+ ///
+ /// ## Peer scoring
+ ///
+ /// This indicates the peer is sending an unexpected gossip blob and should be penalised.
+ BlobNotRequired(Slot),
/// An internal error has occurred when processing the block or sidecars.
///
/// ## Peer scoring
@@ -315,7 +328,7 @@ pub enum BlockError {
InternalError(String),
}
-impl From for BlockError {
+impl From for BlockError {
fn from(e: AvailabilityCheckError) -> Self {
Self::AvailabilityCheck(e)
}
@@ -423,30 +436,25 @@ impl From for ExecutionPayloadError {
}
}
-impl From for BlockError {
+impl From for BlockError {
fn from(e: ExecutionPayloadError) -> Self {
BlockError::ExecutionPayloadError(e)
}
}
-impl From for BlockError {
+impl From for BlockError {
fn from(e: InconsistentFork) -> Self {
BlockError::InconsistentFork(e)
}
}
-impl std::fmt::Display for BlockError