Skip to content

Commit 056f6c5

Browse files
authored
Merge of #6744
2 parents b5d40e3 + 8f31510 commit 056f6c5

27 files changed

+1111
-984
lines changed

beacon_node/beacon_chain/src/beacon_chain.rs

Lines changed: 15 additions & 77 deletions
Original file line numberDiff line numberDiff line change
@@ -33,7 +33,6 @@ use crate::events::ServerSentEventHandler;
3333
use crate::execution_payload::{get_execution_payload, NotifyExecutionLayer, PreparePayloadHandle};
3434
use crate::fork_choice_signal::{ForkChoiceSignalRx, ForkChoiceSignalTx, ForkChoiceWaitResult};
3535
use crate::graffiti_calculator::GraffitiCalculator;
36-
use crate::head_tracker::{HeadTracker, HeadTrackerReader, SszHeadTracker};
3736
use crate::kzg_utils::reconstruct_blobs;
3837
use crate::light_client_finality_update_verification::{
3938
Error as LightClientFinalityUpdateError, VerifiedLightClientFinalityUpdate,
@@ -57,7 +56,7 @@ use crate::observed_block_producers::ObservedBlockProducers;
5756
use crate::observed_data_sidecars::ObservedDataSidecars;
5857
use crate::observed_operations::{ObservationOutcome, ObservedOperations};
5958
use crate::observed_slashable::ObservedSlashable;
60-
use crate::persisted_beacon_chain::{PersistedBeaconChain, DUMMY_CANONICAL_HEAD_BLOCK_ROOT};
59+
use crate::persisted_beacon_chain::PersistedBeaconChain;
6160
use crate::persisted_fork_choice::PersistedForkChoice;
6261
use crate::pre_finalization_cache::PreFinalizationBlockCache;
6362
use crate::shuffling_cache::{BlockShufflingIds, ShufflingCache};
@@ -454,8 +453,6 @@ pub struct BeaconChain<T: BeaconChainTypes> {
454453
/// A handler for events generated by the beacon chain. This is only initialized when the
455454
/// HTTP server is enabled.
456455
pub event_handler: Option<ServerSentEventHandler<T::EthSpec>>,
457-
/// Used to track the heads of the beacon chain.
458-
pub(crate) head_tracker: Arc<HeadTracker>,
459456
/// Caches the attester shuffling for a given epoch and shuffling key root.
460457
pub shuffling_cache: RwLock<ShufflingCache>,
461458
/// A cache of eth1 deposit data at epoch boundaries for deposit finalization
@@ -607,57 +604,13 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
607604
})
608605
}
609606

610-
/// Persists the head tracker and fork choice.
607+
/// Return a database operation for writing the `PersistedBeaconChain` to disk.
611608
///
612-
/// We do it atomically even though no guarantees need to be made about blocks from
613-
/// the head tracker also being present in fork choice.
614-
pub fn persist_head_and_fork_choice(&self) -> Result<(), Error> {
615-
let mut batch = vec![];
616-
617-
let _head_timer = metrics::start_timer(&metrics::PERSIST_HEAD);
618-
619-
// Hold a lock to head_tracker until it has been persisted to disk. Otherwise there's a race
620-
// condition with the pruning thread which can result in a block present in the head tracker
621-
// but absent in the DB. This inconsistency halts pruning and dramastically increases disk
622-
// size. Ref: https://github.com/sigp/lighthouse/issues/4773
623-
let head_tracker = self.head_tracker.0.read();
624-
batch.push(self.persist_head_in_batch(&head_tracker));
625-
626-
let _fork_choice_timer = metrics::start_timer(&metrics::PERSIST_FORK_CHOICE);
627-
batch.push(self.persist_fork_choice_in_batch());
628-
629-
self.store.hot_db.do_atomically(batch)?;
630-
drop(head_tracker);
631-
632-
Ok(())
633-
}
634-
635-
/// Return a `PersistedBeaconChain` without reference to a `BeaconChain`.
636-
pub fn make_persisted_head(
637-
genesis_block_root: Hash256,
638-
head_tracker_reader: &HeadTrackerReader,
639-
) -> PersistedBeaconChain {
640-
PersistedBeaconChain {
641-
_canonical_head_block_root: DUMMY_CANONICAL_HEAD_BLOCK_ROOT,
642-
genesis_block_root,
643-
ssz_head_tracker: SszHeadTracker::from_map(head_tracker_reader),
644-
}
645-
}
646-
647-
/// Return a database operation for writing the beacon chain head to disk.
648-
pub fn persist_head_in_batch(
649-
&self,
650-
head_tracker_reader: &HeadTrackerReader,
651-
) -> KeyValueStoreOp {
652-
Self::persist_head_in_batch_standalone(self.genesis_block_root, head_tracker_reader)
653-
}
654-
655-
pub fn persist_head_in_batch_standalone(
656-
genesis_block_root: Hash256,
657-
head_tracker_reader: &HeadTrackerReader,
658-
) -> KeyValueStoreOp {
659-
Self::make_persisted_head(genesis_block_root, head_tracker_reader)
660-
.as_kv_store_op(BEACON_CHAIN_DB_KEY)
609+
/// These days the `PersistedBeaconChain` is only used to store the genesis block root, so it
610+
/// should only ever be written once at startup. It used to be written more frequently, but
611+
/// this is no longer necessary.
612+
pub fn persist_head_in_batch_standalone(genesis_block_root: Hash256) -> KeyValueStoreOp {
613+
PersistedBeaconChain { genesis_block_root }.as_kv_store_op(BEACON_CHAIN_DB_KEY)
661614
}
662615

663616
/// Load fork choice from disk, returning `None` if it isn't found.
@@ -1450,12 +1403,13 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
14501403
///
14511404
/// Returns `(block_root, block_slot)`.
14521405
pub fn heads(&self) -> Vec<(Hash256, Slot)> {
1453-
self.head_tracker.heads()
1454-
}
1455-
1456-
/// Only used in tests.
1457-
pub fn knows_head(&self, block_hash: &SignedBeaconBlockHash) -> bool {
1458-
self.head_tracker.contains_head((*block_hash).into())
1406+
self.canonical_head
1407+
.fork_choice_read_lock()
1408+
.proto_array()
1409+
.heads_descended_from_finalization::<T::EthSpec>()
1410+
.iter()
1411+
.map(|node| (node.root, node.slot))
1412+
.collect()
14591413
}
14601414

14611415
/// Returns the `BeaconState` at the given slot.
@@ -1735,8 +1689,6 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
17351689
let notif = ManualFinalizationNotification {
17361690
state_root: state_root.into(),
17371691
checkpoint,
1738-
head_tracker: self.head_tracker.clone(),
1739-
genesis_block_root: self.genesis_block_root,
17401692
};
17411693

17421694
self.store_migrator.process_manual_finalization(notif);
@@ -3762,7 +3714,6 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
37623714
state,
37633715
parent_block,
37643716
parent_eth1_finalization_data,
3765-
confirmed_state_roots,
37663717
consensus_context,
37673718
} = import_data;
37683719

@@ -3786,7 +3737,6 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
37863737
block,
37873738
block_root,
37883739
state,
3789-
confirmed_state_roots,
37903740
payload_verification_outcome.payload_verification_status,
37913741
parent_block,
37923742
parent_eth1_finalization_data,
@@ -3824,7 +3774,6 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
38243774
signed_block: AvailableBlock<T::EthSpec>,
38253775
block_root: Hash256,
38263776
mut state: BeaconState<T::EthSpec>,
3827-
confirmed_state_roots: Vec<Hash256>,
38283777
payload_verification_status: PayloadVerificationStatus,
38293778
parent_block: SignedBlindedBeaconBlock<T::EthSpec>,
38303779
parent_eth1_finalization_data: Eth1FinalizationData,
@@ -4012,11 +3961,6 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
40123961

40133962
let block = signed_block.message();
40143963
let db_write_timer = metrics::start_timer(&metrics::BLOCK_PROCESSING_DB_WRITE);
4015-
ops.extend(
4016-
confirmed_state_roots
4017-
.into_iter()
4018-
.map(StoreOp::DeleteStateTemporaryFlag),
4019-
);
40203964
ops.push(StoreOp::PutBlock(block_root, signed_block.clone()));
40213965
ops.push(StoreOp::PutState(block.state_root(), &state));
40223966

@@ -4043,9 +3987,6 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
40433987
// about it.
40443988
let block_time_imported = timestamp_now();
40453989

4046-
let parent_root = block.parent_root();
4047-
let slot = block.slot();
4048-
40493990
let current_eth1_finalization_data = Eth1FinalizationData {
40503991
eth1_data: state.eth1_data().clone(),
40513992
eth1_deposit_index: state.eth1_deposit_index(),
@@ -4066,9 +4007,6 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
40664007
});
40674008
}
40684009

4069-
self.head_tracker
4070-
.register_block(block_root, parent_root, slot);
4071-
40724010
metrics::stop_timer(db_write_timer);
40734011

40744012
metrics::inc_counter(&metrics::BLOCK_PROCESSING_SUCCESSES);
@@ -7208,7 +7146,7 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
72087146
impl<T: BeaconChainTypes> Drop for BeaconChain<T> {
72097147
fn drop(&mut self) {
72107148
let drop = || -> Result<(), Error> {
7211-
self.persist_head_and_fork_choice()?;
7149+
self.persist_fork_choice()?;
72127150
self.persist_op_pool()?;
72137151
self.persist_eth1_cache()
72147152
};

beacon_node/beacon_chain/src/block_verification.rs

Lines changed: 10 additions & 34 deletions
Original file line numberDiff line numberDiff line change
@@ -1453,22 +1453,8 @@ impl<T: BeaconChainTypes> ExecutionPendingBlock<T> {
14531453

14541454
let catchup_timer = metrics::start_timer(&metrics::BLOCK_PROCESSING_CATCHUP_STATE);
14551455

1456-
// Stage a batch of operations to be completed atomically if this block is imported
1457-
// successfully. If there is a skipped slot, we include the state root of the pre-state,
1458-
// which may be an advanced state that was stored in the DB with a `temporary` flag.
14591456
let mut state = parent.pre_state;
14601457

1461-
let mut confirmed_state_roots =
1462-
if block.slot() > state.slot() && state.slot() > parent.beacon_block.slot() {
1463-
// Advanced pre-state. Delete its temporary flag.
1464-
let pre_state_root = state.update_tree_hash_cache()?;
1465-
vec![pre_state_root]
1466-
} else {
1467-
// Pre state is either unadvanced, or should not be stored long-term because there
1468-
// is no skipped slot between `parent` and `block`.
1469-
vec![]
1470-
};
1471-
14721458
// The block must have a higher slot than its parent.
14731459
if block.slot() <= parent.beacon_block.slot() {
14741460
return Err(BlockError::BlockIsNotLaterThanParent {
@@ -1515,38 +1501,29 @@ impl<T: BeaconChainTypes> ExecutionPendingBlock<T> {
15151501
// processing, but we get early access to it.
15161502
let state_root = state.update_tree_hash_cache()?;
15171503

1518-
// Store the state immediately, marking it as temporary, and staging the deletion
1519-
// of its temporary status as part of the larger atomic operation.
1504+
// Store the state immediately.
15201505
let txn_lock = chain.store.hot_db.begin_rw_transaction();
15211506
let state_already_exists =
15221507
chain.store.load_hot_state_summary(&state_root)?.is_some();
15231508

15241509
let state_batch = if state_already_exists {
1525-
// If the state exists, it could be temporary or permanent, but in neither case
1526-
// should we rewrite it or store a new temporary flag for it. We *will* stage
1527-
// the temporary flag for deletion because it's OK to double-delete the flag,
1528-
// and we don't mind if another thread gets there first.
1510+
// If the state exists, we do not need to re-write it.
15291511
vec![]
15301512
} else {
1531-
vec![
1532-
if state.slot() % T::EthSpec::slots_per_epoch() == 0 {
1533-
StoreOp::PutState(state_root, &state)
1534-
} else {
1535-
StoreOp::PutStateSummary(
1536-
state_root,
1537-
HotStateSummary::new(&state_root, &state)?,
1538-
)
1539-
},
1540-
StoreOp::PutStateTemporaryFlag(state_root),
1541-
]
1513+
vec![if state.slot() % T::EthSpec::slots_per_epoch() == 0 {
1514+
StoreOp::PutState(state_root, &state)
1515+
} else {
1516+
StoreOp::PutStateSummary(
1517+
state_root,
1518+
HotStateSummary::new(&state_root, &state)?,
1519+
)
1520+
}]
15421521
};
15431522
chain
15441523
.store
15451524
.do_atomically_with_block_and_blobs_cache(state_batch)?;
15461525
drop(txn_lock);
15471526

1548-
confirmed_state_roots.push(state_root);
1549-
15501527
state_root
15511528
};
15521529

@@ -1713,7 +1690,6 @@ impl<T: BeaconChainTypes> ExecutionPendingBlock<T> {
17131690
state,
17141691
parent_block: parent.beacon_block,
17151692
parent_eth1_finalization_data,
1716-
confirmed_state_roots,
17171693
consensus_context,
17181694
},
17191695
payload_verification_handle,

beacon_node/beacon_chain/src/block_verification_types.rs

Lines changed: 0 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -358,7 +358,6 @@ pub struct BlockImportData<E: EthSpec> {
358358
pub state: BeaconState<E>,
359359
pub parent_block: SignedBeaconBlock<E, BlindedPayload<E>>,
360360
pub parent_eth1_finalization_data: Eth1FinalizationData,
361-
pub confirmed_state_roots: Vec<Hash256>,
362361
pub consensus_context: ConsensusContext<E>,
363362
}
364363

@@ -376,7 +375,6 @@ impl<E: EthSpec> BlockImportData<E> {
376375
eth1_data: <_>::default(),
377376
eth1_deposit_index: 0,
378377
},
379-
confirmed_state_roots: vec![],
380378
consensus_context: ConsensusContext::new(Slot::new(0)),
381379
}
382380
}

beacon_node/beacon_chain/src/builder.rs

Lines changed: 2 additions & 16 deletions
Original file line numberDiff line numberDiff line change
@@ -8,7 +8,6 @@ use crate::eth1_finalization_cache::Eth1FinalizationCache;
88
use crate::fork_choice_signal::ForkChoiceSignalTx;
99
use crate::fork_revert::{reset_fork_choice_to_finalization, revert_to_fork_boundary};
1010
use crate::graffiti_calculator::{GraffitiCalculator, GraffitiOrigin};
11-
use crate::head_tracker::HeadTracker;
1211
use crate::kzg_utils::blobs_to_data_column_sidecars;
1312
use crate::light_client_server_cache::LightClientServerCache;
1413
use crate::migrate::{BackgroundMigrator, MigratorConfig};
@@ -93,7 +92,6 @@ pub struct BeaconChainBuilder<T: BeaconChainTypes> {
9392
slot_clock: Option<T::SlotClock>,
9493
shutdown_sender: Option<Sender<ShutdownReason>>,
9594
light_client_server_tx: Option<Sender<LightClientProducerEvent<T::EthSpec>>>,
96-
head_tracker: Option<HeadTracker>,
9795
validator_pubkey_cache: Option<ValidatorPubkeyCache<T>>,
9896
spec: Arc<ChainSpec>,
9997
chain_config: ChainConfig,
@@ -136,7 +134,6 @@ where
136134
slot_clock: None,
137135
shutdown_sender: None,
138136
light_client_server_tx: None,
139-
head_tracker: None,
140137
validator_pubkey_cache: None,
141138
spec: Arc::new(E::default_spec()),
142139
chain_config: ChainConfig::default(),
@@ -314,10 +311,6 @@ where
314311

315312
self.genesis_block_root = Some(chain.genesis_block_root);
316313
self.genesis_state_root = Some(genesis_block.state_root());
317-
self.head_tracker = Some(
318-
HeadTracker::from_ssz_container(&chain.ssz_head_tracker)
319-
.map_err(|e| format!("Failed to decode head tracker for database: {:?}", e))?,
320-
);
321314
self.validator_pubkey_cache = Some(pubkey_cache);
322315
self.fork_choice = Some(fork_choice);
323316

@@ -729,7 +722,6 @@ where
729722
.genesis_state_root
730723
.ok_or("Cannot build without a genesis state root")?;
731724
let validator_monitor_config = self.validator_monitor_config.unwrap_or_default();
732-
let head_tracker = Arc::new(self.head_tracker.unwrap_or_default());
733725
let beacon_proposer_cache: Arc<Mutex<BeaconProposerCache>> = <_>::default();
734726

735727
let mut validator_monitor =
@@ -769,8 +761,6 @@ where
769761
&self.spec,
770762
)?;
771763

772-
// Update head tracker.
773-
head_tracker.register_block(block_root, block.parent_root(), block.slot());
774764
(block_root, block, true)
775765
}
776766
Err(e) => return Err(descriptive_db_error("head block", &e)),
@@ -846,8 +836,7 @@ where
846836
})?;
847837

848838
let migrator_config = self.store_migrator_config.unwrap_or_default();
849-
let store_migrator =
850-
BackgroundMigrator::new(store.clone(), migrator_config, genesis_block_root);
839+
let store_migrator = BackgroundMigrator::new(store.clone(), migrator_config);
851840

852841
if let Some(slot) = slot_clock.now() {
853842
validator_monitor.process_valid_state(
@@ -872,11 +861,10 @@ where
872861
//
873862
// This *must* be stored before constructing the `BeaconChain`, so that its `Drop` instance
874863
// doesn't write a `PersistedBeaconChain` without the rest of the batch.
875-
let head_tracker_reader = head_tracker.0.read();
876864
self.pending_io_batch.push(BeaconChain::<
877865
Witness<TSlotClock, TEth1Backend, E, THotStore, TColdStore>,
878866
>::persist_head_in_batch_standalone(
879-
genesis_block_root, &head_tracker_reader
867+
genesis_block_root
880868
));
881869
self.pending_io_batch.push(BeaconChain::<
882870
Witness<TSlotClock, TEth1Backend, E, THotStore, TColdStore>,
@@ -887,7 +875,6 @@ where
887875
.hot_db
888876
.do_atomically(self.pending_io_batch)
889877
.map_err(|e| format!("Error writing chain & metadata to disk: {:?}", e))?;
890-
drop(head_tracker_reader);
891878

892879
let genesis_validators_root = head_snapshot.beacon_state.genesis_validators_root();
893880
let genesis_time = head_snapshot.beacon_state.genesis_time();
@@ -968,7 +955,6 @@ where
968955
fork_choice_signal_tx,
969956
fork_choice_signal_rx,
970957
event_handler: self.event_handler,
971-
head_tracker,
972958
shuffling_cache: RwLock::new(ShufflingCache::new(
973959
shuffling_cache_size,
974960
head_shuffling_ids,

0 commit comments

Comments
 (0)