Refactor blocktree storage abstraction (#3588)
This commit is contained in:
@@ -8,7 +8,7 @@ use crate::result::{Error, Result};
|
||||
#[cfg(feature = "kvstore")]
|
||||
use solana_kvstore as kvstore;
|
||||
|
||||
use bincode::{deserialize, serialize};
|
||||
use bincode::deserialize;
|
||||
|
||||
use hashbrown::HashMap;
|
||||
|
||||
@@ -36,16 +36,19 @@ macro_rules! db_imports {
|
||||
{ $mod:ident, $db:ident, $db_path:expr } => {
|
||||
mod $mod;
|
||||
|
||||
pub use db::{
|
||||
Cursor, Database, IndexColumn, IWriteBatch, LedgerColumnFamily,
|
||||
LedgerColumnFamilyRaw,
|
||||
};
|
||||
use $mod::$db;
|
||||
use db::columns as cf;
|
||||
|
||||
pub use db::columns;
|
||||
|
||||
pub type Database = db::Database<$db>;
|
||||
pub type Cursor<C> = db::Cursor<$db, C>;
|
||||
pub type LedgerColumn<C> = db::LedgerColumn<$db, C>;
|
||||
pub type WriteBatch = db::WriteBatch<$db>;
|
||||
|
||||
pub trait Column: db::Column<$db> {}
|
||||
impl<C: db::Column<$db>> Column for C {}
|
||||
|
||||
pub use $mod::{$db, ErasureCf, MetaCf, DataCf, DetachedHeadsCf};
|
||||
pub type BlocktreeRawIterator = <$db as Database>::Cursor;
|
||||
pub type WriteBatch = <$db as Database>::WriteBatch;
|
||||
pub type OwnedKey = <$db as Database>::OwnedKey;
|
||||
pub type Key = <$db as Database>::Key;
|
||||
pub const BLOCKTREE_DIRECTORY: &str = $db_path;
|
||||
};
|
||||
}
|
||||
@@ -117,15 +120,11 @@ impl SlotMeta {
|
||||
|
||||
// ledger window
|
||||
pub struct Blocktree {
|
||||
// Underlying database is automatically closed in the Drop implementation of DB
|
||||
#[cfg(not(feature = "kvstore"))]
|
||||
db: Arc<Rocks>,
|
||||
#[cfg(feature = "kvstore")]
|
||||
db: Arc<Kvs>,
|
||||
meta_cf: MetaCf,
|
||||
data_cf: DataCf,
|
||||
erasure_cf: ErasureCf,
|
||||
detached_heads_cf: DetachedHeadsCf,
|
||||
db: Arc<Database>,
|
||||
meta_cf: LedgerColumn<cf::SlotMeta>,
|
||||
data_cf: LedgerColumn<cf::Data>,
|
||||
erasure_cf: LedgerColumn<cf::Coding>,
|
||||
detached_heads_cf: LedgerColumn<cf::DetachedHeads>,
|
||||
pub new_blobs_signals: Vec<SyncSender<bool>>,
|
||||
}
|
||||
|
||||
@@ -139,6 +138,38 @@ pub const ERASURE_CF: &str = "erasure";
|
||||
pub const DETACHED_HEADS_CF: &str = "detached_heads";
|
||||
|
||||
impl Blocktree {
|
||||
/// Opens a Ledger in directory, provides "infinite" window of blobs
|
||||
pub fn open(ledger_path: &str) -> Result<Blocktree> {
|
||||
use std::path::Path;
|
||||
|
||||
fs::create_dir_all(&ledger_path)?;
|
||||
let ledger_path = Path::new(&ledger_path).join(BLOCKTREE_DIRECTORY);
|
||||
|
||||
// Open the database
|
||||
let db = Arc::new(Database::open(&ledger_path)?);
|
||||
|
||||
// Create the metadata column family
|
||||
let meta_cf = LedgerColumn::new(&db);
|
||||
|
||||
// Create the data column family
|
||||
let data_cf = LedgerColumn::new(&db);
|
||||
|
||||
// Create the erasure column family
|
||||
let erasure_cf = LedgerColumn::new(&db);
|
||||
|
||||
// Create the detached heads column family
|
||||
let detached_heads_cf = LedgerColumn::new(&db);
|
||||
|
||||
Ok(Blocktree {
|
||||
db,
|
||||
meta_cf,
|
||||
data_cf,
|
||||
erasure_cf,
|
||||
detached_heads_cf,
|
||||
new_blobs_signals: vec![],
|
||||
})
|
||||
}
|
||||
|
||||
pub fn open_with_signal(ledger_path: &str) -> Result<(Self, Receiver<bool>)> {
|
||||
let mut blocktree = Self::open(ledger_path)?;
|
||||
let (signal_sender, signal_receiver) = sync_channel(1);
|
||||
@@ -147,37 +178,43 @@ impl Blocktree {
|
||||
Ok((blocktree, signal_receiver))
|
||||
}
|
||||
|
||||
pub fn destroy(ledger_path: &str) -> Result<()> {
|
||||
// Database::destroy() fails is the path doesn't exist
|
||||
fs::create_dir_all(ledger_path)?;
|
||||
let path = std::path::Path::new(ledger_path).join(BLOCKTREE_DIRECTORY);
|
||||
Database::destroy(&path)
|
||||
}
|
||||
|
||||
pub fn meta(&self, slot: u64) -> Result<Option<SlotMeta>> {
|
||||
self.meta_cf.get(&MetaCf::key(&slot))
|
||||
self.meta_cf.get(slot)
|
||||
}
|
||||
|
||||
pub fn detached_head(&self, slot: u64) -> Result<Option<bool>> {
|
||||
self.detached_heads_cf.get(&DetachedHeadsCf::key(&slot))
|
||||
self.detached_heads_cf.get(slot)
|
||||
}
|
||||
|
||||
pub fn reset_slot_consumed(&self, slot: u64) -> Result<()> {
|
||||
let meta_key = MetaCf::key(&slot);
|
||||
if let Some(mut meta) = self.meta_cf.get(&meta_key)? {
|
||||
if let Some(mut meta) = self.meta_cf.get(slot)? {
|
||||
for index in 0..meta.received {
|
||||
self.data_cf.delete_by_index(&(slot, index))?;
|
||||
self.data_cf.delete((slot, index))?;
|
||||
}
|
||||
meta.consumed = 0;
|
||||
meta.received = 0;
|
||||
meta.last_index = std::u64::MAX;
|
||||
meta.next_slots = vec![];
|
||||
self.meta_cf.put(&meta_key, &meta)?;
|
||||
self.meta_cf.put(0, &meta)?;
|
||||
}
|
||||
Ok(())
|
||||
}
|
||||
|
||||
pub fn get_next_slot(&self, slot: u64) -> Result<Option<u64>> {
|
||||
let mut db_iterator = self.db.raw_iterator_cf(self.meta_cf.handle())?;
|
||||
db_iterator.seek(&MetaCf::key(&(slot + 1)));
|
||||
let mut db_iterator = self.db.cursor::<cf::SlotMeta>()?;
|
||||
db_iterator.seek(slot + 1);
|
||||
if !db_iterator.valid() {
|
||||
Ok(None)
|
||||
} else {
|
||||
let key = &db_iterator.key().expect("Expected valid key");
|
||||
Ok(Some(MetaCf::index(&key)))
|
||||
let next_slot = db_iterator.key().expect("Expected valid key");
|
||||
Ok(Some(next_slot))
|
||||
}
|
||||
}
|
||||
|
||||
@@ -328,11 +365,7 @@ impl Blocktree {
|
||||
// Check if the working copy of the metadata has changed
|
||||
if Some(meta) != meta_backup.as_ref() {
|
||||
should_signal = should_signal || Self::slot_has_updates(meta, &meta_backup);
|
||||
write_batch.put_cf(
|
||||
self.meta_cf.handle(),
|
||||
&MetaCf::key(slot),
|
||||
&serialize(&meta)?,
|
||||
)?;
|
||||
write_batch.put::<cf::SlotMeta>(*slot, &meta)?;
|
||||
}
|
||||
}
|
||||
|
||||
@@ -357,9 +390,8 @@ impl Blocktree {
|
||||
buf: &mut [u8],
|
||||
slot: u64,
|
||||
) -> Result<(u64, u64)> {
|
||||
let start_key = DataCf::key(&(slot, start_index));
|
||||
let mut db_iterator = self.db.raw_iterator_cf(self.data_cf.handle())?;
|
||||
db_iterator.seek(&start_key);
|
||||
let mut db_iterator = self.db.cursor::<cf::Data>()?;
|
||||
db_iterator.seek((slot, start_index));
|
||||
let mut total_blobs = 0;
|
||||
let mut total_current_size = 0;
|
||||
for expected_index in start_index..start_index + num_blobs {
|
||||
@@ -376,14 +408,13 @@ impl Blocktree {
|
||||
|
||||
// Check key is the next sequential key based on
|
||||
// blob index
|
||||
let key = &db_iterator.key().expect("Expected valid key");
|
||||
let index = DataCf::index(key).1;
|
||||
let (_, index) = db_iterator.key().expect("Expected valid key");
|
||||
if index != expected_index {
|
||||
break;
|
||||
}
|
||||
|
||||
// Get the blob data
|
||||
let value = &db_iterator.value();
|
||||
let value = &db_iterator.value_bytes();
|
||||
|
||||
if value.is_none() {
|
||||
break;
|
||||
@@ -410,24 +441,24 @@ impl Blocktree {
|
||||
}
|
||||
|
||||
pub fn get_coding_blob_bytes(&self, slot: u64, index: u64) -> Result<Option<Vec<u8>>> {
|
||||
self.erasure_cf.get_by_index(&(slot, index))
|
||||
self.erasure_cf.get_bytes((slot, index))
|
||||
}
|
||||
pub fn delete_coding_blob(&self, slot: u64, index: u64) -> Result<()> {
|
||||
self.erasure_cf.delete_by_index(&(slot, index))
|
||||
self.erasure_cf.delete((slot, index))
|
||||
}
|
||||
pub fn get_data_blob_bytes(&self, slot: u64, index: u64) -> Result<Option<Vec<u8>>> {
|
||||
self.data_cf.get_by_index(&(slot, index))
|
||||
self.data_cf.get_bytes((slot, index))
|
||||
}
|
||||
pub fn put_coding_blob_bytes(&self, slot: u64, index: u64, bytes: &[u8]) -> Result<()> {
|
||||
self.erasure_cf.put_by_index(&(slot, index), bytes)
|
||||
self.erasure_cf.put_bytes((slot, index), bytes)
|
||||
}
|
||||
|
||||
pub fn put_data_raw(&self, key: &Key, value: &[u8]) -> Result<()> {
|
||||
self.data_cf.put_bytes(key, value)
|
||||
pub fn put_data_raw(&self, slot: u64, index: u64, value: &[u8]) -> Result<()> {
|
||||
self.data_cf.put_bytes((slot, index), value)
|
||||
}
|
||||
|
||||
pub fn put_data_blob_bytes(&self, slot: u64, index: u64, bytes: &[u8]) -> Result<()> {
|
||||
self.data_cf.put_by_index(&(slot, index), bytes)
|
||||
self.data_cf.put_bytes((slot, index), bytes)
|
||||
}
|
||||
|
||||
pub fn get_data_blob(&self, slot: u64, blob_index: u64) -> Result<Option<Blob>> {
|
||||
@@ -452,16 +483,16 @@ impl Blocktree {
|
||||
// Given a start and end entry index, find all the missing
|
||||
// indexes in the ledger in the range [start_index, end_index)
|
||||
// for the slot with the specified slot
|
||||
fn find_missing_indexes(
|
||||
db_iterator: &mut BlocktreeRawIterator,
|
||||
fn find_missing_indexes<C>(
|
||||
db_iterator: &mut Cursor<C>,
|
||||
slot: u64,
|
||||
start_index: u64,
|
||||
end_index: u64,
|
||||
key: &dyn Fn(u64, u64) -> OwnedKey,
|
||||
slot_from_key: &dyn Fn(&Key) -> u64,
|
||||
index_from_key: &dyn Fn(&Key) -> u64,
|
||||
max_missing: usize,
|
||||
) -> Vec<u64> {
|
||||
) -> Vec<u64>
|
||||
where
|
||||
C: Column<Index = (u64, u64)>,
|
||||
{
|
||||
if start_index >= end_index || max_missing == 0 {
|
||||
return vec![];
|
||||
}
|
||||
@@ -469,7 +500,7 @@ impl Blocktree {
|
||||
let mut missing_indexes = vec![];
|
||||
|
||||
// Seek to the first blob with index >= start_index
|
||||
db_iterator.seek(&key(slot, start_index));
|
||||
db_iterator.seek((slot, start_index));
|
||||
|
||||
// The index of the first missing blob in the slot
|
||||
let mut prev_index = start_index;
|
||||
@@ -483,13 +514,12 @@ impl Blocktree {
|
||||
}
|
||||
break;
|
||||
}
|
||||
let current_key = db_iterator.key().expect("Expect a valid key");
|
||||
let current_slot = slot_from_key(¤t_key);
|
||||
let (current_slot, index) = db_iterator.key().expect("Expect a valid key");
|
||||
let current_index = {
|
||||
if current_slot > slot {
|
||||
end_index
|
||||
} else {
|
||||
index_from_key(¤t_key)
|
||||
index
|
||||
}
|
||||
};
|
||||
let upper_index = cmp::min(current_index, end_index);
|
||||
@@ -523,18 +553,11 @@ impl Blocktree {
|
||||
end_index: u64,
|
||||
max_missing: usize,
|
||||
) -> Vec<u64> {
|
||||
let mut db_iterator = self.data_cf.raw_iterator();
|
||||
|
||||
Self::find_missing_indexes(
|
||||
&mut db_iterator,
|
||||
slot,
|
||||
start_index,
|
||||
end_index,
|
||||
&|slot, index| DataCf::key(&(slot, index)),
|
||||
&MetaCf::index,
|
||||
&|key| DataCf::index(key).1,
|
||||
max_missing,
|
||||
)
|
||||
if let Ok(mut db_iterator) = self.data_cf.cursor() {
|
||||
Self::find_missing_indexes(&mut db_iterator, slot, start_index, end_index, max_missing)
|
||||
} else {
|
||||
vec![]
|
||||
}
|
||||
}
|
||||
|
||||
pub fn find_missing_coding_indexes(
|
||||
@@ -544,18 +567,11 @@ impl Blocktree {
|
||||
end_index: u64,
|
||||
max_missing: usize,
|
||||
) -> Vec<u64> {
|
||||
let mut db_iterator = self.erasure_cf.raw_iterator();
|
||||
|
||||
Self::find_missing_indexes(
|
||||
&mut db_iterator,
|
||||
slot,
|
||||
start_index,
|
||||
end_index,
|
||||
&|slot, index| ErasureCf::key(&(slot, index)),
|
||||
&MetaCf::index,
|
||||
&|key| ErasureCf::index(key).1,
|
||||
max_missing,
|
||||
)
|
||||
if let Ok(mut db_iterator) = self.erasure_cf.cursor() {
|
||||
Self::find_missing_indexes(&mut db_iterator, slot, start_index, end_index, max_missing)
|
||||
} else {
|
||||
vec![]
|
||||
}
|
||||
}
|
||||
|
||||
/// Returns the entry vector for the slot starting with `blob_start_index`
|
||||
@@ -569,6 +585,77 @@ impl Blocktree {
|
||||
.map(|x| x.0)
|
||||
}
|
||||
|
||||
pub fn read_ledger_blobs(&self) -> impl Iterator<Item = Blob> {
|
||||
self.data_cf
|
||||
.iter()
|
||||
.unwrap()
|
||||
.map(|(_, blob_data)| Blob::new(&blob_data))
|
||||
}
|
||||
|
||||
/// Return an iterator for all the entries in the given file.
|
||||
pub fn read_ledger(&self) -> Result<impl Iterator<Item = Entry>> {
|
||||
use crate::entry::EntrySlice;
|
||||
use std::collections::VecDeque;
|
||||
|
||||
struct EntryIterator {
|
||||
db_iterator: Cursor<cf::Data>,
|
||||
|
||||
// TODO: remove me when replay_stage is iterating by block (Blocktree)
|
||||
// this verification is duplicating that of replay_stage, which
|
||||
// can do this in parallel
|
||||
blockhash: Option<Hash>,
|
||||
// https://github.com/rust-rocksdb/rust-rocksdb/issues/234
|
||||
// rocksdb issue: the _blocktree member must be lower in the struct to prevent a crash
|
||||
// when the db_iterator member above is dropped.
|
||||
// _blocktree is unused, but dropping _blocktree results in a broken db_iterator
|
||||
// you have to hold the database open in order to iterate over it, and in order
|
||||
// for db_iterator to be able to run Drop
|
||||
// _blocktree: Blocktree,
|
||||
entries: VecDeque<Entry>,
|
||||
}
|
||||
|
||||
impl Iterator for EntryIterator {
|
||||
type Item = Entry;
|
||||
|
||||
fn next(&mut self) -> Option<Entry> {
|
||||
if !self.entries.is_empty() {
|
||||
return Some(self.entries.pop_front().unwrap());
|
||||
}
|
||||
|
||||
if self.db_iterator.valid() {
|
||||
if let Some(value) = self.db_iterator.value_bytes() {
|
||||
if let Ok(next_entries) =
|
||||
deserialize::<Vec<Entry>>(&value[BLOB_HEADER_SIZE..])
|
||||
{
|
||||
if let Some(blockhash) = self.blockhash {
|
||||
if !next_entries.verify(&blockhash) {
|
||||
return None;
|
||||
}
|
||||
}
|
||||
self.db_iterator.next();
|
||||
if next_entries.is_empty() {
|
||||
return None;
|
||||
}
|
||||
self.entries = VecDeque::from(next_entries);
|
||||
let entry = self.entries.pop_front().unwrap();
|
||||
self.blockhash = Some(entry.hash);
|
||||
return Some(entry);
|
||||
}
|
||||
}
|
||||
}
|
||||
None
|
||||
}
|
||||
}
|
||||
let mut db_iterator = self.data_cf.cursor()?;
|
||||
|
||||
db_iterator.seek_to_first();
|
||||
Ok(EntryIterator {
|
||||
entries: VecDeque::new(),
|
||||
db_iterator,
|
||||
blockhash: None,
|
||||
})
|
||||
}
|
||||
|
||||
pub fn get_slot_entries_with_blob_count(
|
||||
&self,
|
||||
slot: u64,
|
||||
@@ -648,7 +735,7 @@ impl Blocktree {
|
||||
// Write all the newly changed slots in new_chained_slots to the write_batch
|
||||
for (slot, meta) in new_chained_slots.iter() {
|
||||
let meta: &SlotMeta = &RefCell::borrow(&*meta);
|
||||
write_batch.put_cf(self.meta_cf.handle(), &MetaCf::key(slot), &serialize(meta)?)?;
|
||||
write_batch.put::<cf::SlotMeta>(*slot, meta)?;
|
||||
}
|
||||
Ok(())
|
||||
}
|
||||
@@ -692,21 +779,14 @@ impl Blocktree {
|
||||
);
|
||||
|
||||
if Self::is_detached_head(&RefCell::borrow(&*prev_slot_meta)) {
|
||||
write_batch.put_cf(
|
||||
self.detached_heads_cf.handle(),
|
||||
&DetachedHeadsCf::key(&prev_slot),
|
||||
&serialize(&true)?,
|
||||
)?;
|
||||
write_batch.put::<cf::DetachedHeads>(prev_slot, &true)?;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// At this point this slot has received a parent, so no longer a detached head
|
||||
if is_detached_head {
|
||||
write_batch.delete_cf(
|
||||
self.detached_heads_cf.handle(),
|
||||
&DetachedHeadsCf::key(&slot),
|
||||
)?;
|
||||
write_batch.delete::<cf::DetachedHeads>(slot)?;
|
||||
}
|
||||
}
|
||||
|
||||
@@ -883,12 +963,11 @@ impl Blocktree {
|
||||
}
|
||||
};
|
||||
|
||||
let key = DataCf::key(&(blob_slot, blob_index));
|
||||
let serialized_blob_data = &blob_to_insert.data[..BLOB_HEADER_SIZE + blob_size];
|
||||
|
||||
// Commit step: commit all changes to the mutable structures at once, or none at all.
|
||||
// We don't want only some of these changes going through.
|
||||
write_batch.put_cf(self.data_cf.handle(), &key, serialized_blob_data)?;
|
||||
write_batch.put_bytes::<cf::Data>((blob_slot, blob_index), serialized_blob_data)?;
|
||||
prev_inserted_blob_datas.insert((blob_slot, blob_index), serialized_blob_data);
|
||||
// Index is zero-indexed, while the "received" height starts from 1,
|
||||
// so received = index + 1 for the same blob.
|
||||
@@ -927,7 +1006,7 @@ impl Blocktree {
|
||||
// Try to find the next blob we're looking for in the prev_inserted_blob_datas
|
||||
if let Some(prev_blob_data) = prev_inserted_blob_datas.get(&(slot, current_index)) {
|
||||
blobs.push(Cow::Borrowed(*prev_blob_data));
|
||||
} else if let Some(blob_data) = self.data_cf.get_by_index(&(slot, current_index))? {
|
||||
} else if let Some(blob_data) = self.data_cf.get_bytes((slot, current_index))? {
|
||||
// Try to find the next blob we're looking for in the database
|
||||
blobs.push(Cow::Owned(blob_data));
|
||||
} else {
|
||||
@@ -944,7 +1023,6 @@ impl Blocktree {
|
||||
// don't count as ticks, even if they're empty entries
|
||||
fn write_genesis_blobs(&self, blobs: &[Blob]) -> Result<()> {
|
||||
// TODO: change bootstrap height to number of slots
|
||||
let meta_key = MetaCf::key(&0);
|
||||
let mut bootstrap_meta = SlotMeta::new(0, 1);
|
||||
let last = blobs.last().unwrap();
|
||||
|
||||
@@ -953,15 +1031,10 @@ impl Blocktree {
|
||||
bootstrap_meta.is_connected = true;
|
||||
|
||||
let mut batch = self.db.batch()?;
|
||||
batch.put_cf(
|
||||
self.meta_cf.handle(),
|
||||
&meta_key,
|
||||
&serialize(&bootstrap_meta)?,
|
||||
)?;
|
||||
batch.put::<cf::SlotMeta>(0, &bootstrap_meta)?;
|
||||
for blob in blobs {
|
||||
let key = DataCf::key(&(blob.slot(), blob.index()));
|
||||
let serialized_blob_datas = &blob.data[..BLOB_HEADER_SIZE + blob.size()];
|
||||
batch.put_cf(self.data_cf.handle(), &key, serialized_blob_datas)?;
|
||||
batch.put_bytes::<cf::Data>((blob.slot(), blob.index()), serialized_blob_datas)?;
|
||||
}
|
||||
self.db.write(batch)?;
|
||||
Ok(())
|
||||
@@ -1077,7 +1150,6 @@ pub fn tmp_copy_blocktree(from: &str, name: &str) -> String {
|
||||
#[cfg(test)]
|
||||
pub mod tests {
|
||||
use super::*;
|
||||
use crate::blocktree::db::Database;
|
||||
use crate::entry::{
|
||||
create_ticks, make_tiny_test_entries, make_tiny_test_entries_from_hash, Entry, EntrySlice,
|
||||
};
|
||||
@@ -1208,11 +1280,10 @@ pub mod tests {
|
||||
|
||||
// Test meta column family
|
||||
let meta = SlotMeta::new(0, 1);
|
||||
let meta_key = MetaCf::key(&0);
|
||||
ledger.meta_cf.put(&meta_key, &meta).unwrap();
|
||||
ledger.meta_cf.put(0, &meta).unwrap();
|
||||
let result = ledger
|
||||
.meta_cf
|
||||
.get(&meta_key)
|
||||
.get(0)
|
||||
.unwrap()
|
||||
.expect("Expected meta object to exist");
|
||||
|
||||
@@ -1220,15 +1291,12 @@ pub mod tests {
|
||||
|
||||
// Test erasure column family
|
||||
let erasure = vec![1u8; 16];
|
||||
let erasure_key = ErasureCf::key(&(0, 0));
|
||||
ledger
|
||||
.erasure_cf
|
||||
.put_bytes(&erasure_key[..], &erasure)
|
||||
.unwrap();
|
||||
let erasure_key = (0, 0);
|
||||
ledger.erasure_cf.put_bytes(erasure_key, &erasure).unwrap();
|
||||
|
||||
let result = ledger
|
||||
.erasure_cf
|
||||
.get_bytes(&erasure_key[..])
|
||||
.get_bytes(erasure_key)
|
||||
.unwrap()
|
||||
.expect("Expected erasure object to exist");
|
||||
|
||||
@@ -1236,12 +1304,12 @@ pub mod tests {
|
||||
|
||||
// Test data column family
|
||||
let data = vec![2u8; 16];
|
||||
let data_key = DataCf::key(&(0, 0));
|
||||
ledger.data_cf.put_bytes(&data_key, &data).unwrap();
|
||||
let data_key = (0, 0);
|
||||
ledger.data_cf.put_bytes(data_key, &data).unwrap();
|
||||
|
||||
let result = ledger
|
||||
.data_cf
|
||||
.get_bytes(&data_key)
|
||||
.get_bytes(data_key)
|
||||
.unwrap()
|
||||
.expect("Expected data object to exist");
|
||||
|
||||
@@ -1336,7 +1404,7 @@ pub mod tests {
|
||||
|
||||
let meta = ledger
|
||||
.meta_cf
|
||||
.get(&MetaCf::key(&0))
|
||||
.get(0)
|
||||
.unwrap()
|
||||
.expect("Expected new metadata object to be created");
|
||||
assert!(meta.consumed == 0 && meta.received == num_entries);
|
||||
@@ -1351,7 +1419,7 @@ pub mod tests {
|
||||
|
||||
let meta = ledger
|
||||
.meta_cf
|
||||
.get(&MetaCf::key(&0))
|
||||
.get(0)
|
||||
.unwrap()
|
||||
.expect("Expected new metadata object to exist");
|
||||
assert_eq!(meta.consumed, num_entries);
|
||||
@@ -1381,7 +1449,7 @@ pub mod tests {
|
||||
|
||||
let meta = ledger
|
||||
.meta_cf
|
||||
.get(&MetaCf::key(&0))
|
||||
.get(0)
|
||||
.unwrap()
|
||||
.expect("Expected metadata object to exist");
|
||||
assert_eq!(meta.parent_slot, 0);
|
||||
@@ -1429,16 +1497,15 @@ pub mod tests {
|
||||
|
||||
let mut db_iterator = blocktree
|
||||
.db
|
||||
.raw_iterator_cf(blocktree.data_cf.handle())
|
||||
.cursor::<cf::Data>()
|
||||
.expect("Expected to be able to open database iterator");
|
||||
|
||||
db_iterator.seek(&DataCf::key(&(slot, 1)));
|
||||
db_iterator.seek((slot, 1));
|
||||
|
||||
// Iterate through ledger
|
||||
for i in 0..num_entries {
|
||||
assert!(db_iterator.valid());
|
||||
let current_key = db_iterator.key().expect("Expected a valid key");
|
||||
let current_index = DataCf::index(¤t_key).1;
|
||||
let (_, current_index) = db_iterator.key().expect("Expected a valid key");
|
||||
assert_eq!(current_index, (1 as u64) << (i * 8));
|
||||
db_iterator.next();
|
||||
}
|
||||
@@ -1558,8 +1625,7 @@ pub mod tests {
|
||||
|
||||
assert_eq!(blocktree.get_slot_entries(0, 0, None).unwrap(), vec![]);
|
||||
|
||||
let meta_key = MetaCf::key(&slot);
|
||||
let meta = blocktree.meta_cf.get(&meta_key).unwrap().unwrap();
|
||||
let meta = blocktree.meta_cf.get(slot).unwrap().unwrap();
|
||||
if num_entries % 2 == 0 {
|
||||
assert_eq!(meta.received, num_entries);
|
||||
} else {
|
||||
@@ -1580,8 +1646,7 @@ pub mod tests {
|
||||
original_entries,
|
||||
);
|
||||
|
||||
let meta_key = MetaCf::key(&slot);
|
||||
let meta = blocktree.meta_cf.get(&meta_key).unwrap().unwrap();
|
||||
let meta = blocktree.meta_cf.get(slot).unwrap().unwrap();
|
||||
assert_eq!(meta.received, num_entries);
|
||||
assert_eq!(meta.consumed, num_entries);
|
||||
assert_eq!(meta.parent_slot, 0);
|
||||
@@ -1633,8 +1698,7 @@ pub mod tests {
|
||||
|
||||
assert_eq!(blocktree.get_slot_entries(0, 0, None).unwrap(), expected,);
|
||||
|
||||
let meta_key = MetaCf::key(&0);
|
||||
let meta = blocktree.meta_cf.get(&meta_key).unwrap().unwrap();
|
||||
let meta = blocktree.meta_cf.get(0).unwrap().unwrap();
|
||||
assert_eq!(meta.consumed, num_unique_entries);
|
||||
assert_eq!(meta.received, num_unique_entries);
|
||||
assert_eq!(meta.parent_slot, 0);
|
||||
@@ -2096,17 +2160,14 @@ pub mod tests {
|
||||
assert!(blocktree.get_slots_since(&vec![0]).unwrap().is_empty());
|
||||
|
||||
let mut meta0 = SlotMeta::new(0, 0);
|
||||
blocktree
|
||||
.meta_cf
|
||||
.put_by_index(&0, &serialize(&meta0).unwrap())
|
||||
.unwrap();
|
||||
blocktree.meta_cf.put(0, &meta0).unwrap();
|
||||
|
||||
// Slot exists, chains to nothing
|
||||
let expected: HashMap<u64, Vec<u64>> =
|
||||
HashMap::from_iter(vec![(0, vec![])].into_iter());
|
||||
assert_eq!(blocktree.get_slots_since(&vec![0]).unwrap(), expected);
|
||||
meta0.next_slots = vec![1, 2];
|
||||
blocktree.meta_cf.put(&MetaCf::key(&0), &meta0).unwrap();
|
||||
blocktree.meta_cf.put(0, &meta0).unwrap();
|
||||
|
||||
// Slot exists, chains to some other slots
|
||||
let expected: HashMap<u64, Vec<u64>> =
|
||||
@@ -2116,10 +2177,7 @@ pub mod tests {
|
||||
|
||||
let mut meta3 = SlotMeta::new(3, 1);
|
||||
meta3.next_slots = vec![10, 5];
|
||||
blocktree
|
||||
.meta_cf
|
||||
.put_by_index(&3, &serialize(&meta3).unwrap())
|
||||
.unwrap();
|
||||
blocktree.meta_cf.put(3, &meta3).unwrap();
|
||||
let expected: HashMap<u64, Vec<u64>> =
|
||||
HashMap::from_iter(vec![(0, vec![1, 2]), (3, vec![10, 5])].into_iter());
|
||||
assert_eq!(blocktree.get_slots_since(&vec![0, 1, 3]).unwrap(), expected);
|
||||
@@ -2225,8 +2283,7 @@ pub mod tests {
|
||||
entries[i as usize]
|
||||
);
|
||||
|
||||
let meta_key = MetaCf::key(&i);
|
||||
let meta = blocktree.meta_cf.get(&meta_key).unwrap().unwrap();
|
||||
let meta = blocktree.meta_cf.get(i).unwrap().unwrap();
|
||||
assert_eq!(meta.received, i + 1);
|
||||
assert_eq!(meta.last_index, i);
|
||||
if i != 0 {
|
||||
@@ -2448,13 +2505,10 @@ pub mod tests {
|
||||
|
||||
fn get_detached_heads(blocktree: &Blocktree) -> Vec<u64> {
|
||||
let mut results = vec![];
|
||||
let mut iter = blocktree
|
||||
.db
|
||||
.raw_iterator_cf(blocktree.detached_heads_cf.handle())
|
||||
.unwrap();
|
||||
let mut iter = blocktree.detached_heads_cf.cursor().unwrap();
|
||||
iter.seek_to_first();
|
||||
while iter.valid() {
|
||||
results.push(DetachedHeadsCf::index(&iter.key().unwrap()));
|
||||
results.push(iter.key().unwrap());
|
||||
iter.next();
|
||||
}
|
||||
results
|
||||
|
Reference in New Issue
Block a user