lighthouse/beacon_node/beacon_chain/src/migrate.rs
Michael Sproul acd49d988d Implement database temp states to reduce memory usage (#1798)
## Issue Addressed

Closes #800
Closes #1713

## Proposed Changes

Implement the temporary state storage algorithm described in #800. Specifically:

* Add `DBColumn::BeaconStateTemporary`, for storing 0-length temporary marker values.
* Store intermediate states immediately as they are created, marked temporary. Delete the temporary flag if the block is processed successfully.
* Add a garbage collection process to delete leftover temporary states on start-up.
* Bump the database schema version to 2 so that a DB with temporary states can't accidentally be used with older versions of the software. The auto-migration is a no-op, but puts in place some infra that we can use for future migrations (e.g. #1784)

## Additional Info

There are two known race conditions, one potentially causing permanent faults (hopefully rare), and the other insignificant.

### Race 1: Permanent state marked temporary

EDIT: this has been fixed by the addition of a lock around the relevant critical section

There are 2 threads that are trying to store 2 different blocks that share some intermediate states (e.g. they both skip some slots from the current head). Consider this sequence of events:

1. Thread 1 checks if state `s` already exists, and seeing that it doesn't, prepares an atomic commit of `(s, s_temporary_flag)`.
2. Thread 2 does the same, but also gets as far as committing the state txn, finishing the processing of its block, and _deleting_ the temporary flag.
3. Thread 1 is (finally) scheduled again, and marks `s` as temporary with its transaction.
4.
    a) The process is killed, or thread 1's block fails verification and the temp flag is not deleted. This is a permanent failure! Any attempt to load state `s` will fail... hope it isn't on the main chain! Alternatively (4b) happens...
    b) Thread 1 finishes, and re-deletes the temporary flag. In this case the failure is transient, state `s` will disappear temporarily, but will come back once thread 1 finishes running.

I _hope_ that steps 1-3 only happen very rarely, and 4a even more rarely. It's hard to know

This once again begs the question of why we're using LevelDB (#483), when it clearly doesn't care about atomicity! A ham-fisted fix would be to wrap the hot and cold DBs in locks, which would bring us closer to how other DBs handle read-write transactions. E.g. [LMDB only allows one R/W transaction at a time](https://docs.rs/lmdb/0.8.0/lmdb/struct.Environment.html#method.begin_rw_txn).

### Race 2: Temporary state returned from `get_state`

I don't think this race really matters, but in `load_hot_state`, if another thread stores a state between when we call `load_state_temporary_flag` and when we call `load_hot_state_summary`, then we could end up returning that state even though it's only a temporary state. I can't think of any case where this would be relevant, and I suspect if it did come up, it would be safe/recoverable (having data is safer than _not_ having data).

This could be fixed by using a LevelDB read snapshot, but that would require substantial changes to how we read all our values, so I don't think it's worth it right now.
2020-10-23 01:27:51 +00:00

466 lines
18 KiB
Rust

use crate::beacon_chain::BEACON_CHAIN_DB_KEY;
use crate::errors::BeaconChainError;
use crate::head_tracker::{HeadTracker, SszHeadTracker};
use crate::persisted_beacon_chain::{PersistedBeaconChain, DUMMY_CANONICAL_HEAD_BLOCK_ROOT};
use parking_lot::Mutex;
use slog::{debug, warn, Logger};
use std::collections::{HashMap, HashSet};
use std::mem;
use std::sync::mpsc;
use std::sync::Arc;
use std::thread;
use store::hot_cold_store::{migrate_database, HotColdDBError};
use store::iter::RootsIterator;
use store::{Error, ItemStore, StoreItem, StoreOp};
pub use store::{HotColdDB, MemoryStore};
use types::{
BeaconState, BeaconStateError, BeaconStateHash, Checkpoint, Epoch, EthSpec, Hash256,
SignedBeaconBlockHash, Slot,
};
/// The background migrator runs a thread to perform pruning and migrate state from the hot
/// to the cold database.
pub struct BackgroundMigrator<E: EthSpec, Hot: ItemStore<E>, Cold: ItemStore<E>> {
db: Arc<HotColdDB<E, Hot, Cold>>,
#[allow(clippy::type_complexity)]
tx_thread: Option<
Mutex<(
mpsc::Sender<MigrationNotification<E>>,
thread::JoinHandle<()>,
)>,
>,
latest_checkpoint: Arc<Mutex<Checkpoint>>,
/// Genesis block root, for persisting the `PersistedBeaconChain`.
genesis_block_root: Hash256,
log: Logger,
}
#[derive(Debug, Default, Clone, PartialEq, Eq)]
pub struct MigratorConfig {
pub blocking: bool,
}
impl MigratorConfig {
pub fn blocking(mut self) -> Self {
self.blocking = true;
self
}
}
/// Pruning can be successful, or in rare cases deferred to a later point.
#[derive(Debug, Clone, Copy, PartialEq, Eq)]
pub enum PruningOutcome {
Successful,
DeferredConcurrentMutation,
}
/// Logic errors that can occur during pruning, none of these should ever happen.
#[derive(Debug)]
pub enum PruningError {
IncorrectFinalizedState {
state_slot: Slot,
new_finalized_slot: Slot,
},
MissingInfoForCanonicalChain {
slot: Slot,
},
UnexpectedEqualStateRoots,
UnexpectedUnequalStateRoots,
}
/// Message sent to the migration thread containing the information it needs to run.
pub struct MigrationNotification<E: EthSpec> {
finalized_state_root: BeaconStateHash,
finalized_state: BeaconState<E>,
finalized_checkpoint: Checkpoint,
head_tracker: Arc<HeadTracker>,
latest_checkpoint: Arc<Mutex<Checkpoint>>,
genesis_block_root: Hash256,
}
impl<E: EthSpec, Hot: ItemStore<E>, Cold: ItemStore<E>> BackgroundMigrator<E, Hot, Cold> {
/// Create a new `BackgroundMigrator` and spawn its thread if necessary.
pub fn new(
db: Arc<HotColdDB<E, Hot, Cold>>,
config: MigratorConfig,
genesis_block_root: Hash256,
log: Logger,
) -> Self {
let tx_thread = if config.blocking {
None
} else {
Some(Mutex::new(Self::spawn_thread(db.clone(), log.clone())))
};
let latest_checkpoint = Arc::new(Mutex::new(Checkpoint {
root: Hash256::zero(),
epoch: Epoch::new(0),
}));
Self {
db,
tx_thread,
latest_checkpoint,
genesis_block_root,
log,
}
}
/// Process a finalized checkpoint from the `BeaconChain`.
///
/// If successful, all forks descending from before the `finalized_checkpoint` will be
/// pruned, and the split point of the database will be advanced to the slot of the finalized
/// checkpoint.
pub fn process_finalization(
&self,
finalized_state_root: BeaconStateHash,
finalized_state: BeaconState<E>,
finalized_checkpoint: Checkpoint,
head_tracker: Arc<HeadTracker>,
) -> Result<(), BeaconChainError> {
let notif = MigrationNotification {
finalized_state_root,
finalized_state,
finalized_checkpoint,
head_tracker,
latest_checkpoint: self.latest_checkpoint.clone(),
genesis_block_root: self.genesis_block_root,
};
// Async path, on the background thread.
if let Some(tx_thread) = &self.tx_thread {
let (ref mut tx, ref mut thread) = *tx_thread.lock();
// Restart the background thread if it has crashed.
if let Err(tx_err) = tx.send(notif) {
let (new_tx, new_thread) = Self::spawn_thread(self.db.clone(), self.log.clone());
*tx = new_tx;
let old_thread = mem::replace(thread, new_thread);
// Join the old thread, which will probably have panicked, or may have
// halted normally just now as a result of us dropping the old `mpsc::Sender`.
if let Err(thread_err) = old_thread.join() {
warn!(
self.log,
"Migration thread died, so it was restarted";
"reason" => format!("{:?}", thread_err)
);
}
// Retry at most once, we could recurse but that would risk overflowing the stack.
let _ = tx.send(tx_err.0);
}
}
// Synchronous path, on the current thread.
else {
Self::run_migration(self.db.clone(), notif, &self.log)
}
Ok(())
}
/// Perform the actual work of `process_finalization`.
fn run_migration(
db: Arc<HotColdDB<E, Hot, Cold>>,
notif: MigrationNotification<E>,
log: &Logger,
) {
let mut latest_checkpoint = notif.latest_checkpoint.lock();
let finalized_state_root = notif.finalized_state_root;
let finalized_state = notif.finalized_state;
match Self::prune_abandoned_forks(
db.clone(),
notif.head_tracker,
finalized_state_root,
&finalized_state,
*latest_checkpoint,
notif.finalized_checkpoint,
notif.genesis_block_root,
log,
) {
Ok(PruningOutcome::DeferredConcurrentMutation) => {
warn!(
log,
"Pruning deferred because of a concurrent mutation";
"message" => "this is expected only very rarely!"
);
return;
}
Ok(PruningOutcome::Successful) => {
// Update the migrator's idea of the latest checkpoint only if the
// pruning process was successful.
*latest_checkpoint = notif.finalized_checkpoint;
}
Err(e) => {
warn!(log, "Block pruning failed"; "error" => format!("{:?}", e));
return;
}
};
match migrate_database(db, finalized_state_root.into(), &finalized_state) {
Ok(()) => {}
Err(Error::HotColdDBError(HotColdDBError::FreezeSlotUnaligned(slot))) => {
debug!(
log,
"Database migration postponed, unaligned finalized block";
"slot" => slot.as_u64()
);
}
Err(e) => {
warn!(
log,
"Database migration failed";
"error" => format!("{:?}", e)
);
}
};
}
/// Spawn a new child thread to run the migration process.
///
/// Return a channel handle for sending new finalized states to the thread.
fn spawn_thread(
db: Arc<HotColdDB<E, Hot, Cold>>,
log: Logger,
) -> (
mpsc::Sender<MigrationNotification<E>>,
thread::JoinHandle<()>,
) {
let (tx, rx) = mpsc::channel();
let thread = thread::spawn(move || {
while let Ok(notif) = rx.recv() {
Self::run_migration(db.clone(), notif, &log);
}
});
(tx, thread)
}
/// Traverses live heads and prunes blocks and states of chains that we know can't be built
/// upon because finalization would prohibit it. This is an optimisation intended to save disk
/// space.
#[allow(clippy::too_many_arguments)]
fn prune_abandoned_forks(
store: Arc<HotColdDB<E, Hot, Cold>>,
head_tracker: Arc<HeadTracker>,
new_finalized_state_hash: BeaconStateHash,
new_finalized_state: &BeaconState<E>,
old_finalized_checkpoint: Checkpoint,
new_finalized_checkpoint: Checkpoint,
genesis_block_root: Hash256,
log: &Logger,
) -> Result<PruningOutcome, BeaconChainError> {
let old_finalized_slot = old_finalized_checkpoint
.epoch
.start_slot(E::slots_per_epoch());
let new_finalized_slot = new_finalized_checkpoint
.epoch
.start_slot(E::slots_per_epoch());
let new_finalized_block_hash = new_finalized_checkpoint.root.into();
// The finalized state must be for the epoch boundary slot, not the slot of the finalized
// block.
if new_finalized_state.slot != new_finalized_slot {
return Err(PruningError::IncorrectFinalizedState {
state_slot: new_finalized_state.slot,
new_finalized_slot,
}
.into());
}
debug!(
log,
"Starting database pruning";
"old_finalized_epoch" => old_finalized_checkpoint.epoch,
"old_finalized_root" => format!("{:?}", old_finalized_checkpoint.root),
"new_finalized_epoch" => new_finalized_checkpoint.epoch,
"new_finalized_root" => format!("{:?}", new_finalized_checkpoint.root),
);
// For each slot between the new finalized checkpoint and the old finalized checkpoint,
// collect the beacon block root and state root of the canonical chain.
let newly_finalized_chain: HashMap<Slot, (SignedBeaconBlockHash, BeaconStateHash)> =
std::iter::once(Ok((
new_finalized_slot,
(new_finalized_block_hash, new_finalized_state_hash),
)))
.chain(
RootsIterator::new(store.clone(), new_finalized_state).map(|res| {
res.map(|(block_root, state_root, slot)| {
(slot, (block_root.into(), state_root.into()))
})
}),
)
.take_while(|res| {
res.as_ref()
.map_or(true, |(slot, _)| *slot >= old_finalized_slot)
})
.collect::<Result<_, _>>()?;
// We don't know which blocks are shared among abandoned chains, so we buffer and delete
// everything in one fell swoop.
let mut abandoned_blocks: HashSet<SignedBeaconBlockHash> = HashSet::new();
let mut abandoned_states: HashSet<(Slot, BeaconStateHash)> = HashSet::new();
let mut abandoned_heads: HashSet<Hash256> = HashSet::new();
let heads = head_tracker.heads();
debug!(log, "Pruning {} heads", heads.len());
for (head_hash, head_slot) in heads {
let mut potentially_abandoned_head = Some(head_hash);
let mut potentially_abandoned_blocks = vec![];
let head_state_hash = store
.get_block(&head_hash)?
.ok_or_else(|| BeaconStateError::MissingBeaconBlock(head_hash.into()))?
.state_root();
// Iterate backwards from this head, staging blocks and states for deletion.
let iter = std::iter::once(Ok((head_hash, head_state_hash, head_slot)))
.chain(RootsIterator::from_block(store.clone(), head_hash)?);
for maybe_tuple in iter {
let (block_root, state_root, slot) = maybe_tuple?;
let block_root = SignedBeaconBlockHash::from(block_root);
let state_root = BeaconStateHash::from(state_root);
match newly_finalized_chain.get(&slot) {
// If there's no information about a slot on the finalized chain, then
// it should be because it's ahead of the new finalized slot. Stage
// the fork's block and state for possible deletion.
None => {
if slot > new_finalized_slot {
potentially_abandoned_blocks.push((
slot,
Some(block_root),
Some(state_root),
));
} else if slot >= old_finalized_slot {
return Err(PruningError::MissingInfoForCanonicalChain { slot }.into());
} else {
// We must assume here any candidate chains include the old finalized
// checkpoint, i.e. there aren't any forks starting at a block that is a
// strict ancestor of old_finalized_checkpoint.
warn!(
log,
"Found a chain that should already have been pruned";
"head_block_root" => format!("{:?}", head_hash),
"head_slot" => head_slot,
);
potentially_abandoned_head.take();
break;
}
}
Some((finalized_block_root, finalized_state_root)) => {
// This fork descends from a newly finalized block, we can stop.
if block_root == *finalized_block_root {
// Sanity check: if the slot and block root match, then the
// state roots should match too.
if state_root != *finalized_state_root {
return Err(PruningError::UnexpectedUnequalStateRoots.into());
}
// If the fork descends from the whole finalized chain,
// do not prune it. Otherwise continue to delete all
// of the blocks and states that have been staged for
// deletion so far.
if slot == new_finalized_slot {
potentially_abandoned_blocks.clear();
potentially_abandoned_head.take();
}
// If there are skipped slots on the fork to be pruned, then
// we will have just staged the common block for deletion.
// Unstage it.
else {
for (_, block_root, _) in
potentially_abandoned_blocks.iter_mut().rev()
{
if block_root.as_ref() == Some(finalized_block_root) {
*block_root = None;
} else {
break;
}
}
}
break;
} else {
if state_root == *finalized_state_root {
return Err(PruningError::UnexpectedEqualStateRoots.into());
}
potentially_abandoned_blocks.push((
slot,
Some(block_root),
Some(state_root),
));
}
}
}
}
if let Some(abandoned_head) = potentially_abandoned_head {
debug!(
log,
"Pruning head";
"head_block_root" => format!("{:?}", abandoned_head),
"head_slot" => head_slot,
);
abandoned_heads.insert(abandoned_head);
abandoned_blocks.extend(
potentially_abandoned_blocks
.iter()
.filter_map(|(_, maybe_block_hash, _)| *maybe_block_hash),
);
abandoned_states.extend(potentially_abandoned_blocks.iter().filter_map(
|(slot, _, maybe_state_hash)| maybe_state_hash.map(|sr| (*slot, sr)),
));
}
}
// Update the head tracker before the database, so that we maintain the invariant
// that a block present in the head tracker is present in the database.
// See https://github.com/sigp/lighthouse/issues/1557
let mut head_tracker_lock = head_tracker.0.write();
// Check that all the heads to be deleted are still present. The absence of any
// head indicates a race, that will likely resolve itself, so we defer pruning until
// later.
for head_hash in &abandoned_heads {
if !head_tracker_lock.contains_key(head_hash) {
return Ok(PruningOutcome::DeferredConcurrentMutation);
}
}
// Then remove them for real.
for head_hash in abandoned_heads {
head_tracker_lock.remove(&head_hash);
}
let batch: Vec<StoreOp<E>> = abandoned_blocks
.into_iter()
.map(Into::into)
.map(StoreOp::DeleteBlock)
.chain(
abandoned_states
.into_iter()
.map(|(slot, state_hash)| StoreOp::DeleteState(state_hash.into(), Some(slot))),
)
.collect();
let mut kv_batch = store.convert_to_kv_batch(&batch)?;
// Persist the head in case the process is killed or crashes here. This prevents
// the head tracker reverting after our mutation above.
let persisted_head = PersistedBeaconChain {
_canonical_head_block_root: DUMMY_CANONICAL_HEAD_BLOCK_ROOT,
genesis_block_root,
ssz_head_tracker: SszHeadTracker::from_map(&*head_tracker_lock),
};
drop(head_tracker_lock);
kv_batch.push(persisted_head.as_kv_store_op(BEACON_CHAIN_DB_KEY));
store.hot_db.do_atomically(kv_batch)?;
debug!(log, "Database pruning complete");
Ok(PruningOutcome::Successful)
}
}