Fix block backfill with genesis skip slots (#4820)
## Issue Addressed Closes #4817. ## Proposed Changes - Fill in the linear block roots array between 0 and the slot of the first block (e.g. slots 0 and 1 on Holesky). - Backport the `--freezer`, `--skip` and `--limit` options for `lighthouse db inspect` from tree-states. This allows us to easily view the database corruption of 4817 using `lighthouse db inspect --network holesky --freezer --column bbr --output values --limit 2`. - Backport the `iter_column_from` change and `MemoryStore` overhaul from tree-states. These are required to enable `lighthouse db inspect`. - Rework `freezer_upper_limit` to allow state lookups for slots below the `state_lower_limit`. Currently state lookups will fail until state reconstruction completes entirely. There is a new regression test for the main bug, but no test for the `freezer_upper_limit` fix because we don't currently support running state reconstruction partially (see #3026). This will be fixed once we merge `tree-states`! In lieu of an automated test, I've tested manually on a Holesky node while it was reconstructing. ## Additional Info Users who backfilled Holesky to slot 0 (e.g. using `--reconstruct-historic-states`) need to either: - Re-sync from genesis. - Re-sync using checkpoint sync and the changes from this PR. Due to the recency of the Holesky genesis, writing a custom pass to fix up broken databases (which would require its own thorough testing) was deemed unnecessary. This is the primary reason for this PR being marked `backwards-incompat`. This will create few conflicts with Deneb, which I've already resolved on `tree-states-deneb` and will be happy to backport to Deneb once this PR is merged to unstable.
This commit is contained in:
parent
b82d1a993c
commit
c574f8136e
1
Cargo.lock
generated
1
Cargo.lock
generated
@ -1495,6 +1495,7 @@ dependencies = [
|
|||||||
"clap",
|
"clap",
|
||||||
"clap_utils",
|
"clap_utils",
|
||||||
"environment",
|
"environment",
|
||||||
|
"hex",
|
||||||
"logging",
|
"logging",
|
||||||
"slog",
|
"slog",
|
||||||
"sloggers",
|
"sloggers",
|
||||||
|
@ -135,15 +135,14 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
|
|||||||
prev_block_slot = block.slot();
|
prev_block_slot = block.slot();
|
||||||
expected_block_root = block.message().parent_root();
|
expected_block_root = block.message().parent_root();
|
||||||
|
|
||||||
// If we've reached genesis, add the genesis block root to the batch and set the
|
// If we've reached genesis, add the genesis block root to the batch for all slots
|
||||||
// anchor slot to 0 to indicate completion.
|
// between 0 and the first block slot, and set the anchor slot to 0 to indicate
|
||||||
|
// completion.
|
||||||
if expected_block_root == self.genesis_block_root {
|
if expected_block_root == self.genesis_block_root {
|
||||||
let genesis_slot = self.spec.genesis_slot;
|
let genesis_slot = self.spec.genesis_slot;
|
||||||
chunk_writer.set(
|
for slot in genesis_slot.as_usize()..block.slot().as_usize() {
|
||||||
genesis_slot.as_usize(),
|
chunk_writer.set(slot, self.genesis_block_root, &mut cold_batch)?;
|
||||||
self.genesis_block_root,
|
}
|
||||||
&mut cold_batch,
|
|
||||||
)?;
|
|
||||||
prev_block_slot = genesis_slot;
|
prev_block_slot = genesis_slot;
|
||||||
expected_block_root = Hash256::zero();
|
expected_block_root = Hash256::zero();
|
||||||
break;
|
break;
|
||||||
|
@ -119,10 +119,13 @@ pub fn start_otb_verification_service<T: BeaconChainTypes>(
|
|||||||
pub fn load_optimistic_transition_blocks<T: BeaconChainTypes>(
|
pub fn load_optimistic_transition_blocks<T: BeaconChainTypes>(
|
||||||
chain: &BeaconChain<T>,
|
chain: &BeaconChain<T>,
|
||||||
) -> Result<Vec<OptimisticTransitionBlock>, StoreError> {
|
) -> Result<Vec<OptimisticTransitionBlock>, StoreError> {
|
||||||
process_results(chain.store.hot_db.iter_column(OTBColumn), |iter| {
|
process_results(
|
||||||
iter.map(|(_, bytes)| OptimisticTransitionBlock::from_store_bytes(&bytes))
|
chain.store.hot_db.iter_column::<Hash256>(OTBColumn),
|
||||||
.collect()
|
|iter| {
|
||||||
})?
|
iter.map(|(_, bytes)| OptimisticTransitionBlock::from_store_bytes(&bytes))
|
||||||
|
.collect()
|
||||||
|
},
|
||||||
|
)?
|
||||||
}
|
}
|
||||||
|
|
||||||
#[derive(Debug)]
|
#[derive(Debug)]
|
||||||
|
@ -2273,6 +2273,18 @@ async fn weak_subjectivity_sync_unaligned_unadvanced_checkpoint() {
|
|||||||
weak_subjectivity_sync_test(slots, checkpoint_slot).await
|
weak_subjectivity_sync_test(slots, checkpoint_slot).await
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// Regression test for https://github.com/sigp/lighthouse/issues/4817
|
||||||
|
// Skip 3 slots immediately after genesis, creating a gap between the genesis block and the first
|
||||||
|
// real block.
|
||||||
|
#[tokio::test]
|
||||||
|
async fn weak_subjectivity_sync_skips_at_genesis() {
|
||||||
|
let start_slot = 4;
|
||||||
|
let end_slot = E::slots_per_epoch() * 4;
|
||||||
|
let slots = (start_slot..end_slot).map(Slot::new).collect();
|
||||||
|
let checkpoint_slot = Slot::new(E::slots_per_epoch() * 2);
|
||||||
|
weak_subjectivity_sync_test(slots, checkpoint_slot).await
|
||||||
|
}
|
||||||
|
|
||||||
async fn weak_subjectivity_sync_test(slots: Vec<Slot>, checkpoint_slot: Slot) {
|
async fn weak_subjectivity_sync_test(slots: Vec<Slot>, checkpoint_slot: Slot) {
|
||||||
// Build an initial chain on one harness, representing a synced node with full history.
|
// Build an initial chain on one harness, representing a synced node with full history.
|
||||||
let num_final_blocks = E::slots_per_epoch() * 2;
|
let num_final_blocks = E::slots_per_epoch() * 2;
|
||||||
|
@ -45,6 +45,8 @@ pub enum Error {
|
|||||||
BlockReplayError(BlockReplayError),
|
BlockReplayError(BlockReplayError),
|
||||||
AddPayloadLogicError,
|
AddPayloadLogicError,
|
||||||
SlotClockUnavailableForMigration,
|
SlotClockUnavailableForMigration,
|
||||||
|
InvalidKey,
|
||||||
|
InvalidBytes,
|
||||||
UnableToDowngrade,
|
UnableToDowngrade,
|
||||||
InconsistentFork(InconsistentFork),
|
InconsistentFork(InconsistentFork),
|
||||||
}
|
}
|
||||||
|
@ -1490,10 +1490,17 @@ impl<E: EthSpec, Hot: ItemStore<E>, Cold: ItemStore<E>> HotColdDB<E, Hot, Cold>
|
|||||||
let split_slot = self.get_split_slot();
|
let split_slot = self.get_split_slot();
|
||||||
let anchor = self.get_anchor_info();
|
let anchor = self.get_anchor_info();
|
||||||
|
|
||||||
// There are no restore points stored if the state upper limit lies in the hot database.
|
// There are no restore points stored if the state upper limit lies in the hot database,
|
||||||
// It hasn't been reached yet, and may never be.
|
// and the lower limit is zero. It hasn't been reached yet, and may never be.
|
||||||
if anchor.map_or(false, |a| a.state_upper_limit >= split_slot) {
|
if anchor.as_ref().map_or(false, |a| {
|
||||||
|
a.state_upper_limit >= split_slot && a.state_lower_limit == 0
|
||||||
|
}) {
|
||||||
None
|
None
|
||||||
|
} else if let Some(lower_limit) = anchor
|
||||||
|
.map(|a| a.state_lower_limit)
|
||||||
|
.filter(|limit| *limit > 0)
|
||||||
|
{
|
||||||
|
Some(lower_limit)
|
||||||
} else {
|
} else {
|
||||||
Some(
|
Some(
|
||||||
(split_slot - 1) / self.config.slots_per_restore_point
|
(split_slot - 1) / self.config.slots_per_restore_point
|
||||||
|
@ -1,7 +1,6 @@
|
|||||||
use super::*;
|
use super::*;
|
||||||
use crate::hot_cold_store::HotColdDBError;
|
use crate::hot_cold_store::HotColdDBError;
|
||||||
use crate::metrics;
|
use crate::metrics;
|
||||||
use db_key::Key;
|
|
||||||
use leveldb::compaction::Compaction;
|
use leveldb::compaction::Compaction;
|
||||||
use leveldb::database::batch::{Batch, Writebatch};
|
use leveldb::database::batch::{Batch, Writebatch};
|
||||||
use leveldb::database::kv::KV;
|
use leveldb::database::kv::KV;
|
||||||
@ -176,10 +175,8 @@ impl<E: EthSpec> KeyValueStore<E> for LevelDB<E> {
|
|||||||
Ok(())
|
Ok(())
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Iterate through all keys and values in a particular column.
|
fn iter_column_from<K: Key>(&self, column: DBColumn, from: &[u8]) -> ColumnIter<K> {
|
||||||
fn iter_column(&self, column: DBColumn) -> ColumnIter {
|
let start_key = BytesKey::from_vec(get_key_for_col(column.into(), from));
|
||||||
let start_key =
|
|
||||||
BytesKey::from_vec(get_key_for_col(column.into(), Hash256::zero().as_bytes()));
|
|
||||||
|
|
||||||
let iter = self.db.iter(self.read_options());
|
let iter = self.db.iter(self.read_options());
|
||||||
iter.seek(&start_key);
|
iter.seek(&start_key);
|
||||||
@ -187,13 +184,12 @@ impl<E: EthSpec> KeyValueStore<E> for LevelDB<E> {
|
|||||||
Box::new(
|
Box::new(
|
||||||
iter.take_while(move |(key, _)| key.matches_column(column))
|
iter.take_while(move |(key, _)| key.matches_column(column))
|
||||||
.map(move |(bytes_key, value)| {
|
.map(move |(bytes_key, value)| {
|
||||||
let key =
|
let key = bytes_key.remove_column_variable(column).ok_or_else(|| {
|
||||||
bytes_key
|
HotColdDBError::IterationError {
|
||||||
.remove_column(column)
|
unexpected_key: bytes_key.clone(),
|
||||||
.ok_or(HotColdDBError::IterationError {
|
}
|
||||||
unexpected_key: bytes_key,
|
})?;
|
||||||
})?;
|
Ok((K::from_bytes(key)?, value))
|
||||||
Ok((key, value))
|
|
||||||
}),
|
}),
|
||||||
)
|
)
|
||||||
}
|
}
|
||||||
@ -254,12 +250,12 @@ impl<E: EthSpec> KeyValueStore<E> for LevelDB<E> {
|
|||||||
impl<E: EthSpec> ItemStore<E> for LevelDB<E> {}
|
impl<E: EthSpec> ItemStore<E> for LevelDB<E> {}
|
||||||
|
|
||||||
/// Used for keying leveldb.
|
/// Used for keying leveldb.
|
||||||
#[derive(Debug, PartialEq)]
|
#[derive(Debug, Clone, PartialEq, Eq, PartialOrd, Ord)]
|
||||||
pub struct BytesKey {
|
pub struct BytesKey {
|
||||||
key: Vec<u8>,
|
key: Vec<u8>,
|
||||||
}
|
}
|
||||||
|
|
||||||
impl Key for BytesKey {
|
impl db_key::Key for BytesKey {
|
||||||
fn from_u8(key: &[u8]) -> Self {
|
fn from_u8(key: &[u8]) -> Self {
|
||||||
Self { key: key.to_vec() }
|
Self { key: key.to_vec() }
|
||||||
}
|
}
|
||||||
@ -275,12 +271,20 @@ impl BytesKey {
|
|||||||
self.key.starts_with(column.as_bytes())
|
self.key.starts_with(column.as_bytes())
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Remove the column from a key, returning its `Hash256` portion.
|
/// Remove the column from a 32 byte key, yielding the `Hash256` key.
|
||||||
pub fn remove_column(&self, column: DBColumn) -> Option<Hash256> {
|
pub fn remove_column(&self, column: DBColumn) -> Option<Hash256> {
|
||||||
|
let key = self.remove_column_variable(column)?;
|
||||||
|
(column.key_size() == 32).then(|| Hash256::from_slice(key))
|
||||||
|
}
|
||||||
|
|
||||||
|
/// Remove the column from a key.
|
||||||
|
///
|
||||||
|
/// Will return `None` if the value doesn't match the column or has the wrong length.
|
||||||
|
pub fn remove_column_variable(&self, column: DBColumn) -> Option<&[u8]> {
|
||||||
if self.matches_column(column) {
|
if self.matches_column(column) {
|
||||||
let subkey = &self.key[column.as_bytes().len()..];
|
let subkey = &self.key[column.as_bytes().len()..];
|
||||||
if subkey.len() == 32 {
|
if subkey.len() == column.key_size() {
|
||||||
return Some(Hash256::from_slice(subkey));
|
return Some(subkey);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
None
|
None
|
||||||
|
@ -44,7 +44,7 @@ use std::sync::Arc;
|
|||||||
use strum::{EnumString, IntoStaticStr};
|
use strum::{EnumString, IntoStaticStr};
|
||||||
pub use types::*;
|
pub use types::*;
|
||||||
|
|
||||||
pub type ColumnIter<'a> = Box<dyn Iterator<Item = Result<(Hash256, Vec<u8>), Error>> + 'a>;
|
pub type ColumnIter<'a, K> = Box<dyn Iterator<Item = Result<(K, Vec<u8>), Error>> + 'a>;
|
||||||
pub type ColumnKeyIter<'a> = Box<dyn Iterator<Item = Result<Hash256, Error>> + 'a>;
|
pub type ColumnKeyIter<'a> = Box<dyn Iterator<Item = Result<Hash256, Error>> + 'a>;
|
||||||
|
|
||||||
pub type RawEntryIter<'a> = Box<dyn Iterator<Item = Result<(Vec<u8>, Vec<u8>), Error>> + 'a>;
|
pub type RawEntryIter<'a> = Box<dyn Iterator<Item = Result<(Vec<u8>, Vec<u8>), Error>> + 'a>;
|
||||||
@ -84,11 +84,12 @@ pub trait KeyValueStore<E: EthSpec>: Sync + Send + Sized + 'static {
|
|||||||
fn compact(&self) -> Result<(), Error>;
|
fn compact(&self) -> Result<(), Error>;
|
||||||
|
|
||||||
/// Iterate through all keys and values in a particular column.
|
/// Iterate through all keys and values in a particular column.
|
||||||
fn iter_column(&self, _column: DBColumn) -> ColumnIter {
|
fn iter_column<K: Key>(&self, column: DBColumn) -> ColumnIter<K> {
|
||||||
// Default impl for non LevelDB databases
|
self.iter_column_from(column, &vec![0; column.key_size()])
|
||||||
Box::new(std::iter::empty())
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
fn iter_column_from<K: Key>(&self, column: DBColumn, from: &[u8]) -> ColumnIter<K>;
|
||||||
|
|
||||||
fn iter_raw_entries(&self, _column: DBColumn, _prefix: &[u8]) -> RawEntryIter {
|
fn iter_raw_entries(&self, _column: DBColumn, _prefix: &[u8]) -> RawEntryIter {
|
||||||
Box::new(std::iter::empty())
|
Box::new(std::iter::empty())
|
||||||
}
|
}
|
||||||
@ -98,9 +99,26 @@ pub trait KeyValueStore<E: EthSpec>: Sync + Send + Sized + 'static {
|
|||||||
}
|
}
|
||||||
|
|
||||||
/// Iterate through all keys in a particular column.
|
/// Iterate through all keys in a particular column.
|
||||||
fn iter_column_keys(&self, _column: DBColumn) -> ColumnKeyIter {
|
fn iter_column_keys(&self, column: DBColumn) -> ColumnKeyIter;
|
||||||
// Default impl for non LevelDB databases
|
}
|
||||||
Box::new(std::iter::empty())
|
|
||||||
|
pub trait Key: Sized + 'static {
|
||||||
|
fn from_bytes(key: &[u8]) -> Result<Self, Error>;
|
||||||
|
}
|
||||||
|
|
||||||
|
impl Key for Hash256 {
|
||||||
|
fn from_bytes(key: &[u8]) -> Result<Self, Error> {
|
||||||
|
if key.len() == 32 {
|
||||||
|
Ok(Hash256::from_slice(key))
|
||||||
|
} else {
|
||||||
|
Err(Error::InvalidKey)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
impl Key for Vec<u8> {
|
||||||
|
fn from_bytes(key: &[u8]) -> Result<Self, Error> {
|
||||||
|
Ok(key.to_vec())
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -250,6 +268,35 @@ impl DBColumn {
|
|||||||
pub fn as_bytes(self) -> &'static [u8] {
|
pub fn as_bytes(self) -> &'static [u8] {
|
||||||
self.as_str().as_bytes()
|
self.as_str().as_bytes()
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/// Most database keys are 32 bytes, but some freezer DB keys are 8 bytes.
|
||||||
|
///
|
||||||
|
/// This function returns the number of bytes used by keys in a given column.
|
||||||
|
pub fn key_size(self) -> usize {
|
||||||
|
match self {
|
||||||
|
Self::OverflowLRUCache => 40,
|
||||||
|
Self::BeaconMeta
|
||||||
|
| Self::BeaconBlock
|
||||||
|
| Self::BeaconState
|
||||||
|
| Self::BeaconBlob
|
||||||
|
| Self::BeaconStateSummary
|
||||||
|
| Self::BeaconStateTemporary
|
||||||
|
| Self::ExecPayload
|
||||||
|
| Self::BeaconChain
|
||||||
|
| Self::OpPool
|
||||||
|
| Self::Eth1Cache
|
||||||
|
| Self::ForkChoice
|
||||||
|
| Self::PubkeyCache
|
||||||
|
| Self::BeaconRestorePoint
|
||||||
|
| Self::DhtEnrs
|
||||||
|
| Self::OptimisticTransitionBlock => 32,
|
||||||
|
Self::BeaconBlockRoots
|
||||||
|
| Self::BeaconStateRoots
|
||||||
|
| Self::BeaconHistoricalRoots
|
||||||
|
| Self::BeaconHistoricalSummaries
|
||||||
|
| Self::BeaconRandaoMixes => 8,
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/// An item that may stored in a `Store` by serializing and deserializing from bytes.
|
/// An item that may stored in a `Store` by serializing and deserializing from bytes.
|
||||||
|
@ -1,17 +1,17 @@
|
|||||||
use super::{Error, ItemStore, KeyValueStore, KeyValueStoreOp};
|
use crate::{
|
||||||
use crate::{ColumnIter, DBColumn};
|
get_key_for_col, leveldb_store::BytesKey, ColumnIter, ColumnKeyIter, DBColumn, Error,
|
||||||
|
ItemStore, Key, KeyValueStore, KeyValueStoreOp,
|
||||||
|
};
|
||||||
use parking_lot::{Mutex, MutexGuard, RwLock};
|
use parking_lot::{Mutex, MutexGuard, RwLock};
|
||||||
use std::collections::{HashMap, HashSet};
|
use std::collections::BTreeMap;
|
||||||
use std::marker::PhantomData;
|
use std::marker::PhantomData;
|
||||||
use types::*;
|
use types::*;
|
||||||
|
|
||||||
type DBHashMap = HashMap<Vec<u8>, Vec<u8>>;
|
type DBMap = BTreeMap<BytesKey, Vec<u8>>;
|
||||||
type DBKeyMap = HashMap<Vec<u8>, HashSet<Vec<u8>>>;
|
|
||||||
|
|
||||||
/// A thread-safe `HashMap` wrapper.
|
/// A thread-safe `BTreeMap` wrapper.
|
||||||
pub struct MemoryStore<E: EthSpec> {
|
pub struct MemoryStore<E: EthSpec> {
|
||||||
db: RwLock<DBHashMap>,
|
db: RwLock<DBMap>,
|
||||||
col_keys: RwLock<DBKeyMap>,
|
|
||||||
transaction_mutex: Mutex<()>,
|
transaction_mutex: Mutex<()>,
|
||||||
_phantom: PhantomData<E>,
|
_phantom: PhantomData<E>,
|
||||||
}
|
}
|
||||||
@ -20,36 +20,24 @@ impl<E: EthSpec> MemoryStore<E> {
|
|||||||
/// Create a new, empty database.
|
/// Create a new, empty database.
|
||||||
pub fn open() -> Self {
|
pub fn open() -> Self {
|
||||||
Self {
|
Self {
|
||||||
db: RwLock::new(HashMap::new()),
|
db: RwLock::new(BTreeMap::new()),
|
||||||
col_keys: RwLock::new(HashMap::new()),
|
|
||||||
transaction_mutex: Mutex::new(()),
|
transaction_mutex: Mutex::new(()),
|
||||||
_phantom: PhantomData,
|
_phantom: PhantomData,
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
fn get_key_for_col(col: &str, key: &[u8]) -> Vec<u8> {
|
|
||||||
let mut col = col.as_bytes().to_vec();
|
|
||||||
col.append(&mut key.to_vec());
|
|
||||||
col
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
impl<E: EthSpec> KeyValueStore<E> for MemoryStore<E> {
|
impl<E: EthSpec> KeyValueStore<E> for MemoryStore<E> {
|
||||||
/// Get the value of some key from the database. Returns `None` if the key does not exist.
|
/// Get the value of some key from the database. Returns `None` if the key does not exist.
|
||||||
fn get_bytes(&self, col: &str, key: &[u8]) -> Result<Option<Vec<u8>>, Error> {
|
fn get_bytes(&self, col: &str, key: &[u8]) -> Result<Option<Vec<u8>>, Error> {
|
||||||
let column_key = Self::get_key_for_col(col, key);
|
let column_key = BytesKey::from_vec(get_key_for_col(col, key));
|
||||||
Ok(self.db.read().get(&column_key).cloned())
|
Ok(self.db.read().get(&column_key).cloned())
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Puts a key in the database.
|
/// Puts a key in the database.
|
||||||
fn put_bytes(&self, col: &str, key: &[u8], val: &[u8]) -> Result<(), Error> {
|
fn put_bytes(&self, col: &str, key: &[u8], val: &[u8]) -> Result<(), Error> {
|
||||||
let column_key = Self::get_key_for_col(col, key);
|
let column_key = BytesKey::from_vec(get_key_for_col(col, key));
|
||||||
self.db.write().insert(column_key, val.to_vec());
|
self.db.write().insert(column_key, val.to_vec());
|
||||||
self.col_keys
|
|
||||||
.write()
|
|
||||||
.entry(col.as_bytes().to_vec())
|
|
||||||
.or_default()
|
|
||||||
.insert(key.to_vec());
|
|
||||||
Ok(())
|
Ok(())
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -64,18 +52,14 @@ impl<E: EthSpec> KeyValueStore<E> for MemoryStore<E> {
|
|||||||
|
|
||||||
/// Return true if some key exists in some column.
|
/// Return true if some key exists in some column.
|
||||||
fn key_exists(&self, col: &str, key: &[u8]) -> Result<bool, Error> {
|
fn key_exists(&self, col: &str, key: &[u8]) -> Result<bool, Error> {
|
||||||
let column_key = Self::get_key_for_col(col, key);
|
let column_key = BytesKey::from_vec(get_key_for_col(col, key));
|
||||||
Ok(self.db.read().contains_key(&column_key))
|
Ok(self.db.read().contains_key(&column_key))
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Delete some key from the database.
|
/// Delete some key from the database.
|
||||||
fn key_delete(&self, col: &str, key: &[u8]) -> Result<(), Error> {
|
fn key_delete(&self, col: &str, key: &[u8]) -> Result<(), Error> {
|
||||||
let column_key = Self::get_key_for_col(col, key);
|
let column_key = BytesKey::from_vec(get_key_for_col(col, key));
|
||||||
self.db.write().remove(&column_key);
|
self.db.write().remove(&column_key);
|
||||||
self.col_keys
|
|
||||||
.write()
|
|
||||||
.get_mut(&col.as_bytes().to_vec())
|
|
||||||
.map(|set| set.remove(key));
|
|
||||||
Ok(())
|
Ok(())
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -83,35 +67,41 @@ impl<E: EthSpec> KeyValueStore<E> for MemoryStore<E> {
|
|||||||
for op in batch {
|
for op in batch {
|
||||||
match op {
|
match op {
|
||||||
KeyValueStoreOp::PutKeyValue(key, value) => {
|
KeyValueStoreOp::PutKeyValue(key, value) => {
|
||||||
self.db.write().insert(key, value);
|
self.db.write().insert(BytesKey::from_vec(key), value);
|
||||||
}
|
}
|
||||||
|
|
||||||
KeyValueStoreOp::DeleteKey(hash) => {
|
KeyValueStoreOp::DeleteKey(key) => {
|
||||||
self.db.write().remove(&hash);
|
self.db.write().remove(&BytesKey::from_vec(key));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
Ok(())
|
Ok(())
|
||||||
}
|
}
|
||||||
|
|
||||||
// pub type ColumnIter<'a> = Box<dyn Iterator<Item = Result<(Hash256, Vec<u8>), Error>> + 'a>;
|
fn iter_column_from<K: Key>(&self, column: DBColumn, from: &[u8]) -> ColumnIter<K> {
|
||||||
fn iter_column(&self, column: DBColumn) -> ColumnIter {
|
// We use this awkward pattern because we can't lock the `self.db` field *and* maintain a
|
||||||
|
// reference to the lock guard across calls to `.next()`. This would be require a
|
||||||
|
// struct with a field (the iterator) which references another field (the lock guard).
|
||||||
|
let start_key = BytesKey::from_vec(get_key_for_col(column.as_str(), from));
|
||||||
let col = column.as_str();
|
let col = column.as_str();
|
||||||
if let Some(keys) = self
|
let keys = self
|
||||||
.col_keys
|
.db
|
||||||
.read()
|
.read()
|
||||||
.get(col.as_bytes())
|
.range(start_key..)
|
||||||
.map(|set| set.iter().cloned().collect::<Vec<_>>())
|
.take_while(|(k, _)| k.remove_column_variable(column).is_some())
|
||||||
{
|
.filter_map(|(k, _)| k.remove_column_variable(column).map(|k| k.to_vec()))
|
||||||
Box::new(keys.into_iter().filter_map(move |key| {
|
.collect::<Vec<_>>();
|
||||||
let hash = Hash256::from_slice(&key);
|
Box::new(keys.into_iter().filter_map(move |key| {
|
||||||
self.get_bytes(col, &key)
|
self.get_bytes(col, &key).transpose().map(|res| {
|
||||||
.transpose()
|
let k = K::from_bytes(&key)?;
|
||||||
.map(|res| res.map(|bytes| (hash, bytes)))
|
let v = res?;
|
||||||
}))
|
Ok((k, v))
|
||||||
} else {
|
})
|
||||||
Box::new(std::iter::empty())
|
}))
|
||||||
}
|
}
|
||||||
|
|
||||||
|
fn iter_column_keys(&self, column: DBColumn) -> ColumnKeyIter {
|
||||||
|
Box::new(self.iter_column(column).map(|res| res.map(|(k, _)| k)))
|
||||||
}
|
}
|
||||||
|
|
||||||
fn begin_rw_transaction(&self) -> MutexGuard<()> {
|
fn begin_rw_transaction(&self) -> MutexGuard<()> {
|
||||||
|
@ -9,6 +9,7 @@ beacon_node = { workspace = true }
|
|||||||
clap = { workspace = true }
|
clap = { workspace = true }
|
||||||
clap_utils = { workspace = true }
|
clap_utils = { workspace = true }
|
||||||
environment = { workspace = true }
|
environment = { workspace = true }
|
||||||
|
hex = { workspace = true }
|
||||||
logging = { workspace = true }
|
logging = { workspace = true }
|
||||||
sloggers = { workspace = true }
|
sloggers = { workspace = true }
|
||||||
store = { workspace = true }
|
store = { workspace = true }
|
||||||
|
@ -60,6 +60,24 @@ pub fn inspect_cli_app<'a, 'b>() -> App<'a, 'b> {
|
|||||||
.default_value("sizes")
|
.default_value("sizes")
|
||||||
.possible_values(InspectTarget::VARIANTS),
|
.possible_values(InspectTarget::VARIANTS),
|
||||||
)
|
)
|
||||||
|
.arg(
|
||||||
|
Arg::with_name("skip")
|
||||||
|
.long("skip")
|
||||||
|
.value_name("N")
|
||||||
|
.help("Skip over the first N keys"),
|
||||||
|
)
|
||||||
|
.arg(
|
||||||
|
Arg::with_name("limit")
|
||||||
|
.long("limit")
|
||||||
|
.value_name("N")
|
||||||
|
.help("Output at most N keys"),
|
||||||
|
)
|
||||||
|
.arg(
|
||||||
|
Arg::with_name("freezer")
|
||||||
|
.long("freezer")
|
||||||
|
.help("Inspect the freezer DB rather than the hot DB")
|
||||||
|
.takes_value(false),
|
||||||
|
)
|
||||||
.arg(
|
.arg(
|
||||||
Arg::with_name("output-dir")
|
Arg::with_name("output-dir")
|
||||||
.long("output-dir")
|
.long("output-dir")
|
||||||
@ -208,6 +226,9 @@ pub enum InspectTarget {
|
|||||||
pub struct InspectConfig {
|
pub struct InspectConfig {
|
||||||
column: DBColumn,
|
column: DBColumn,
|
||||||
target: InspectTarget,
|
target: InspectTarget,
|
||||||
|
skip: Option<usize>,
|
||||||
|
limit: Option<usize>,
|
||||||
|
freezer: bool,
|
||||||
/// Configures where the inspect output should be stored.
|
/// Configures where the inspect output should be stored.
|
||||||
output_dir: PathBuf,
|
output_dir: PathBuf,
|
||||||
}
|
}
|
||||||
@ -215,11 +236,18 @@ pub struct InspectConfig {
|
|||||||
fn parse_inspect_config(cli_args: &ArgMatches) -> Result<InspectConfig, String> {
|
fn parse_inspect_config(cli_args: &ArgMatches) -> Result<InspectConfig, String> {
|
||||||
let column = clap_utils::parse_required(cli_args, "column")?;
|
let column = clap_utils::parse_required(cli_args, "column")?;
|
||||||
let target = clap_utils::parse_required(cli_args, "output")?;
|
let target = clap_utils::parse_required(cli_args, "output")?;
|
||||||
|
let skip = clap_utils::parse_optional(cli_args, "skip")?;
|
||||||
|
let limit = clap_utils::parse_optional(cli_args, "limit")?;
|
||||||
|
let freezer = cli_args.is_present("freezer");
|
||||||
|
|
||||||
let output_dir: PathBuf =
|
let output_dir: PathBuf =
|
||||||
clap_utils::parse_optional(cli_args, "output-dir")?.unwrap_or_else(PathBuf::new);
|
clap_utils::parse_optional(cli_args, "output-dir")?.unwrap_or_else(PathBuf::new);
|
||||||
Ok(InspectConfig {
|
Ok(InspectConfig {
|
||||||
column,
|
column,
|
||||||
target,
|
target,
|
||||||
|
skip,
|
||||||
|
limit,
|
||||||
|
freezer,
|
||||||
output_dir,
|
output_dir,
|
||||||
})
|
})
|
||||||
}
|
}
|
||||||
@ -247,6 +275,17 @@ pub fn inspect_db<E: EthSpec>(
|
|||||||
.map_err(|e| format!("{:?}", e))?;
|
.map_err(|e| format!("{:?}", e))?;
|
||||||
|
|
||||||
let mut total = 0;
|
let mut total = 0;
|
||||||
|
let mut num_keys = 0;
|
||||||
|
|
||||||
|
let sub_db = if inspect_config.freezer {
|
||||||
|
&db.cold_db
|
||||||
|
} else {
|
||||||
|
&db.hot_db
|
||||||
|
};
|
||||||
|
|
||||||
|
let skip = inspect_config.skip.unwrap_or(0);
|
||||||
|
let limit = inspect_config.limit.unwrap_or(usize::MAX);
|
||||||
|
|
||||||
let base_path = &inspect_config.output_dir;
|
let base_path = &inspect_config.output_dir;
|
||||||
|
|
||||||
if let InspectTarget::Values = inspect_config.target {
|
if let InspectTarget::Values = inspect_config.target {
|
||||||
@ -254,20 +293,24 @@ pub fn inspect_db<E: EthSpec>(
|
|||||||
.map_err(|e| format!("Unable to create import directory: {:?}", e))?;
|
.map_err(|e| format!("Unable to create import directory: {:?}", e))?;
|
||||||
}
|
}
|
||||||
|
|
||||||
for res in db.hot_db.iter_column(inspect_config.column) {
|
for res in sub_db
|
||||||
|
.iter_column::<Vec<u8>>(inspect_config.column)
|
||||||
|
.skip(skip)
|
||||||
|
.take(limit)
|
||||||
|
{
|
||||||
let (key, value) = res.map_err(|e| format!("{:?}", e))?;
|
let (key, value) = res.map_err(|e| format!("{:?}", e))?;
|
||||||
|
|
||||||
match inspect_config.target {
|
match inspect_config.target {
|
||||||
InspectTarget::ValueSizes => {
|
InspectTarget::ValueSizes => {
|
||||||
println!("{:?}: {} bytes", key, value.len());
|
println!("{}: {} bytes", hex::encode(&key), value.len());
|
||||||
total += value.len();
|
|
||||||
}
|
|
||||||
InspectTarget::ValueTotal => {
|
|
||||||
total += value.len();
|
|
||||||
}
|
}
|
||||||
|
InspectTarget::ValueTotal => (),
|
||||||
InspectTarget::Values => {
|
InspectTarget::Values => {
|
||||||
let file_path =
|
let file_path = base_path.join(format!(
|
||||||
base_path.join(format!("{}_{}.ssz", inspect_config.column.as_str(), key));
|
"{}_{}.ssz",
|
||||||
|
inspect_config.column.as_str(),
|
||||||
|
hex::encode(&key)
|
||||||
|
));
|
||||||
|
|
||||||
let write_result = fs::OpenOptions::new()
|
let write_result = fs::OpenOptions::new()
|
||||||
.create(true)
|
.create(true)
|
||||||
@ -283,17 +326,14 @@ pub fn inspect_db<E: EthSpec>(
|
|||||||
} else {
|
} else {
|
||||||
println!("Successfully saved values to file: {:?}", file_path);
|
println!("Successfully saved values to file: {:?}", file_path);
|
||||||
}
|
}
|
||||||
|
|
||||||
total += value.len();
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
total += value.len();
|
||||||
|
num_keys += 1;
|
||||||
}
|
}
|
||||||
|
|
||||||
match inspect_config.target {
|
println!("Num keys: {}", num_keys);
|
||||||
InspectTarget::ValueSizes | InspectTarget::ValueTotal | InspectTarget::Values => {
|
println!("Total: {} bytes", total);
|
||||||
println!("Total: {} bytes", total);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
Ok(())
|
Ok(())
|
||||||
}
|
}
|
||||||
|
Loading…
Reference in New Issue
Block a user