From 2eee9241fc1d0d52ea1f9ee74082a39a31fd2924 Mon Sep 17 00:00:00 2001 From: Sean Lawlor Date: Tue, 22 Nov 2022 10:54:43 -0500 Subject: [PATCH 01/13] Adding support for dashmap and the ability to measure the size of the cache --- akd/Cargo.toml | 8 + akd/src/append_only_zks.rs | 8 +- akd/src/helper_structs.rs | 7 + akd/src/node_label.rs | 6 + .../{timed_cache.rs => caches/basic.rs} | 65 +++--- akd/src/storage/caches/high_parallelism.rs | 203 ++++++++++++++++++ akd/src/storage/caches/mod.rs | 47 ++++ akd/src/storage/mod.rs | 8 +- akd/src/storage/types.rs | 32 +++ akd/src/tree_node.rs | 26 +++ akd_mysql/src/mysql.rs | 8 +- 11 files changed, 375 insertions(+), 43 deletions(-) rename akd/src/storage/{timed_cache.rs => caches/basic.rs} (85%) create mode 100644 akd/src/storage/caches/high_parallelism.rs create mode 100644 akd/src/storage/caches/mod.rs diff --git a/akd/Cargo.toml b/akd/Cargo.toml index 4899c901..741b084e 100644 --- a/akd/Cargo.toml +++ b/akd/Cargo.toml @@ -21,6 +21,13 @@ public-tests = ["rand", "bincode", "colored", "once_cell", "serde_serialization" public_auditing = ["protobuf", "thiserror"] default = [] serde_serialization = ["serde", "ed25519-dalek/serde"] +# Represents that we're going to support a higher-level of parallelism and therefore should avoid +# usages of a RwLock around a HashMap which doesn't scale well for parallel access (i.e. in the storage +# caches) +high_parallelism = ["dashmap"] +# Add support for handling memory pressure process on the caches. This costs some performance, however +# provides the ability to protect against OOMing +memory_pressure = [] [dependencies] ## Required dependencies ## @@ -35,6 +42,7 @@ winter-math = "0.2" keyed_priority_queue = "0.3" ## Optional Dependencies ## +dashmap = { version = "5", optional = true } bincode = { version = "1", optional = true } serde = { version = "1", features = ["derive"], optional = true } rand = { version = "0.7", optional = true } diff --git a/akd/src/append_only_zks.rs b/akd/src/append_only_zks.rs index 7532cfff..29071097 100644 --- a/akd/src/append_only_zks.rs +++ b/akd/src/append_only_zks.rs @@ -9,7 +9,7 @@ use crate::{ errors::TreeNodeError, proof_structs::{AppendOnlyProof, MembershipProof, NonMembershipProof, SingleAppendOnlyProof}, - storage::{Storable, Storage}, + storage::{SizeOf, Storable, Storage}, tree_node::*, }; @@ -44,6 +44,12 @@ pub struct Azks { pub num_nodes: u64, // The size of the tree } +impl SizeOf for Azks { + fn size_of(&self) -> usize { + std::mem::size_of::() * 2 + } +} + impl Storable for Azks { type StorageKey = u8; diff --git a/akd/src/helper_structs.rs b/akd/src/helper_structs.rs index 7568838f..d4668f54 100644 --- a/akd/src/helper_structs.rs +++ b/akd/src/helper_structs.rs @@ -11,6 +11,7 @@ #[cfg(feature = "serde_serialization")] use crate::serialization::{digest_deserialize, digest_serialize}; +use winter_crypto::Digest; use winter_crypto::Hasher; use crate::{storage::types::ValueState, NodeLabel}; @@ -36,6 +37,12 @@ pub struct Node { pub hash: H::Digest, } +impl crate::storage::SizeOf for Node { + fn size_of(&self) -> usize { + self.label.size_of() + self.hash.as_bytes().len() + } +} + // can't use #derive because it doesn't bind correctly // #derive and generics are not friendly; might make Debug weird too ... // see also: diff --git a/akd/src/node_label.rs b/akd/src/node_label.rs index 5a678677..c1dca372 100644 --- a/akd/src/node_label.rs +++ b/akd/src/node_label.rs @@ -45,6 +45,12 @@ pub struct NodeLabel { pub label_len: u32, } +impl crate::storage::SizeOf for NodeLabel { + fn size_of(&self) -> usize { + 32 + std::mem::size_of::() + } +} + impl PartialOrd for NodeLabel { fn partial_cmp(&self, other: &Self) -> Option { Some(self.cmp(other)) diff --git a/akd/src/storage/timed_cache.rs b/akd/src/storage/caches/basic.rs similarity index 85% rename from akd/src/storage/timed_cache.rs rename to akd/src/storage/caches/basic.rs index 04be5192..7f01462c 100644 --- a/akd/src/storage/timed_cache.rs +++ b/akd/src/storage/caches/basic.rs @@ -5,24 +5,20 @@ // License, Version 2.0 found in the LICENSE-APACHE file in the root directory // of this source tree. -//! This module implements a basic async timed cache +//! This module implements a basic async timed cache. It additionally counts some +//! metrics related to access counts which can be helpful for profiling/debugging +use super::{CachedItem, CACHE_CLEAN_FREQUENCY_MS, DEFAULT_ITEM_LIFETIME_MS}; use crate::storage::DbRecord; +#[cfg(feature = "memory_pressure")] +use crate::storage::SizeOf; use crate::storage::Storable; use log::{debug, error, info, trace, warn}; -use std::collections::{HashMap, HashSet}; +use std::collections::HashMap; +use std::sync::atomic::{AtomicBool, Ordering}; use std::sync::Arc; -use std::time::{Duration, Instant}; - -// item's live for 30s -const DEFAULT_ITEM_LIFETIME_MS: u64 = 30000; -// clean the cache every 15s -const CACHE_CLEAN_FREQUENCY_MS: u64 = 15000; - -struct CachedItem { - expiration: Instant, - data: DbRecord, -} +use std::time::Duration; +use std::time::Instant; /// Implements a basic cahce with timing information which automatically flushes /// expired entries and removes them @@ -30,7 +26,7 @@ pub struct TimedCache { azks: Arc>>, map: Arc, CachedItem>>>, last_clean: Arc>, - can_clean: Arc>, + can_clean: Arc, item_lifetime: Duration, hit_count: Arc>, } @@ -72,8 +68,8 @@ impl Clone for TimedCache { impl TimedCache { async fn clean(&self) { - let can_clean_guard = self.can_clean.read().await; - if !*can_clean_guard { + let can_clean = self.can_clean.load(Ordering::Relaxed); + if !can_clean { // cleaning is disabled return; } @@ -86,19 +82,8 @@ impl TimedCache { if do_clean { debug!("BEGIN clean cache"); let now = Instant::now(); - let mut keys_to_flush = HashSet::new(); - let mut write = self.map.write().await; - for (key, value) in write.iter() { - if value.expiration < now { - keys_to_flush.insert(key.clone()); - } - } - if !keys_to_flush.is_empty() { - for key in keys_to_flush.into_iter() { - write.remove(&key); - } - } + write.retain(|_, v| v.expiration >= now); debug!("END clean cache"); // update last clean time @@ -106,6 +91,15 @@ impl TimedCache { } } + #[cfg(feature = "memory_pressure")] + /// Measure the size of the underlying hashmap and storage utilized + pub async fn measure(&self) -> usize { + let read = self.map.read().await; + read.iter() + .map(|(key, item)| key.len() + item.size_of()) + .sum() + } + /// Create a new timed cache instance. You can supply an optional item lifetime parameter /// or take the default (30s) pub fn new(o_lifetime: Option) -> Self { @@ -117,7 +111,7 @@ impl TimedCache { azks: Arc::new(tokio::sync::RwLock::new(None)), map: Arc::new(tokio::sync::RwLock::new(HashMap::new())), last_clean: Arc::new(tokio::sync::RwLock::new(Instant::now())), - can_clean: Arc::new(tokio::sync::RwLock::new(true)), + can_clean: Arc::new(AtomicBool::new(true)), item_lifetime: lifetime, hit_count: Arc::new(tokio::sync::RwLock::new(0)), } @@ -153,8 +147,7 @@ impl TimedCache { debug!("END cache retrieve"); if let Some(result) = ptr.get(&full_key) { *(self.hit_count.write().await) += 1; - - let ignore_clean = !*self.can_clean.read().await; + let ignore_clean = !self.can_clean.load(Ordering::Relaxed); // if we've disabled cache cleaning, we're in the middle // of an in-memory transaction and should ignore expiration // of cache items until this flag is disabled again @@ -221,16 +214,14 @@ impl TimedCache { } /// Disable cache-cleaning (i.e. during a transaction) - pub async fn disable_clean(&self) { + pub fn disable_clean(&self) { debug!("Disabling MySQL object cache cleaning"); - let mut guard = self.can_clean.write().await; - (*guard) = false; + self.can_clean.store(false, Ordering::Relaxed); } /// Re-enable cache cleaning (i.e. when a transaction is over) - pub async fn enable_clean(&self) { + pub fn enable_clean(&self) { debug!("Enabling MySQL object cache cleaning"); - let mut guard = self.can_clean.write().await; - (*guard) = true; + self.can_clean.store(true, Ordering::Relaxed); } } diff --git a/akd/src/storage/caches/high_parallelism.rs b/akd/src/storage/caches/high_parallelism.rs new file mode 100644 index 00000000..60fb8383 --- /dev/null +++ b/akd/src/storage/caches/high_parallelism.rs @@ -0,0 +1,203 @@ +// Copyright (c) Meta Platforms, Inc. and affiliates. +// +// This source code is licensed under both the MIT license found in the +// LICENSE-MIT file in the root directory of this source tree and the Apache +// License, Version 2.0 found in the LICENSE-APACHE file in the root directory +// of this source tree. + +//! This module implements a higher-parallelism, async temporary cache for database +//! objects + +use super::{CachedItem, CACHE_CLEAN_FREQUENCY_MS, DEFAULT_ITEM_LIFETIME_MS}; +use crate::storage::DbRecord; +#[cfg(feature = "memory_pressure")] +use crate::storage::SizeOf; +use crate::storage::Storable; +use dashmap::DashMap; +use log::debug; +use std::sync::atomic::{AtomicBool, Ordering}; +use std::sync::Arc; +use std::time::{Duration, Instant}; + +/// Implements a basic cahce with timing information which automatically flushes +/// expired entries and removes them +pub struct TimedCache { + azks: Arc>>, + map: Arc, CachedItem>>, + last_clean: Arc>, + can_clean: Arc, + item_lifetime: Duration, +} + +impl TimedCache { + /// Log cache access metrics along with size information + pub async fn log_metrics(&self, _level: log::Level) { + // in high-parallelism, we don't keep any metric counters to minimize thread locking + } +} + +impl Clone for TimedCache { + fn clone(&self) -> Self { + TimedCache { + azks: self.azks.clone(), + map: self.map.clone(), + last_clean: self.last_clean.clone(), + can_clean: self.can_clean.clone(), + item_lifetime: self.item_lifetime, + } + } +} + +impl TimedCache { + async fn clean(&self) { + let can_clean = self.can_clean.load(Ordering::Relaxed); + if !can_clean { + // cleaning is disabled + return; + } + + let do_clean = { + // we need the {} brackets in order to release the read lock, since we _may_ acquire a write lock shortly later + *(self.last_clean.read().await) + Duration::from_millis(CACHE_CLEAN_FREQUENCY_MS) + < Instant::now() + }; + if do_clean { + let mut last_clean_write = self.last_clean.write().await; + debug!("BEGIN clean cache"); + + let now = Instant::now(); + self.map.retain(|_, v| v.expiration >= now); + + debug!("END clean cache"); + + // update last clean time + *last_clean_write = Instant::now(); + } + } + + #[cfg(feature = "memory_pressure")] + /// Measure the size of the underlying hashmap and storage utilized + pub fn measure(&self) -> usize { + self.map + .iter() + .map(|(key, item)| key.len() + item.size_of()) + .sum() + } + + /// Create a new timed cache instance. You can supply an optional item lifetime parameter + /// or take the default (30s) + pub fn new(o_lifetime: Option) -> Self { + let lifetime = match o_lifetime { + Some(life) if life > Duration::from_secs(1) => life, + _ => Duration::from_millis(DEFAULT_ITEM_LIFETIME_MS), + }; + Self { + azks: Arc::new(tokio::sync::RwLock::new(None)), + map: Arc::new(DashMap::new()), + last_clean: Arc::new(tokio::sync::RwLock::new(Instant::now())), + can_clean: Arc::new(AtomicBool::new(true)), + item_lifetime: lifetime, + } + } + + /// Perform a hit-test of the cache for a given key. If successful, Some(record) will be returned + pub async fn hit_test(&self, key: &St::StorageKey) -> Option { + self.clean().await; + + debug!("BEGIN cache retrieve {:?}", key); + + let full_key = St::get_full_binary_key_id(key); + + // special case for AZKS + if full_key + == crate::append_only_zks::Azks::get_full_binary_key_id( + &crate::append_only_zks::DEFAULT_AZKS_KEY, + ) + { + // someone's requesting the AZKS object, return it from the special "cache" storage + let record = self.azks.read().await.clone(); + + debug!("END cache retrieve"); + + // AZKS objects cannot expire, they need to be manually flushed, so we don't need + // to check the expiration as below + return record; + } + + if let Some(result) = self.map.get(&full_key) { + let ignore_clean = !self.can_clean.load(Ordering::Relaxed); + // if we've disabled cache cleaning, we're in the middle + // of an in-memory transaction and should ignore expiration + // of cache items until this flag is disabled again + if ignore_clean || result.expiration > Instant::now() { + debug!("END cache retrieve"); + return Some(result.data.clone()); + } + } + debug!("END cache retrieve"); + None + } + + /// Put an item into the cache + pub async fn put(&self, record: &DbRecord) { + self.clean().await; + + debug!("BEGIN cache put"); + let key = record.get_full_binary_id(); + + // special case for AZKS + if let DbRecord::Azks(azks_ref) = &record { + let mut guard = self.azks.write().await; + *guard = Some(DbRecord::Azks(azks_ref.clone())); + } else { + let item = CachedItem { + expiration: Instant::now() + self.item_lifetime, + data: record.clone(), + }; + self.map.insert(key, item); + } + debug!("END cache put"); + } + + /// Put a batch of items into the cache, utilizing a single write lock + pub async fn batch_put(&self, records: &[DbRecord]) { + self.clean().await; + + debug!("BEGIN cache put batch"); + for record in records.iter() { + if let DbRecord::Azks(azks_ref) = &record { + let mut azks_guard = self.azks.write().await; + *azks_guard = Some(DbRecord::Azks(azks_ref.clone())); + } else { + let key = record.get_full_binary_id(); + let item = CachedItem { + expiration: Instant::now() + self.item_lifetime, + data: record.clone(), + }; + self.map.insert(key, item); + } + } + debug!("END cache put batch"); + } + + /// Flush the cache + pub async fn flush(&self) { + debug!("BEGIN cache flush"); + self.map.clear(); + let mut azks_guard = self.azks.write().await; + *azks_guard = None; + debug!("END cache flush"); + } + + /// Disable cache-cleaning (i.e. during a transaction) + pub fn disable_clean(&self) { + debug!("Disabling MySQL object cache cleaning"); + self.can_clean.store(false, Ordering::Relaxed); + } + + /// Re-enable cache cleaning (i.e. when a transaction is over) + pub fn enable_clean(&self) { + debug!("Enabling MySQL object cache cleaning"); + self.can_clean.store(true, Ordering::Relaxed); + } +} diff --git a/akd/src/storage/caches/mod.rs b/akd/src/storage/caches/mod.rs new file mode 100644 index 00000000..7c9a9f8f --- /dev/null +++ b/akd/src/storage/caches/mod.rs @@ -0,0 +1,47 @@ +// Copyright (c) Meta Platforms, Inc. and affiliates. +// +// This source code is licensed under both the MIT license found in the +// LICENSE-MIT file in the root directory of this source tree and the Apache +// License, Version 2.0 found in the LICENSE-APACHE file in the root directory +// of this source tree. + +//! This module handles various types of caches supported in the AKD crate which are +//! helpful for caching storage results for faster re-access + +use crate::storage::DbRecord; +use std::time::Instant; + +/// item's live for 30s +pub(crate) const DEFAULT_ITEM_LIFETIME_MS: u64 = 30000; +/// clean the cache every 15s +pub(crate) const CACHE_CLEAN_FREQUENCY_MS: u64 = 15000; +#[cfg(feature = "memory_pressure")] +/// Default memory limit in bytes ~ 1GB +pub(crate) const DEFAULT_MEMORY_LIMIT_BYTES: usize = 1024 * 1024 * 1024; + +pub(crate) struct CachedItem { + pub(crate) expiration: Instant, + pub(crate) data: DbRecord, +} + +impl super::SizeOf for CachedItem { + fn size_of(&self) -> usize { + // the size of an "Instant" varies based on the underlying implementation, so + // we assume the largest which is 16 bytes on linux + 16 + self.data.size_of() + } +} + +// -------- sub modules -------- // + +#[cfg(not(feature = "high_parallelism"))] +pub mod basic; +#[cfg(feature = "high_parallelism")] +pub mod high_parallelism; + +// -------- cache exports -------- // + +#[cfg(not(feature = "high_parallelism"))] +pub use basic::TimedCache; +#[cfg(feature = "high_parallelism")] +pub use high_parallelism::TimedCache; diff --git a/akd/src/storage/mod.rs b/akd/src/storage/mod.rs index 3118618b..59c95804 100644 --- a/akd/src/storage/mod.rs +++ b/akd/src/storage/mod.rs @@ -17,7 +17,7 @@ use std::collections::HashMap; use std::hash::Hash; use std::marker::Send; -pub mod timed_cache; +pub mod caches; pub mod transaction; pub mod types; @@ -29,6 +29,12 @@ pub mod memory; #[cfg(any(test, feature = "public-tests"))] pub mod tests; +/// Support getting the size of a struct or item in bytes +pub trait SizeOf { + /// Retrieve the size of the item in bytes + fn size_of(&self) -> usize; +} + /// Storable represents an _item_ which can be stored in the storage layer #[cfg(feature = "serde_serialization")] pub trait Storable: Clone + Serialize + DeserializeOwned + Sync { diff --git a/akd/src/storage/types.rs b/akd/src/storage/types.rs index 660aee43..4759e1fa 100644 --- a/akd/src/storage/types.rs +++ b/akd/src/storage/types.rs @@ -44,6 +44,12 @@ pub struct AkdLabel( pub Vec, ); +impl crate::storage::SizeOf for AkdLabel { + fn size_of(&self) -> usize { + self.0.len() + } +} + impl AkdLabel { /// Build an [`AkdLabel`] struct from an UTF8 string pub fn from_utf8_str(value: &str) -> Self { @@ -76,6 +82,12 @@ pub struct AkdValue( pub Vec, ); +impl crate::storage::SizeOf for AkdValue { + fn size_of(&self) -> usize { + self.0.len() + } +} + impl AkdValue { /// Build an [`AkdValue`] struct from an UTF8 string pub fn from_utf8_str(value: &str) -> Self { @@ -118,6 +130,16 @@ pub struct ValueState { pub username: AkdLabel, } +impl super::SizeOf for ValueState { + fn size_of(&self) -> usize { + self.plaintext_val.size_of() + + std::mem::size_of::() + + self.label.size_of() + + std::mem::size_of::() + + self.username.size_of() + } +} + impl crate::storage::Storable for ValueState { type StorageKey = ValueStateKey; @@ -216,6 +238,16 @@ pub enum DbRecord { ValueState(ValueState), } +impl super::SizeOf for DbRecord { + fn size_of(&self) -> usize { + match &self { + DbRecord::Azks(azks) => azks.size_of(), + DbRecord::TreeNode(node) => node.size_of(), + DbRecord::ValueState(state) => state.size_of(), + } + } +} + impl Clone for DbRecord { fn clone(&self) -> Self { match &self { diff --git a/akd/src/tree_node.rs b/akd/src/tree_node.rs index cb948fa5..e1929d05 100644 --- a/akd/src/tree_node.rs +++ b/akd/src/tree_node.rs @@ -42,6 +42,12 @@ pub enum NodeType { Interior = 3, } +impl crate::storage::SizeOf for NodeType { + fn size_of(&self) -> usize { + 1 + } +} + impl NodeType { pub(crate) fn from_u8(code: u8) -> Self { match code { @@ -83,6 +89,14 @@ pub struct TreeNodeWithPreviousValue { pub previous_node: Option, } +impl crate::storage::SizeOf for TreeNodeWithPreviousValue { + fn size_of(&self) -> usize { + self.label.size_of() + + self.latest_node.size_of() + + self.previous_node.as_ref().map_or(8, |v| v.size_of() + 8) + } +} + impl Storable for TreeNodeWithPreviousValue { type StorageKey = NodeKey; @@ -239,6 +253,18 @@ pub struct TreeNode { pub hash: [u8; 32], } +impl crate::storage::SizeOf for TreeNode { + fn size_of(&self) -> usize { + self.label.size_of() + + std::mem::size_of::() * 2 + + self.parent.size_of() + + self.node_type.size_of() + + self.left_child.as_ref().map_or(8, |v| v.size_of() + 8) + + self.right_child.as_ref().map_or(8, |v| v.size_of() + 8) + + 32 + } +} + impl TreeNode { // Storage operations pub(crate) async fn write_to_storage( diff --git a/akd_mysql/src/mysql.rs b/akd_mysql/src/mysql.rs index 28af52a7..1827ea56 100644 --- a/akd_mysql/src/mysql.rs +++ b/akd_mysql/src/mysql.rs @@ -30,7 +30,7 @@ use tokio::time::{Duration, Instant}; type MySqlError = mysql_async::Error; type LocalTransaction = akd::storage::transaction::Transaction; -use akd::storage::timed_cache::*; +use akd::storage::caches::*; const TABLE_AZKS: &str = crate::mysql_storables::TABLE_AZKS; const TABLE_HISTORY_TREE_NODES: &str = crate::mysql_storables::TABLE_HISTORY_TREE_NODES; @@ -720,7 +720,7 @@ impl Storage for AsyncMySqlDatabase { // disable the cache cleaning since we're in a write transaction // and will want to keep cache'd objects for the life of the transaction if let Some(cache) = &self.cache { - cache.disable_clean().await; + cache.disable_clean(); } self.trans.begin_transaction().await @@ -731,7 +731,7 @@ impl Storage for AsyncMySqlDatabase { // The transaction is now complete (or reverted) and therefore we can re-enable // the cache cleaning status if let Some(cache) = &self.cache { - cache.enable_clean().await; + cache.enable_clean(); } // this retrieves all the trans operations, and "de-activates" the transaction flag @@ -753,7 +753,7 @@ impl Storage for AsyncMySqlDatabase { // The transaction is being reverted and therefore we can re-enable // the cache cleaning status if let Some(cache) = &self.cache { - cache.enable_clean().await; + cache.enable_clean(); } self.trans.rollback_transaction().await From d505e49ccf797b9d800ae4f77d907faf56d9ca8a Mon Sep 17 00:00:00 2001 From: Sean Lawlor Date: Tue, 22 Nov 2022 12:13:40 -0500 Subject: [PATCH 02/13] Adding support for memory pressure processing, such that we can protect out-of-memory (OOM) crashes --- .github/workflows/mysql.yml | 6 +++ akd/src/storage/caches/basic.rs | 52 +++++++++++++++++-- akd/src/storage/caches/high_parallelism.rs | 58 ++++++++++++++++++++-- akd/src/storage/caches/mod.rs | 1 - akd_mysql/src/mysql.rs | 10 ++-- integration_tests/Cargo.toml | 7 ++- integration_tests/src/mysql_tests.rs | 4 ++ 7 files changed, 126 insertions(+), 12 deletions(-) diff --git a/.github/workflows/mysql.yml b/.github/workflows/mysql.yml index 1e70d6ba..391a153e 100644 --- a/.github/workflows/mysql.yml +++ b/.github/workflows/mysql.yml @@ -62,6 +62,12 @@ jobs: command: test args: --manifest-path Cargo.toml -p akd_integration_tests + - name: Run integration tests with high-parallelism and memory pressure support + uses: actions-rs/cargo@v1 + with: + command: test + args: --manifest-path Cargo.toml -p akd_integration_tests --features high_parallelism,memory_pressure,limited_memory_test + - name: Cleanup docker container run: docker-compose -f docker-compose.yml down -v diff --git a/akd/src/storage/caches/basic.rs b/akd/src/storage/caches/basic.rs index 7f01462c..4db7a931 100644 --- a/akd/src/storage/caches/basic.rs +++ b/akd/src/storage/caches/basic.rs @@ -8,7 +8,9 @@ //! This module implements a basic async timed cache. It additionally counts some //! metrics related to access counts which can be helpful for profiling/debugging -use super::{CachedItem, CACHE_CLEAN_FREQUENCY_MS, DEFAULT_ITEM_LIFETIME_MS}; +use super::{ + CachedItem, CACHE_CLEAN_FREQUENCY_MS, DEFAULT_ITEM_LIFETIME_MS, DEFAULT_MEMORY_LIMIT_BYTES, +}; use crate::storage::DbRecord; #[cfg(feature = "memory_pressure")] use crate::storage::SizeOf; @@ -29,6 +31,7 @@ pub struct TimedCache { can_clean: Arc, item_lifetime: Duration, hit_count: Arc>, + memory_limit_bytes: usize, } impl TimedCache { @@ -62,6 +65,7 @@ impl Clone for TimedCache { can_clean: self.can_clean.clone(), item_lifetime: self.item_lifetime, hit_count: self.hit_count.clone(), + memory_limit_bytes: self.memory_limit_bytes, } } } @@ -81,9 +85,49 @@ impl TimedCache { }; if do_clean { debug!("BEGIN clean cache"); - let now = Instant::now(); let mut write = self.map.write().await; + let now = Instant::now(); + #[cfg(not(feature = "memory_pressure"))] write.retain(|_, v| v.expiration >= now); + #[cfg(feature = "memory_pressure")] + { + let mut retained_size = 0; + let mut retained_elements = 0f64; + write.retain(|k, v| { + if v.expiration >= now { + retained_size += k.len() + v.size_of(); + retained_elements += 1.0; + true + } else { + false + } + }); + debug!("Retained cache size is {} bytes", retained_size); + if retained_size > self.memory_limit_bytes { + debug!("BEGIN cache memory pressure clean"); + info!("Retained cache size has exceeded the predefined limit, cleaning old entries"); + // calculate the percentage we'd need to trim off to get to 100% utilization and take another 5% + let percent_clean = + 0.05 + 1.0 - (self.memory_limit_bytes as f64) / (retained_size as f64); + // convert that to the number of items to delete based on the size of the dictionary + let num_clean = (retained_elements * percent_clean).round() as usize; + // sort the dict based on the oldest entries + let mut keys_and_expiration = write + .iter() + .map(|(k, v)| (k.clone(), v.expiration)) + .collect::>(); + keys_and_expiration.sort_by(|(_, a), (_, b)| a.cmp(b)); + // take those old entries, and remove them + for key in keys_and_expiration + .into_iter() + .take(num_clean) + .map(|(k, _)| k) + { + write.remove(&key); + } + debug!("END cache memory pressure clean") + } + } debug!("END clean cache"); // update last clean time @@ -102,11 +146,12 @@ impl TimedCache { /// Create a new timed cache instance. You can supply an optional item lifetime parameter /// or take the default (30s) - pub fn new(o_lifetime: Option) -> Self { + pub fn new(o_lifetime: Option, o_memory_limit_bytes: Option) -> Self { let lifetime = match o_lifetime { Some(life) if life > Duration::from_secs(1) => life, _ => Duration::from_millis(DEFAULT_ITEM_LIFETIME_MS), }; + let memory_limit_bytes: usize = o_memory_limit_bytes.unwrap_or(DEFAULT_MEMORY_LIMIT_BYTES); Self { azks: Arc::new(tokio::sync::RwLock::new(None)), map: Arc::new(tokio::sync::RwLock::new(HashMap::new())), @@ -114,6 +159,7 @@ impl TimedCache { can_clean: Arc::new(AtomicBool::new(true)), item_lifetime: lifetime, hit_count: Arc::new(tokio::sync::RwLock::new(0)), + memory_limit_bytes, } } diff --git a/akd/src/storage/caches/high_parallelism.rs b/akd/src/storage/caches/high_parallelism.rs index 60fb8383..07e89101 100644 --- a/akd/src/storage/caches/high_parallelism.rs +++ b/akd/src/storage/caches/high_parallelism.rs @@ -8,13 +8,17 @@ //! This module implements a higher-parallelism, async temporary cache for database //! objects -use super::{CachedItem, CACHE_CLEAN_FREQUENCY_MS, DEFAULT_ITEM_LIFETIME_MS}; +use super::{ + CachedItem, CACHE_CLEAN_FREQUENCY_MS, DEFAULT_ITEM_LIFETIME_MS, DEFAULT_MEMORY_LIMIT_BYTES, +}; use crate::storage::DbRecord; #[cfg(feature = "memory_pressure")] use crate::storage::SizeOf; use crate::storage::Storable; use dashmap::DashMap; use log::debug; +#[cfg(feature = "memory_pressure")] +use log::info; use std::sync::atomic::{AtomicBool, Ordering}; use std::sync::Arc; use std::time::{Duration, Instant}; @@ -27,6 +31,7 @@ pub struct TimedCache { last_clean: Arc>, can_clean: Arc, item_lifetime: Duration, + memory_limit_bytes: usize, } impl TimedCache { @@ -44,6 +49,7 @@ impl Clone for TimedCache { last_clean: self.last_clean.clone(), can_clean: self.can_clean.clone(), item_lifetime: self.item_lifetime, + memory_limit_bytes: self.memory_limit_bytes, } } } @@ -66,7 +72,51 @@ impl TimedCache { debug!("BEGIN clean cache"); let now = Instant::now(); + #[cfg(not(feature = "memory_pressure"))] self.map.retain(|_, v| v.expiration >= now); + #[cfg(feature = "memory_pressure")] + { + let mut retained_size = 0; + let mut retained_elements = 0f64; + let mut removed = 0; + self.map.retain(|k, v| { + if v.expiration >= now { + retained_size += k.len() + v.size_of(); + retained_elements += 1.0; + true + } else { + removed += 1; + false + } + }); + info!("Removed {} expired elements from the cache", removed); + debug!("Retained cache size is {} bytes", retained_size); + if retained_size > self.memory_limit_bytes { + debug!("BEGIN cache memory pressure clean"); + info!("Retained cache size has exceeded the predefined limit, cleaning old entries"); + // calculate the percentage we'd need to trim off to get to 100% utilization and take another 5% + let percent_clean = + 0.05 + 1.0 - (self.memory_limit_bytes as f64) / (retained_size as f64); + // convert that to the number of items to delete based on the size of the dictionary + let num_clean = (retained_elements * percent_clean).round() as usize; + // sort the dict based on the oldest entries + let mut keys_and_expiration = self + .map + .iter() + .map(|kv| (kv.key().clone(), kv.value().expiration)) + .collect::>(); + keys_and_expiration.sort_by(|(_, a), (_, b)| a.cmp(b)); + // take those old entries, and remove them + for key in keys_and_expiration + .into_iter() + .take(num_clean) + .map(|(k, _)| k) + { + self.map.remove(&key); + } + debug!("END cache memory pressure clean") + } + } debug!("END clean cache"); @@ -80,23 +130,25 @@ impl TimedCache { pub fn measure(&self) -> usize { self.map .iter() - .map(|(key, item)| key.len() + item.size_of()) + .map(|kv| kv.key().len() + kv.value().size_of()) .sum() } /// Create a new timed cache instance. You can supply an optional item lifetime parameter /// or take the default (30s) - pub fn new(o_lifetime: Option) -> Self { + pub fn new(o_lifetime: Option, o_memory_limit_bytes: Option) -> Self { let lifetime = match o_lifetime { Some(life) if life > Duration::from_secs(1) => life, _ => Duration::from_millis(DEFAULT_ITEM_LIFETIME_MS), }; + let memory_limit_bytes: usize = o_memory_limit_bytes.unwrap_or(DEFAULT_MEMORY_LIMIT_BYTES); Self { azks: Arc::new(tokio::sync::RwLock::new(None)), map: Arc::new(DashMap::new()), last_clean: Arc::new(tokio::sync::RwLock::new(Instant::now())), can_clean: Arc::new(AtomicBool::new(true)), item_lifetime: lifetime, + memory_limit_bytes, } } diff --git a/akd/src/storage/caches/mod.rs b/akd/src/storage/caches/mod.rs index 7c9a9f8f..4b74ea85 100644 --- a/akd/src/storage/caches/mod.rs +++ b/akd/src/storage/caches/mod.rs @@ -15,7 +15,6 @@ use std::time::Instant; pub(crate) const DEFAULT_ITEM_LIFETIME_MS: u64 = 30000; /// clean the cache every 15s pub(crate) const CACHE_CLEAN_FREQUENCY_MS: u64 = 15000; -#[cfg(feature = "memory_pressure")] /// Default memory limit in bytes ~ 1GB pub(crate) const DEFAULT_MEMORY_LIMIT_BYTES: usize = 1024 * 1024 * 1024; diff --git a/akd_mysql/src/mysql.rs b/akd_mysql/src/mysql.rs index 1827ea56..0e4fb93f 100644 --- a/akd_mysql/src/mysql.rs +++ b/akd_mysql/src/mysql.rs @@ -65,8 +65,8 @@ pub enum MySqlCacheOptions { None, /// Utilize the default caching settings Default, - /// Customize the caching options (cache item duration) - Specific(std::time::Duration), + /// Customize the caching options (cache item duration, memory_limit_bytes) + Specific(std::time::Duration, Option), } /// Represents an _asynchronous_ connection to a MySQL database @@ -160,8 +160,10 @@ impl<'a> AsyncMySqlDatabase { let cache = match cache_options { MySqlCacheOptions::None => None, - MySqlCacheOptions::Default => Some(TimedCache::new(None)), - MySqlCacheOptions::Specific(timing) => Some(TimedCache::new(Some(timing))), + MySqlCacheOptions::Default => Some(TimedCache::new(None, None)), + MySqlCacheOptions::Specific(timing, memory_limit_bytes) => { + Some(TimedCache::new(Some(timing), memory_limit_bytes)) + } }; Self { diff --git a/integration_tests/Cargo.toml b/integration_tests/Cargo.toml index 60845cd3..ea852453 100644 --- a/integration_tests/Cargo.toml +++ b/integration_tests/Cargo.toml @@ -5,9 +5,15 @@ authors = ["Harjasleen Malvai ", "Kevin Lewi ", "S edition = "2018" publish = false +[features] +high_parallelism = ["akd/high_parallelism"] +memory_pressure = ["akd/memory_pressure"] +limited_memory_test = [] + [dependencies] winter-crypto = "0.2" winter-math = "0.2" +akd = { path = "../akd", features = ["public-tests", "rand", "serde_serialization"] } [dev-dependencies] log = { version = "0.4.8", features = ["kv_unstable"] } @@ -20,6 +26,5 @@ thread-id = "3" multi_log = "0.1" hex = "0.4.3" -akd = { path = "../akd", features = ["public-tests", "rand", "serde_serialization"] } akd_mysql = { path = "../akd_mysql" } akd_test_tools = { path = "../akd_test_tools" } diff --git a/integration_tests/src/mysql_tests.rs b/integration_tests/src/mysql_tests.rs index a92ff4aa..108fe462 100644 --- a/integration_tests/src/mysql_tests.rs +++ b/integration_tests/src/mysql_tests.rs @@ -36,6 +36,10 @@ async fn test_directory_operations() { Option::from("root"), Option::from("example"), Option::from(8001), + // clean every 1s and limit to 1KB + #[cfg(feature = "limited_memory_test")] + MySqlCacheOptions::Specific(std::time::Duration::from_secs(1), Some(1024)), + #[cfg(not(feature = "limited_memory_test"))] MySqlCacheOptions::Default, 200, ) From 9d0af9a5e48b791ce55a43dde3935301750bb946 Mon Sep 17 00:00:00 2001 From: Sean Lawlor Date: Wed, 23 Nov 2022 13:33:41 -0500 Subject: [PATCH 03/13] AKD rewrite with new storage model. This abstracts the notion of caching and transactions away from the underlying database implementation so the implementer won't care. --- akd/Cargo.toml | 2 + akd/src/append_only_zks.rs | 82 +-- akd/src/auditor.rs | 14 +- akd/src/directory.rs | 22 +- akd/src/lib.rs | 39 +- akd/src/serialization.rs | 10 +- akd/src/storage/caches/basic.rs | 56 +- akd/src/storage/caches/high_parallelism.rs | 36 +- akd/src/storage/memory.rs | 536 +------------------ akd/src/storage/mod.rs | 47 +- akd/src/storage/storage.rs | 573 +++++++++++++++++++++ akd/src/storage/tests.rs | 23 +- akd/src/storage/transaction.rs | 118 ++++- akd/src/tests.rs | 51 +- akd/src/tree_node.rs | 57 +- 15 files changed, 958 insertions(+), 708 deletions(-) create mode 100644 akd/src/storage/storage.rs diff --git a/akd/Cargo.toml b/akd/Cargo.toml index 741b084e..c6271758 100644 --- a/akd/Cargo.toml +++ b/akd/Cargo.toml @@ -28,6 +28,8 @@ high_parallelism = ["dashmap"] # Add support for handling memory pressure process on the caches. This costs some performance, however # provides the ability to protect against OOMing memory_pressure = [] +# Collect runtime metrics on db access calls + timing +runtime_metrics = [] [dependencies] ## Required dependencies ## diff --git a/akd/src/append_only_zks.rs b/akd/src/append_only_zks.rs index 29071097..c80a8d50 100644 --- a/akd/src/append_only_zks.rs +++ b/akd/src/append_only_zks.rs @@ -6,15 +6,15 @@ // of this source tree. //! An implementation of an append-only zero knowledge set +use crate::serialization::to_digest; +use crate::storage::storage::StorageManager; use crate::{ errors::TreeNodeError, proof_structs::{AppendOnlyProof, MembershipProof, NonMembershipProof, SingleAppendOnlyProof}, - storage::{SizeOf, Storable, Storage}, + storage::{Database as Storage, SizeOf, Storable}, tree_node::*, }; -use crate::serialization::to_digest; - use crate::storage::types::StorageType; use crate::{errors::*, node_label::*, tree_node::TreeNode, ARITY, *}; use async_recursion::async_recursion; @@ -86,7 +86,9 @@ impl Clone for Azks { impl Azks { /// Creates a new azks - pub async fn new(storage: &S) -> Result { + pub async fn new( + storage: &StorageManager, + ) -> Result { create_empty_root::(storage, Option::Some(0), Option::Some(0)).await?; let azks = Azks { latest_epoch: 0, @@ -101,7 +103,7 @@ impl Azks { #[cfg(test)] pub async fn insert_leaf( &mut self, - storage: &S, + storage: &StorageManager, node: Node, epoch: u64, ) -> Result<(), AkdError> { @@ -134,7 +136,7 @@ impl Azks { /// Insert a batch of new leaves pub async fn batch_insert_leaves( &mut self, - storage: &S, + storage: &StorageManager, insertion_set: Vec>, ) -> Result<(), AkdError> { self.batch_insert_leaves_helper::<_, H>(storage, insertion_set, false) @@ -143,7 +145,7 @@ impl Azks { async fn preload_nodes_for_insertion( &self, - storage: &S, + storage: &StorageManager, insertion_set: &[Node], ) -> Result { let prefixes_set = crate::utils::build_prefixes_set( @@ -160,7 +162,7 @@ impl Azks { /// Preloads given nodes using breadth-first search. pub async fn bfs_preload_nodes( &self, - storage: &S, + storage: &StorageManager, nodes_to_load: HashSet, ) -> Result { let mut load_count: u64 = 0; @@ -200,7 +202,7 @@ impl Azks { /// bool keeps track of this. pub async fn batch_insert_leaves_helper( &mut self, - storage: &S, + storage: &StorageManager, insertion_set: Vec>, append_only_exclude_usage: bool, ) -> Result<(), AkdError> { @@ -284,7 +286,7 @@ impl Azks { // Assumes the verifier has access to the root at epoch pub async fn get_membership_proof( &self, - storage: &S, + storage: &StorageManager, label: NodeLabel, _epoch: u64, ) -> Result, AkdError> { @@ -301,7 +303,7 @@ impl Azks { /// none of the children is equal to the given label. pub async fn get_non_membership_proof( &self, - storage: &S, + storage: &StorageManager, label: NodeLabel, ) -> Result, AkdError> { let (longest_prefix_membership_proof, lcp_node_label) = @@ -361,7 +363,7 @@ impl Azks { /// * `start_epoch` and `end_epoch` are both existing epochs of this AZKS pub async fn get_append_only_proof( &self, - storage: &S, + storage: &StorageManager, start_epoch: u64, end_epoch: u64, ) -> Result, AkdError> { @@ -432,7 +434,7 @@ impl Azks { async fn gather_audit_proof_nodes( &self, nodes: Vec, - storage: &S, + storage: &StorageManager, start_epoch: u64, end_epoch: u64, ) -> Result { @@ -463,7 +465,7 @@ impl Azks { #[async_recursion] async fn get_append_only_proof_helper( &self, - storage: &S, + storage: &StorageManager, node: TreeNode, start_epoch: u64, end_epoch: u64, @@ -527,7 +529,7 @@ impl Azks { /// Gets the root hash for this azks pub async fn get_root_hash( &self, - storage: &S, + storage: &StorageManager, ) -> Result { self.get_root_hash_at_epoch::<_, H>(storage, self.get_latest_epoch()) .await @@ -538,7 +540,7 @@ impl Azks { /// the azks does, this would never be called at an epoch before the birth of the root node. pub async fn get_root_hash_at_epoch( &self, - storage: &S, + storage: &StorageManager, epoch: u64, ) -> Result { if self.latest_epoch < epoch { @@ -573,7 +575,7 @@ impl Azks { /// This is meant to be used in both, getting membership proofs and getting non-membership proofs. pub async fn get_membership_proof_and_node( &self, - storage: &S, + storage: &StorageManager, label: NodeLabel, ) -> Result<(MembershipProof, NodeLabel), AkdError> { let mut layer_proofs = Vec::new(); @@ -683,7 +685,8 @@ mod tests { async fn test_batch_insert_basic() -> Result<(), AkdError> { let mut rng = OsRng; let num_nodes = 10; - let db = AsyncInMemoryDatabase::new(); + let database = AsyncInMemoryDatabase::new(); + let db = StorageManager::new_no_cache(&database); let mut azks1 = Azks::new::<_, Blake3>(&db).await?; azks1.increment_epoch(); @@ -699,7 +702,8 @@ mod tests { azks1.insert_leaf::<_, Blake3>(&db, node, 1).await?; } - let db2 = AsyncInMemoryDatabase::new(); + let database2 = AsyncInMemoryDatabase::new(); + let db2 = StorageManager::new_no_cache(&database2); let mut azks2 = Azks::new::<_, Blake3>(&db2).await?; azks2 @@ -719,7 +723,8 @@ mod tests { async fn test_insert_permuted() -> Result<(), AkdError> { let num_nodes = 10; let mut rng = OsRng; - let db = AsyncInMemoryDatabase::new(); + let database = AsyncInMemoryDatabase::new(); + let db = StorageManager::new_no_cache(&database); let mut azks1 = Azks::new::<_, Blake3>(&db).await?; azks1.increment_epoch(); let mut insertion_set: Vec> = vec![]; @@ -737,7 +742,8 @@ mod tests { // Try randomly permuting insertion_set.shuffle(&mut rng); - let db2 = AsyncInMemoryDatabase::new(); + let database2 = AsyncInMemoryDatabase::new(); + let db2 = StorageManager::new_no_cache(&database2); let mut azks2 = Azks::new::<_, Blake3>(&db2).await?; azks2 @@ -771,7 +777,8 @@ mod tests { // Try randomly permuting insertion_set.shuffle(&mut rng); - let db = AsyncInMemoryDatabase::new(); + let database = AsyncInMemoryDatabase::new(); + let db = StorageManager::new_no_cache(&database); let mut azks = Azks::new::<_, Blake3>(&db).await?; azks.batch_insert_leaves::<_, Blake3>(&db, insertion_set.clone()) .await?; @@ -801,7 +808,8 @@ mod tests { insertion_set.push(node); } - let db = AsyncInMemoryDatabase::new(); + let database = AsyncInMemoryDatabase::new(); + let db = StorageManager::new_no_cache(&database); let mut azks = Azks::new::<_, Blake3>(&db).await?; azks.batch_insert_leaves::<_, Blake3>(&db, insertion_set.clone()) .await?; @@ -833,7 +841,8 @@ mod tests { // Try randomly permuting insertion_set.shuffle(&mut rng); - let db = AsyncInMemoryDatabase::new(); + let database = AsyncInMemoryDatabase::new(); + let db = StorageManager::new_no_cache(&database); let mut azks = Azks::new::<_, Blake3>(&db).await?; azks.batch_insert_leaves::<_, Blake3>(&db, insertion_set.clone()) .await?; @@ -858,7 +867,8 @@ mod tests { #[tokio::test] async fn test_membership_proof_intermediate() -> Result<(), AkdError> { - let db = AsyncInMemoryDatabase::new(); + let database = AsyncInMemoryDatabase::new(); + let db = StorageManager::new_no_cache(&database); let mut insertion_set: Vec> = vec![]; insertion_set.push(Node { @@ -910,7 +920,8 @@ mod tests { let node = Node:: { label, hash }; insertion_set.push(node); } - let db = AsyncInMemoryDatabase::new(); + let database = AsyncInMemoryDatabase::new(); + let db = StorageManager::new_no_cache(&database); let mut azks = Azks::new::<_, Blake3>(&db).await?; let search_label = insertion_set[0].label; azks.batch_insert_leaves::<_, Blake3>(&db, insertion_set.clone()[1..2].to_vec()) @@ -939,7 +950,8 @@ mod tests { let node = Node:: { label, hash }; insertion_set.push(node); } - let db = AsyncInMemoryDatabase::new(); + let database = AsyncInMemoryDatabase::new(); + let db = StorageManager::new_no_cache(&database); let mut azks = Azks::new::<_, Blake3>(&db).await?; let search_label = insertion_set[num_nodes - 1].label; azks.batch_insert_leaves::<_, Blake3>( @@ -969,7 +981,8 @@ mod tests { let node = Node:: { label, hash }; insertion_set.push(node); } - let db = AsyncInMemoryDatabase::new(); + let database = AsyncInMemoryDatabase::new(); + let db = StorageManager::new_no_cache(&database); let mut azks = Azks::new::<_, Blake3>(&db).await?; let search_label = insertion_set[num_nodes - 1].label; azks.batch_insert_leaves::<_, Blake3>( @@ -986,7 +999,8 @@ mod tests { #[tokio::test] async fn test_append_only_proof_very_tiny() -> Result<(), AkdError> { - let db = AsyncInMemoryDatabase::new(); + let database = AsyncInMemoryDatabase::new(); + let db = StorageManager::new_no_cache(&database); let mut azks = Azks::new::<_, Blake3>(&db).await?; let mut insertion_set_1: Vec> = vec![]; @@ -1016,7 +1030,8 @@ mod tests { #[tokio::test] async fn test_append_only_proof_tiny() -> Result<(), AkdError> { - let db = AsyncInMemoryDatabase::new(); + let database = AsyncInMemoryDatabase::new(); + let db = StorageManager::new_no_cache(&database); let mut azks = Azks::new::<_, Blake3>(&db).await?; let mut insertion_set_1: Vec> = vec![]; @@ -1068,7 +1083,8 @@ mod tests { insertion_set_1.push(node); } - let db = AsyncInMemoryDatabase::new(); + let database = AsyncInMemoryDatabase::new(); + let db = StorageManager::new_no_cache(&database); let mut azks = Azks::new::<_, Blake3>(&db).await?; azks.batch_insert_leaves::<_, Blake3>(&db, insertion_set_1.clone()) .await?; @@ -1116,7 +1132,9 @@ mod tests { #[tokio::test] async fn future_epoch_throws_error() -> Result<(), AkdError> { - let db = AsyncInMemoryDatabase::new(); + let database = AsyncInMemoryDatabase::new(); + + let db = StorageManager::new_no_cache(&database); let azks = Azks::new::<_, Blake3>(&db).await?; let out = azks.get_root_hash_at_epoch::<_, Blake3>(&db, 123).await; diff --git a/akd/src/auditor.rs b/akd/src/auditor.rs index cbe0f4ba..f303e0c1 100644 --- a/akd/src/auditor.rs +++ b/akd/src/auditor.rs @@ -14,7 +14,7 @@ use winter_crypto::Hasher; use crate::{ errors::{AkdError, AuditorError, AzksError}, proof_structs::{AppendOnlyProof, SingleAppendOnlyProof}, - storage::memory::AsyncInMemoryDatabase, + storage::{memory::AsyncInMemoryDatabase, storage::StorageManager}, Azks, }; @@ -65,10 +65,12 @@ pub async fn verify_consecutive_append_only( let inserted = proof.inserted.clone(); let db = AsyncInMemoryDatabase::new(); - let mut azks = Azks::new::<_, H>(&db).await?; - azks.batch_insert_leaves_helper::<_, H>(&db, unchanged_nodes, true) + let manager = StorageManager::new_no_cache(&db); + + let mut azks = Azks::new::<_, H>(&manager).await?; + azks.batch_insert_leaves_helper::<_, H>(&manager, unchanged_nodes, true) .await?; - let computed_start_root_hash: H::Digest = azks.get_root_hash::<_, H>(&db).await?; + let computed_start_root_hash: H::Digest = azks.get_root_hash::<_, H>(&manager).await?; let mut verified = computed_start_root_hash == start_hash; azks.latest_epoch = epoch - 1; let updated_inserted = inserted @@ -79,9 +81,9 @@ pub async fn verify_consecutive_append_only( y }) .collect(); - azks.batch_insert_leaves_helper::<_, H>(&db, updated_inserted, true) + azks.batch_insert_leaves_helper::<_, H>(&manager, updated_inserted, true) .await?; - let computed_end_root_hash: H::Digest = azks.get_root_hash::<_, H>(&db).await?; + let computed_end_root_hash: H::Digest = azks.get_root_hash::<_, H>(&manager).await?; verified = verified && (computed_end_root_hash == end_hash); if !verified { return Err(AkdError::AzksErr(AzksError::VerifyAppendOnlyProof)); diff --git a/akd/src/directory.rs b/akd/src/directory.rs index 0a972037..64dace03 100644 --- a/akd/src/directory.rs +++ b/akd/src/directory.rs @@ -8,15 +8,13 @@ //! Implementation of a auditable key directory use crate::append_only_zks::Azks; - use crate::ecvrf::{VRFKeyStorage, VRFPublicKey}; -use crate::proof_structs::*; -use crate::{helper_structs::LookupInfo, EpochHash, Node}; - use crate::errors::{AkdError, DirectoryError, StorageError}; - +use crate::proof_structs::*; +use crate::storage::storage::StorageManager; use crate::storage::types::{AkdLabel, AkdValue, DbRecord, ValueState, ValueStateRetrievalFlag}; -use crate::storage::Storage; +use crate::storage::Database as Storage; +use crate::{helper_structs::LookupInfo, EpochHash, Node}; use log::{debug, error, info}; @@ -45,8 +43,8 @@ impl AkdLabel { } /// The representation of a auditable key directory -pub struct Directory { - storage: S, +pub struct Directory { + storage: StorageManager, vrf: V, hasher: PhantomData, read_only: bool, @@ -76,7 +74,11 @@ impl Directory { /// Creates a new (stateless) instance of a auditable key directory. /// Takes as input a pointer to the storage being used for this instance. /// The state is stored in the storage. - pub async fn new(storage: &S, vrf: &V, read_only: bool) -> Result { + pub async fn new( + storage: &StorageManager, + vrf: &V, + read_only: bool, + ) -> Result { let azks = Directory::::get_azks_from_storage(storage, false).await; if read_only && azks.is_err() { @@ -680,7 +682,7 @@ impl Directory { } async fn get_azks_from_storage( - storage: &S, + storage: &StorageManager, ignore_cache: bool, ) -> Result { let got = if ignore_cache { diff --git a/akd/src/lib.rs b/akd/src/lib.rs index 111e4aab..0532e669 100644 --- a/akd/src/lib.rs +++ b/akd/src/lib.rs @@ -49,15 +49,17 @@ //! as `false`. //! ``` //! use akd::Blake3; +//! use akd::storage::StorageManager; //! use akd::storage::memory::AsyncInMemoryDatabase; //! use akd::ecvrf::HardCodedAkdVRF; //! use akd::directory::Directory; //! //! let db = AsyncInMemoryDatabase::new(); +//! let storage_manager = StorageManager::new_no_cache(&db); //! let vrf = HardCodedAkdVRF{}; //! //! # tokio_test::block_on(async { -//! let mut akd = Directory::<_, _, Blake3>::new(&db, &vrf, false) +//! let mut akd = Directory::<_, _, Blake3>::new(&storage_manager, &vrf, false) //! .await //! .expect("Could not create a new directory"); //! # }); @@ -69,11 +71,13 @@ //! the new epoch number and root hash are returned. //! ``` //! # use akd::Blake3; +//! # use akd::storage::StorageManager; //! # use akd::storage::memory::AsyncInMemoryDatabase; //! # use akd::ecvrf::HardCodedAkdVRF; //! # use akd::directory::Directory; //! # //! # let db = AsyncInMemoryDatabase::new(); +//! # let storage_manager = StorageManager::new_no_cache(&db); //! # let vrf = HardCodedAkdVRF{}; //! use akd::EpochHash; //! use akd::storage::types::{AkdLabel, AkdValue}; @@ -84,10 +88,11 @@ //! (AkdLabel::from_utf8_str("second entry"), AkdValue::from_utf8_str("second value")), //! ]; //! # let db = AsyncInMemoryDatabase::new(); +//! # let storage_manager = StorageManager::new_no_cache(&db); //! //! # tokio_test::block_on(async { //! # let vrf = HardCodedAkdVRF{}; -//! # let mut akd = Directory::<_, _, Blake3>::new(&db, &vrf, false).await.unwrap(); +//! # let mut akd = Directory::<_, _, Blake3>::new(&storage_manager, &vrf, false).await.unwrap(); //! let EpochHash(epoch, root_hash) = akd.publish(entries) //! .await.expect("Error with publishing"); //! println!("Published epoch {} with root hash: {}", epoch, hex::encode(root_hash.as_bytes())); @@ -101,11 +106,13 @@ //! of a client lookup for an existing entry. //! ``` //! # use akd::Blake3; +//! # use akd::storage::StorageManager; //! # use akd::storage::memory::AsyncInMemoryDatabase; //! # use akd::ecvrf::HardCodedAkdVRF; //! # use akd::directory::Directory; //! # //! # let db = AsyncInMemoryDatabase::new(); +//! # let storage_manager = StorageManager::new_no_cache(&db); //! # let vrf = HardCodedAkdVRF{}; //! # use akd::EpochHash; //! # use akd::storage::types::{AkdLabel, AkdValue}; @@ -116,10 +123,11 @@ //! # (AkdLabel::from_utf8_str("second entry"), AkdValue::from_utf8_str("second value")), //! # ]; //! # let db = AsyncInMemoryDatabase::new(); +//! # let storage_manager = StorageManager::new_no_cache(&db); //! # //! # tokio_test::block_on(async { //! # let vrf = HardCodedAkdVRF{}; -//! # let mut akd = Directory::<_, _, Blake3>::new(&db, &vrf, false).await.unwrap(); +//! # let mut akd = Directory::<_, _, Blake3>::new(&storage_manager, &vrf, false).await.unwrap(); //! # let EpochHash(epoch, root_hash) = akd.publish(entries) //! # .await.expect("Error with publishing"); //! let lookup_proof = akd.lookup( @@ -132,11 +140,13 @@ //! the server's public key. //! ``` //! # use akd::Blake3; +//! # use akd::storage::StorageManager; //! # use akd::storage::memory::AsyncInMemoryDatabase; //! # use akd::ecvrf::HardCodedAkdVRF; //! # use akd::directory::Directory; //! # //! # let db = AsyncInMemoryDatabase::new(); +//! # let storage_manager = StorageManager::new_no_cache(&db); //! # let vrf = HardCodedAkdVRF{}; //! # use akd::EpochHash; //! # use akd::storage::types::{AkdLabel, AkdValue}; @@ -147,10 +157,11 @@ //! # (AkdLabel::from_utf8_str("second entry"), AkdValue::from_utf8_str("second value")), //! # ]; //! # let db = AsyncInMemoryDatabase::new(); +//! # let storage_manager = StorageManager::new_no_cache(&db); //! # //! # tokio_test::block_on(async { //! # let vrf = HardCodedAkdVRF{}; -//! # let mut akd = Directory::<_, _, Blake3>::new(&db, &vrf, false).await.unwrap(); +//! # let mut akd = Directory::<_, _, Blake3>::new(&storage_manager, &vrf, false).await.unwrap(); //! # let EpochHash(epoch, root_hash) = akd.publish(entries) //! # .await.expect("Error with publishing"); //! # let lookup_proof = akd.lookup( @@ -176,11 +187,13 @@ //! We can use [`Directory::key_history`] to prove the history of a key's values at a given epoch, as follows. //! ``` //! # use akd::Blake3; +//! # use akd::storage::StorageManager; //! # use akd::storage::memory::AsyncInMemoryDatabase; //! # use akd::ecvrf::HardCodedAkdVRF; //! # use akd::directory::Directory; //! # //! # let db = AsyncInMemoryDatabase::new(); +//! # let storage_manager = StorageManager::new_no_cache(&db); //! # let vrf = HardCodedAkdVRF{}; //! # use akd::EpochHash; //! # use akd::storage::types::{AkdLabel, AkdValue}; @@ -191,10 +204,11 @@ //! # (AkdLabel::from_utf8_str("second entry"), AkdValue::from_utf8_str("second value")), //! # ]; //! # let db = AsyncInMemoryDatabase::new(); +//! # let storage_manager = StorageManager::new_no_cache(&db); //! # //! # tokio_test::block_on(async { //! # let vrf = HardCodedAkdVRF{}; -//! # let mut akd = Directory::<_, _, Blake3>::new(&db, &vrf, false).await.unwrap(); +//! # let mut akd = Directory::<_, _, Blake3>::new(&storage_manager, &vrf, false).await.unwrap(); //! # let EpochHash(epoch, root_hash) = akd.publish(entries) //! # .await.expect("Error with publishing"); //! let history_proof = akd.key_history( @@ -206,11 +220,13 @@ //! with respect to the latest root hash and public key, as follows: //! ``` //! # use akd::Blake3; +//! # use akd::storage::StorageManager; //! # use akd::storage::memory::AsyncInMemoryDatabase; //! # use akd::ecvrf::HardCodedAkdVRF; //! # use akd::directory::Directory; //! # //! # let db = AsyncInMemoryDatabase::new(); +//! # let storage_manager = StorageManager::new_no_cache(&db); //! # let vrf = HardCodedAkdVRF{}; //! # use akd::EpochHash; //! # use akd::storage::types::{AkdLabel, AkdValue}; @@ -221,10 +237,11 @@ //! # (AkdLabel::from_utf8_str("second entry"), AkdValue::from_utf8_str("second value")), //! # ]; //! # let db = AsyncInMemoryDatabase::new(); +//! # let storage_manager = StorageManager::new_no_cache(&db); //! # //! # tokio_test::block_on(async { //! # let vrf = HardCodedAkdVRF{}; -//! # let mut akd = Directory::<_, _, Blake3>::new(&db, &vrf, false).await.unwrap(); +//! # let mut akd = Directory::<_, _, Blake3>::new(&storage_manager, &vrf, false).await.unwrap(); //! # let EpochHash(epoch, root_hash) = akd.publish(entries) //! # .await.expect("Error with publishing"); //! # let history_proof = akd.key_history( @@ -252,11 +269,13 @@ //! Below we illustrate how the server responds to an audit query between two epochs. //! ``` //! # use akd::Blake3; +//! # use akd::storage::StorageManager; //! # use akd::storage::memory::AsyncInMemoryDatabase; //! # use akd::ecvrf::HardCodedAkdVRF; //! # use akd::directory::Directory; //! # //! # let db = AsyncInMemoryDatabase::new(); +//! # let storage_manager = StorageManager::new_no_cache(&db); //! # let vrf = HardCodedAkdVRF{}; //! # use akd::EpochHash; //! # use akd::storage::types::{AkdLabel, AkdValue}; @@ -267,10 +286,11 @@ //! # (AkdLabel::from_utf8_str("second entry"), AkdValue::from_utf8_str("second value")), //! # ]; //! # let db = AsyncInMemoryDatabase::new(); +//! # let storage_manager = StorageManager::new_no_cache(&db); //! # //! # tokio_test::block_on(async { //! # let vrf = HardCodedAkdVRF{}; -//! # let mut akd = Directory::<_, _, Blake3>::new(&db, &vrf, false).await.unwrap(); +//! # let mut akd = Directory::<_, _, Blake3>::new(&storage_manager, &vrf, false).await.unwrap(); //! # let EpochHash(epoch, root_hash) = akd.publish(entries) //! # .await.expect("Error with publishing"); //! // Publish new entries into a second epoch @@ -289,11 +309,13 @@ //! The auditor then verifies the above [`AppendOnlyProof`] using [`auditor::audit_verify`]. //! ``` //! # use akd::Blake3; +//! # use akd::storage::StorageManager; //! # use akd::storage::memory::AsyncInMemoryDatabase; //! # use akd::ecvrf::HardCodedAkdVRF; //! # use akd::directory::Directory; //! # //! # let db = AsyncInMemoryDatabase::new(); +//! # let storage_manager = StorageManager::new_no_cache(&db); //! # let vrf = HardCodedAkdVRF{}; //! # use akd::EpochHash; //! # use akd::storage::types::{AkdLabel, AkdValue}; @@ -304,10 +326,11 @@ //! # (AkdLabel::from_utf8_str("second entry"), AkdValue::from_utf8_str("second value")), //! # ]; //! # let db = AsyncInMemoryDatabase::new(); +//! # let storage_manager = StorageManager::new_no_cache(&db); //! # //! # tokio_test::block_on(async { //! # let vrf = HardCodedAkdVRF{}; -//! # let mut akd = Directory::<_, _, Blake3>::new(&db, &vrf, false).await.unwrap(); +//! # let mut akd = Directory::<_, _, Blake3>::new(&storage_manager, &vrf, false).await.unwrap(); //! # let EpochHash(epoch, root_hash) = akd.publish(entries) //! # .await.expect("Error with publishing"); //! # // Publish new entries into a second epoch diff --git a/akd/src/serialization.rs b/akd/src/serialization.rs index fde85a35..bc092725 100644 --- a/akd/src/serialization.rs +++ b/akd/src/serialization.rs @@ -80,6 +80,7 @@ mod tests { use crate::errors::AkdError; use crate::proof_structs::{AppendOnlyProof, HistoryProof, LookupProof}; use crate::storage::memory::AsyncInMemoryDatabase; + use crate::storage::storage::StorageManager; use crate::storage::types::{AkdLabel, AkdValue}; use crate::Blake3; @@ -109,7 +110,8 @@ mod tests { #[tokio::test] pub async fn lookup_proof_roundtrip() -> Result<(), AkdError> { - let db = AsyncInMemoryDatabase::new(); + let database = AsyncInMemoryDatabase::new(); + let db = StorageManager::new_no_cache(&database); let vrf = HardCodedAkdVRF {}; let akd = Directory::<_, _, Blake3>::new(&db, &vrf, false) @@ -140,7 +142,8 @@ mod tests { #[tokio::test] pub async fn history_proof_roundtrip() -> Result<(), AkdError> { - let db = AsyncInMemoryDatabase::new(); + let database = AsyncInMemoryDatabase::new(); + let db = StorageManager::new_no_cache(&database); let vrf = HardCodedAkdVRF {}; let akd = Directory::<_, _, Blake3>::new(&db, &vrf, false) .await @@ -173,7 +176,8 @@ mod tests { #[tokio::test] pub async fn audit_proof_roundtrip() -> Result<(), AkdError> { - let db = AsyncInMemoryDatabase::new(); + let database = AsyncInMemoryDatabase::new(); + let db = StorageManager::new_no_cache(&database); let vrf = HardCodedAkdVRF {}; let akd = Directory::<_, _, Blake3>::new(&db, &vrf, false) .await diff --git a/akd/src/storage/caches/basic.rs b/akd/src/storage/caches/basic.rs index 4db7a931..5e43c057 100644 --- a/akd/src/storage/caches/basic.rs +++ b/akd/src/storage/caches/basic.rs @@ -15,9 +15,14 @@ use crate::storage::DbRecord; #[cfg(feature = "memory_pressure")] use crate::storage::SizeOf; use crate::storage::Storable; -use log::{debug, error, info, trace, warn}; +#[cfg(not(feature = "runtime_metrics"))] +use log::debug; +#[cfg(all(feature = "memory_pressure", not(feature = "runtime_metrics")))] +use log::info; +#[cfg(feature = "runtime_metrics")] +use log::{debug, error, info, warn}; use std::collections::HashMap; -use std::sync::atomic::{AtomicBool, Ordering}; +use std::sync::atomic::{AtomicBool, AtomicU64, Ordering}; use std::sync::Arc; use std::time::Duration; use std::time::Instant; @@ -30,28 +35,32 @@ pub struct TimedCache { last_clean: Arc>, can_clean: Arc, item_lifetime: Duration, - hit_count: Arc>, + hit_count: Arc, memory_limit_bytes: usize, } impl TimedCache { /// Log cache access metrics along with size information - pub async fn log_metrics(&self, level: log::Level) { - let mut hit = self.hit_count.write().await; - let hit_count = *hit; - *hit = 0; - let guard = self.map.read().await; - let cache_size = (*guard).keys().len(); - let msg = format!( - "Cache hit since last: {}, cached size: {} items", - hit_count, cache_size - ); - match level { - log::Level::Trace => trace!("{}", msg), - log::Level::Debug => debug!("{}", msg), - log::Level::Info => info!("{}", msg), - log::Level::Warn => warn!("{}", msg), - _ => error!("{}", msg), + pub async fn log_metrics(&self, _level: log::Level) { + #[cfg(feature = "runtime_metrics")] + { + let hit_count = self.hit_count.swap(0, Ordering::Relaxed); + + let guard = self.map.read().await; + let cache_size = (*guard).keys().len(); + drop(guard); + + let msg = format!( + "Cache hit since last: {}, cached size: {} items", + hit_count, cache_size + ); + match level { + log::Level::Trace => trace!("{}", msg), + log::Level::Debug => debug!("{}", msg), + log::Level::Info => info!("{}", msg), + log::Level::Warn => warn!("{}", msg), + _ => error!("{}", msg), + } } } } @@ -158,7 +167,7 @@ impl TimedCache { last_clean: Arc::new(tokio::sync::RwLock::new(Instant::now())), can_clean: Arc::new(AtomicBool::new(true)), item_lifetime: lifetime, - hit_count: Arc::new(tokio::sync::RwLock::new(0)), + hit_count: Arc::new(AtomicU64::new(0)), memory_limit_bytes, } } @@ -180,8 +189,9 @@ impl TimedCache { // someone's requesting the AZKS object, return it from the special "cache" storage let record = self.azks.read().await.clone(); debug!("END cache retrieve"); + #[cfg(feature = "runtime_metrics")] if record.is_some() { - *(self.hit_count.write().await) += 1; + self.hit_count.fetch_add(1, Ordering::Relaxed); } // AZKS objects cannot expire, they need to be manually flushed, so we don't need // to check the expiration as below @@ -192,7 +202,9 @@ impl TimedCache { let ptr: &HashMap<_, _> = &*guard; debug!("END cache retrieve"); if let Some(result) = ptr.get(&full_key) { - *(self.hit_count.write().await) += 1; + #[cfg(feature = "runtime_metrics")] + self.hit_count.fetch_add(1, Ordering::Relaxed); + let ignore_clean = !self.can_clean.load(Ordering::Relaxed); // if we've disabled cache cleaning, we're in the middle // of an in-memory transaction and should ignore expiration diff --git a/akd/src/storage/caches/high_parallelism.rs b/akd/src/storage/caches/high_parallelism.rs index 07e89101..5bed6ff2 100644 --- a/akd/src/storage/caches/high_parallelism.rs +++ b/akd/src/storage/caches/high_parallelism.rs @@ -16,10 +16,14 @@ use crate::storage::DbRecord; use crate::storage::SizeOf; use crate::storage::Storable; use dashmap::DashMap; +#[cfg(not(feature = "runtime_metrics"))] use log::debug; -#[cfg(feature = "memory_pressure")] +#[cfg(all(feature = "memory_pressure", not(feature = "runtime_metrics")))] use log::info; -use std::sync::atomic::{AtomicBool, Ordering}; +#[cfg(feature = "runtime_metrics")] +use log::{debug, error, info, warn}; + +use std::sync::atomic::{AtomicBool, AtomicU64, Ordering}; use std::sync::Arc; use std::time::{Duration, Instant}; @@ -32,12 +36,29 @@ pub struct TimedCache { can_clean: Arc, item_lifetime: Duration, memory_limit_bytes: usize, + hit_count: Arc, } impl TimedCache { /// Log cache access metrics along with size information pub async fn log_metrics(&self, _level: log::Level) { - // in high-parallelism, we don't keep any metric counters to minimize thread locking + #[cfg(feature = "runtime_metrics")] + { + let hit_count = self.hit_count.swap(0, Ordering::Relaxed); + let cache_size = self.map.len(); + + let msg = format!( + "Cache hit since last: {}, cached size: {} items", + hit_count, cache_size + ); + match _level { + log::Level::Trace => println!("{}", msg), + log::Level::Debug => debug!("{}", msg), + log::Level::Info => info!("{}", msg), + log::Level::Warn => warn!("{}", msg), + _ => error!("{}", msg), + } + } } } @@ -50,6 +71,7 @@ impl Clone for TimedCache { can_clean: self.can_clean.clone(), item_lifetime: self.item_lifetime, memory_limit_bytes: self.memory_limit_bytes, + hit_count: self.hit_count.clone(), } } } @@ -149,6 +171,7 @@ impl TimedCache { can_clean: Arc::new(AtomicBool::new(true)), item_lifetime: lifetime, memory_limit_bytes, + hit_count: Arc::new(AtomicU64::new(0u64)), } } @@ -171,12 +194,18 @@ impl TimedCache { debug!("END cache retrieve"); + #[cfg(feature = "runtime_metrics")] + self.hit_count.fetch_add(1, Ordering::Relaxed); + // AZKS objects cannot expire, they need to be manually flushed, so we don't need // to check the expiration as below return record; } if let Some(result) = self.map.get(&full_key) { + #[cfg(feature = "runtime_metrics")] + self.hit_count.fetch_add(1, Ordering::Relaxed); + let ignore_clean = !self.can_clean.load(Ordering::Relaxed); // if we've disabled cache cleaning, we're in the middle // of an in-memory transaction and should ignore expiration @@ -208,6 +237,7 @@ impl TimedCache { }; self.map.insert(key, item); } + debug!("END cache put"); } diff --git a/akd/src/storage/memory.rs b/akd/src/storage/memory.rs index e24b70a1..c0886af8 100644 --- a/akd/src/storage/memory.rs +++ b/akd/src/storage/memory.rs @@ -10,14 +10,13 @@ //! benchmarking use crate::errors::StorageError; -use crate::storage::transaction::Transaction; use crate::storage::types::{ AkdLabel, AkdValue, DbRecord, KeyData, StorageType, ValueState, ValueStateKey, ValueStateRetrievalFlag, }; -use crate::storage::{Storable, Storage, StorageUtil}; +use crate::storage::{Database, Storable, StorageUtil}; use async_trait::async_trait; -use log::{debug, error, info, trace, warn}; +use log::debug; use std::collections::HashMap; use std::sync::Arc; @@ -32,7 +31,6 @@ type UserStates = HashMap, UserValueMap>; pub struct AsyncInMemoryDatabase { db: Arc, DbRecord>>>, user_info: Arc>, - trans: Transaction, } unsafe impl Send for AsyncInMemoryDatabase {} @@ -44,7 +42,6 @@ impl AsyncInMemoryDatabase { Self { db: Arc::new(tokio::sync::RwLock::new(HashMap::new())), user_info: Arc::new(tokio::sync::RwLock::new(HashMap::new())), - trans: Transaction::new(), } } } @@ -60,58 +57,13 @@ impl Clone for AsyncInMemoryDatabase { Self { db: self.db.clone(), user_info: self.user_info.clone(), - trans: Transaction::new(), } } } #[async_trait] -impl Storage for AsyncInMemoryDatabase { - async fn log_metrics(&self, level: log::Level) { - let size = self.db.read().await; - let msg = format!("InMemDb record count: {}", size.keys().len()); - - match level { - log::Level::Trace => trace!("{}", msg), - log::Level::Debug => debug!("{}", msg), - log::Level::Info => info!("{}", msg), - log::Level::Warn => warn!("{}", msg), - _ => error!("{}", msg), - } - } - - async fn begin_transaction(&self) -> bool { - self.trans.begin_transaction().await - } - - async fn commit_transaction(&self) -> Result<(), StorageError> { - // this retrieves all the trans operations, and "de-activates" the transaction flag - let ops = self.trans.commit_transaction().await?; - - let _epoch = match ops.last() { - Some(DbRecord::Azks(azks)) => Ok(azks.latest_epoch), - other => Err(StorageError::Transaction(format!( - "The last record in the transaction log is NOT an Azks record {:?}", - other - ))), - }?; - self.batch_set(ops).await - } - - async fn rollback_transaction(&self) -> Result<(), StorageError> { - self.trans.rollback_transaction().await - } - - async fn is_transaction_active(&self) -> bool { - self.trans.is_transaction_active().await - } - +impl Database for AsyncInMemoryDatabase { async fn set(&self, record: DbRecord) -> Result<(), StorageError> { - if self.is_transaction_active().await { - self.trans.set(&record).await; - return Ok(()); - } - if let DbRecord::ValueState(value_state) = &record { let mut u_guard = self.user_info.write().await; let username = value_state.username.to_vec(); @@ -140,14 +92,6 @@ impl Storage for AsyncInMemoryDatabase { // nothing to do, save the cycles return Ok(()); } - - if self.is_transaction_active().await { - for record in records { - self.trans.set(&record).await; - } - return Ok(()); - } - let mut u_guard = self.user_info.write().await; let mut guard = self.db.write().await; @@ -175,20 +119,6 @@ impl Storage for AsyncInMemoryDatabase { /// Retrieve a stored record from the data layer async fn get(&self, id: &St::StorageKey) -> Result { - if self.is_transaction_active().await { - if let Some(result) = self.trans.get::(id).await { - // there's a transacted item, return that one since it's "more up to date" - return Ok(result); - } - } - self.get_direct::(id).await - } - - /// Retrieve a record from the data layer, ignoring any caching or transaction pending - async fn get_direct( - &self, - id: &St::StorageKey, - ) -> Result { let bin_id = St::get_full_binary_key_id(id); // if the request is for a value state, look in the value state set if St::data_type() == StorageType::ValueState { @@ -215,11 +145,6 @@ impl Storage for AsyncInMemoryDatabase { } } - /// Flush the caching of objects (if present) - async fn flush_cache(&self) { - // no-op - } - /// Retrieve a batch of records by id async fn batch_get( &self, @@ -395,458 +320,3 @@ impl StorageUtil for AsyncInMemoryDatabase { Ok(records) } } - -// ===== In-Memory database w/caching (for benchmarking) ==== // - -/// Represents an in-memory database with caching and metric calculation for benchmarking -#[derive(Debug)] -#[cfg(feature = "public-tests")] -pub struct AsyncInMemoryDbWithCache { - db: Arc, DbRecord>>>, - cache: Arc, DbRecord>>>, - stats: Arc>>, - - user_info: Arc>, - trans: Transaction, -} - -#[cfg(feature = "public-tests")] -unsafe impl Send for AsyncInMemoryDbWithCache {} -#[cfg(feature = "public-tests")] -unsafe impl Sync for AsyncInMemoryDbWithCache {} - -#[cfg(feature = "public-tests")] -impl Default for AsyncInMemoryDbWithCache { - fn default() -> Self { - Self::new() - } -} -#[cfg(feature = "public-tests")] -impl Clone for AsyncInMemoryDbWithCache { - fn clone(&self) -> Self { - Self { - db: self.db.clone(), - cache: self.cache.clone(), - stats: self.stats.clone(), - - user_info: self.user_info.clone(), - trans: Transaction::new(), - } - } -} -#[cfg(feature = "public-tests")] -impl AsyncInMemoryDbWithCache { - /// Creates a new in memory db with caching - pub fn new() -> Self { - Self { - db: Arc::new(tokio::sync::RwLock::new(HashMap::new())), - cache: Arc::new(tokio::sync::RwLock::new(HashMap::new())), - stats: Arc::new(tokio::sync::RwLock::new(HashMap::new())), - - user_info: Arc::new(tokio::sync::RwLock::new(HashMap::new())), - trans: Transaction::new(), - } - } - - /// Flushes the cache and clearn any associated stats - pub async fn clear_stats(&self) { - // Flush cache to db - - let mut cache = self.cache.write().await; - - let mut db = self.db.write().await; - for (key, val) in cache.iter() { - db.insert(key.clone(), val.clone()); - } - - cache.clear(); - - let mut stats = self.stats.write().await; - stats.clear(); - } - - /// Prints db states - pub async fn print_stats(&self) { - println!("Statistics collected:"); - println!("---------------------"); - - let stats = self.stats.read().await; - for (key, val) in stats.iter() { - println!("{:?}: {}", key, val); - } - - println!("---------------------"); - } - - /// Prints the distribution of the lengths of entries in a db - pub async fn print_hashmap_distribution(&self) { - println!("Cache distribution of length of entries (in bytes):"); - println!("---------------------"); - - let cache = self.cache.read().await; - - let mut distribution: HashMap = HashMap::new(); - - for (_, val) in cache.iter() { - if let Ok(len) = bincode::serialize(val).map(|item| item.len()) { - let counter = distribution.entry(len).or_insert(0); - *counter += 1; - } - } - - let mut sorted_keys: Vec = distribution.keys().cloned().collect(); - sorted_keys.sort_unstable(); - - for key in sorted_keys { - println!("{}: {}", key, distribution[&key]); - } - println!("---------------------"); - println!("Cache number of elements: {}", cache.len()); - println!("---------------------"); - } -} -#[cfg(feature = "public-tests")] -#[async_trait] -impl Storage for AsyncInMemoryDbWithCache { - async fn log_metrics(&self, level: log::Level) { - let size = self.db.read().await; - let cache_size = self.cache.read().await; - let msg = format!( - "InMemDbWCache record count: {}, cache count: {}", - size.keys().len(), - cache_size.keys().len() - ); - - match level { - log::Level::Trace => trace!("{}", msg), - log::Level::Debug => debug!("{}", msg), - log::Level::Info => info!("{}", msg), - log::Level::Warn => warn!("{}", msg), - _ => error!("{}", msg), - } - } - - async fn begin_transaction(&self) -> bool { - self.trans.begin_transaction().await - } - - async fn commit_transaction(&self) -> Result<(), StorageError> { - // this retrieves all the trans operations, and "de-activates" the transaction flag - let ops = self.trans.commit_transaction().await?; - let _epoch = match ops.last() { - Some(DbRecord::Azks(azks)) => Ok(azks.latest_epoch), - other => Err(StorageError::Transaction(format!( - "The last record in the transaction log is NOT an Azks record {:?}", - other - ))), - }?; - self.batch_set(ops).await - } - - async fn rollback_transaction(&self) -> Result<(), StorageError> { - self.trans.rollback_transaction().await - } - - async fn is_transaction_active(&self) -> bool { - self.trans.is_transaction_active().await - } - - async fn set(&self, record: DbRecord) -> Result<(), StorageError> { - if self.is_transaction_active().await { - self.trans.set(&record).await; - return Ok(()); - } - - if let DbRecord::ValueState(value_state) = &record { - let mut u_guard = self.user_info.write().await; - let username = value_state.username.to_vec(); - match u_guard.get(&username) { - Some(old_states) => { - let mut new_states = old_states.clone(); - new_states.insert(value_state.epoch, value_state.clone()); - u_guard.insert(username, new_states); - } - None => { - let mut new_map = HashMap::new(); - new_map.insert(value_state.epoch, value_state.clone()); - u_guard.insert(username, new_map); - } - } - } else { - let mut stats = self.stats.write().await; - let calls = stats.entry(String::from("calls_to_cache_set")).or_insert(0); - *calls += 1; - - let mut guard = self.cache.write().await; - guard.insert(record.get_full_binary_id(), record); - } - - Ok(()) - } - - async fn batch_set(&self, records: Vec) -> Result<(), StorageError> { - if records.is_empty() { - // nothing to do, save the cycles - return Ok(()); - } - - if self.is_transaction_active().await { - for record in records { - self.trans.set(&record).await; - } - return Ok(()); - } - - let mut u_guard = self.user_info.write().await; - let mut stats = self.stats.write().await; - let mut guard = self.cache.write().await; - let calls = stats.entry(String::from("calls_to_cache_set")).or_insert(0); - - for record in records.into_iter() { - if let DbRecord::ValueState(value_state) = &record { - let username = value_state.username.to_vec(); - match u_guard.get(&username) { - Some(old_states) => { - let mut new_states = old_states.clone(); - new_states.insert(value_state.epoch, value_state.clone()); - u_guard.insert(username, new_states); - } - None => { - let mut new_map = HashMap::new(); - new_map.insert(value_state.epoch, value_state.clone()); - u_guard.insert(username, new_map); - } - } - } else { - *calls += 1; - guard.insert(record.get_full_binary_id(), record); - } - } - Ok(()) - } - - async fn get(&self, id: &St::StorageKey) -> Result { - if self.is_transaction_active().await { - if let Some(result) = self.trans.get::(id).await { - // there's a transacted item, return that one since it's "more up to date" - return Ok(result); - } - } - self.get_direct::(id).await - } - - /// Retrieve a record from the data layer, ignoring any caching or transaction pending - async fn get_direct( - &self, - id: &St::StorageKey, - ) -> Result { - let bin_id = St::get_full_binary_key_id(id); - // if the request is for a value state, look in the value state set - if St::data_type() == StorageType::ValueState { - if let Ok(ValueStateKey(username, epoch)) = ValueState::key_from_full_binary(&bin_id) { - let u_guard = self.user_info.read().await; - if let Some(state) = (*u_guard).get(&username).cloned() { - if let Some(found) = state.get(&epoch) { - return Ok(DbRecord::ValueState(found.clone())); - } - } - return Err(StorageError::NotFound(format!("ValueState {:?}", id))); - } - } - let mut stats = self.stats.write().await; - let calls_to_cache_get = stats.entry(String::from("calls_to_cache_get")).or_insert(0); - *calls_to_cache_get += 1; - - let mut cache = self.cache.write().await; - match cache.get(&bin_id).cloned() { - Some(value) => Ok(value), - None => { - // fallback to regular get/set db - let guard = self.db.read().await; - if let Some(result) = (*guard).get(&bin_id).cloned() { - // cache the item - cache.insert(bin_id, result.clone()); - - Ok(result) - } else { - Err(StorageError::NotFound(format!( - "{:?} {:?}", - St::data_type(), - id - ))) - } - } - } - } - - async fn flush_cache(&self) { - // no-op - } - - async fn batch_get( - &self, - ids: &[St::StorageKey], - ) -> Result, StorageError> { - let mut map = Vec::new(); - for key in ids.iter() { - if let Ok(result) = self.get::(key).await { - map.push(result); - } - // swallow errors (i.e. not found) - } - Ok(map) - } - - async fn tombstone_value_states(&self, keys: &[ValueStateKey]) -> Result<(), StorageError> { - if keys.is_empty() { - return Ok(()); - } - - let data = self.batch_get::(keys).await?; - let mut new_data = vec![]; - for record in data { - if let DbRecord::ValueState(value_state) = record { - new_data.push(DbRecord::ValueState(ValueState { - plaintext_val: crate::AkdValue(crate::TOMBSTONE.to_vec()), - ..value_state - })); - } - } - - if !new_data.is_empty() { - self.batch_set(new_data).await?; - } - - Ok(()) - } - - async fn get_user_data(&self, username: &AkdLabel) -> Result { - let guard = self.user_info.read().await; - if let Some(result) = guard.get(&username.0) { - let mut results: Vec = result.values().cloned().collect(); - // return ordered by epoch (from smallest -> largest) - results.sort_by(|a, b| a.epoch.cmp(&b.epoch)); - - Ok(KeyData { states: results }) - } else { - Err(StorageError::NotFound(format!("ValueState {:?}", username))) - } - } - - async fn get_user_state( - &self, - username: &AkdLabel, - flag: ValueStateRetrievalFlag, - ) -> Result { - let intermediate = self.get_user_data(username).await?.states; - match flag { - ValueStateRetrievalFlag::MaxEpoch => - // retrieve by max epoch - { - if let Some(value) = intermediate.iter().max_by(|a, b| a.epoch.cmp(&b.epoch)) { - return Ok(value.clone()); - } - } - ValueStateRetrievalFlag::MinEpoch => - // retrieve by min epoch - { - if let Some(value) = intermediate.iter().min_by(|a, b| a.epoch.cmp(&b.epoch)) { - return Ok(value.clone()); - } - } - _ => - // search for specific property - { - let mut tracked_epoch = 0u64; - let mut tracker = None; - for kvp in intermediate.iter() { - match flag { - ValueStateRetrievalFlag::SpecificVersion(version) - if version == kvp.version => - { - return Ok(kvp.clone()) - } - ValueStateRetrievalFlag::LeqEpoch(epoch) if epoch == kvp.epoch => { - return Ok(kvp.clone()); - } - ValueStateRetrievalFlag::LeqEpoch(epoch) if kvp.epoch < epoch => { - match tracked_epoch { - 0u64 => { - tracked_epoch = kvp.epoch; - tracker = Some(kvp.clone()); - } - other_epoch => { - if kvp.epoch > other_epoch { - tracker = Some(kvp.clone()); - tracked_epoch = kvp.epoch; - } - } - } - } - ValueStateRetrievalFlag::SpecificEpoch(epoch) if epoch == kvp.epoch => { - return Ok(kvp.clone()) - } - _ => continue, - } - } - - if let Some(r) = tracker { - return Ok(r); - } - } - } - Err(StorageError::NotFound(format!("ValueState {:?}", username))) - } - - async fn get_user_state_versions( - &self, - keys: &[AkdLabel], - flag: ValueStateRetrievalFlag, - ) -> Result, StorageError> { - let mut map = HashMap::new(); - for username in keys.iter() { - if let Ok(result) = self.get_user_state(username, flag).await { - map.insert( - AkdLabel(result.username.to_vec()), - (result.version, AkdValue(result.plaintext_val.to_vec())), - ); - } - } - Ok(map) - } -} - -#[cfg(feature = "public-tests")] -#[async_trait] -impl StorageUtil for AsyncInMemoryDbWithCache { - async fn batch_get_type_direct(&self) -> Result, StorageError> { - let records = self - .batch_get_all_direct() - .await? - .into_iter() - .filter(|record| match record { - DbRecord::Azks(_) => St::data_type() == StorageType::Azks, - DbRecord::TreeNode(_) => St::data_type() == StorageType::TreeNode, - DbRecord::ValueState(_) => St::data_type() == StorageType::ValueState, - }) - .collect(); - - Ok(records) - } - - async fn batch_get_all_direct(&self) -> Result, StorageError> { - // get value states - let u_guard = self.user_info.read().await; - let u_records = u_guard - .values() - .cloned() - .flat_map(|v| v.into_values()) - .map(DbRecord::ValueState); - - // get other records and collect - let guard = self.db.read().await; - let records = guard.values().cloned().chain(u_records).collect(); - - Ok(records) - } -} diff --git a/akd/src/storage/mod.rs b/akd/src/storage/mod.rs index 59c95804..7128895d 100644 --- a/akd/src/storage/mod.rs +++ b/akd/src/storage/mod.rs @@ -25,6 +25,9 @@ pub mod types; Various implementations supported by the library are imported here and usable at various checkpoints */ pub mod memory; +pub mod storage; + +pub use storage::StorageManager; #[cfg(any(test, feature = "public-tests"))] pub mod tests; @@ -85,39 +88,23 @@ pub trait Storable: Clone + Sync { fn key_from_full_binary(bin: &[u8]) -> Result; } -/// Storage layer with support for asynchronous work and batched operations +/// A database implementation backing storage for the AKD #[async_trait] -pub trait Storage: Clone { - /// Log some information about the cache (hit rate, etc) - async fn log_metrics(&self, level: log::Level); - - /// Start a transaction in the storage layer - async fn begin_transaction(&self) -> bool; - - /// Commit a transaction in the storage layer - async fn commit_transaction(&self) -> Result<(), StorageError>; - - /// Rollback a transaction - async fn rollback_transaction(&self) -> Result<(), StorageError>; - - /// Retrieve a flag determining if there is a transaction active - async fn is_transaction_active(&self) -> bool; - - /// Set a record in the data layer +pub trait Database: Clone { + /// Set a record in the database async fn set(&self, record: DbRecord) -> Result<(), StorageError>; - /// Set multiple records in transactional operation + /// Set multiple records in the database with a minimal set of operations async fn batch_set(&self, records: Vec) -> Result<(), StorageError>; - /// Retrieve a stored record from the data layer + /// Retrieve a stored record from the database async fn get(&self, id: &St::StorageKey) -> Result; - /// Retrieve a record from the data layer, ignoring any caching or transaction pending - async fn get_direct(&self, id: &St::StorageKey) - -> Result; - - /// Flush the caching of objects (if present) - async fn flush_cache(&self); + /// Retrieve a batch of records by id from the database + async fn batch_get( + &self, + ids: &[St::StorageKey], + ) -> Result, StorageError>; /// Convert the given value state's into tombstones, replacing the plaintext value with /// the tombstone key array @@ -126,12 +113,6 @@ pub trait Storage: Clone { keys: &[types::ValueStateKey], ) -> Result<(), StorageError>; - /// Retrieve a batch of records by id - async fn batch_get( - &self, - ids: &[St::StorageKey], - ) -> Result, StorageError>; - /* User data searching */ /// Retrieve the user data for a given user @@ -157,7 +138,7 @@ pub trait Storage: Clone { /// Optional storage layer utility functions for debug and test purposes #[async_trait] -pub trait StorageUtil: Storage { +pub trait StorageUtil: Database { /// Retrieves all stored records of a given type from the data layer, ignoring any caching or transaction pending async fn batch_get_type_direct(&self) -> Result, StorageError>; diff --git a/akd/src/storage/storage.rs b/akd/src/storage/storage.rs new file mode 100644 index 00000000..c54a7226 --- /dev/null +++ b/akd/src/storage/storage.rs @@ -0,0 +1,573 @@ +// Copyright (c) Meta Platforms, Inc. and affiliates. +// +// This source code is licensed under both the MIT license found in the +// LICENSE-MIT file in the root directory of this source tree and the Apache +// License, Version 2.0 found in the LICENSE-APACHE file in the root directory +// of this source tree. + +//! Storage management module for AKD. A wrapper around the underlying database interaction +//! to manage interactions with the data layer to optimize things like caching and +//! transaction management + +use crate::storage::caches::TimedCache; +use crate::storage::transaction::Transaction; +use crate::storage::types::DbRecord; +use crate::storage::types::KeyData; +use crate::storage::types::ValueState; +use crate::storage::types::ValueStateKey; +use crate::storage::Database; +use crate::storage::Storable; +use crate::storage::StorageError; +use crate::AkdLabel; +use crate::AkdValue; + +use log::{debug, error, info, warn}; +use std::collections::HashMap; +use std::collections::HashSet; +use std::sync::Arc; +use std::time::Duration; + +use super::types::ValueStateRetrievalFlag; + +type Metric = usize; + +const METRIC_GET: Metric = 0; +const METRIC_BATCH_GET: Metric = 1; +const METRIC_SET: Metric = 2; +const METRIC_BATCH_SET: Metric = 3; +const METRIC_READ_TIME: Metric = 4; +const METRIC_WRITE_TIME: Metric = 5; +const METRIC_TOMBSTONE: Metric = 6; +const METRIC_GET_USER_STATE: Metric = 7; +const METRIC_GET_USER_DATA: Metric = 8; +const METRIC_GET_USER_STATE_VERSIONS: Metric = 9; + +const NUM_METRICS: usize = 10; + +/// Represents the manager of the storage mediums, including caching +/// and transactional operations (creating the transaction, commiting it, etc) +pub struct StorageManager { + cache: Option, + transaction: Transaction, + db: Db, + + metrics: Arc>, +} + +impl Clone for StorageManager { + fn clone(&self) -> Self { + Self { + cache: self.cache.clone(), + transaction: Transaction::new(), + db: self.db.clone(), + metrics: self.metrics.clone(), + } + } +} + +unsafe impl Sync for StorageManager {} +unsafe impl Send for StorageManager {} + +impl StorageManager { + /// Create a new storage manager with NO CACHE + pub fn new_no_cache(db: &Db) -> Self { + Self { + cache: None, + transaction: Transaction::new(), + db: db.clone(), + metrics: Arc::new(tokio::sync::RwLock::new([0; NUM_METRICS])), + } + } + + /// Create a new storage manager with a cache utilizing the options provided (or defaults) + pub fn new( + db: &Db, + cache_item_lifetime: Option, + cache_limit_bytes: Option, + ) -> Self { + Self { + cache: Some(TimedCache::new(cache_item_lifetime, cache_limit_bytes)), + transaction: Transaction::new(), + db: db.clone(), + metrics: Arc::new(tokio::sync::RwLock::new([0; NUM_METRICS])), + } + } + + /// Log metrics from the storage manager (cache, transaction, and storage hit rates etc) + pub async fn log_metrics(&self, level: log::Level) { + if let Some(cache) = &self.cache { + cache.log_metrics(level).await + } + + self.transaction.log_metrics(level).await; + + let mut metric_guard = self.metrics.write().await; + let snapshot = metric_guard.clone(); + // clear the metrics + *metric_guard = [0; NUM_METRICS]; + // free the metric's guard to not block metric collection processing in parallel + drop(metric_guard); + + let msg = format!( + "=================================================== +============ Database operation counts ============ +=================================================== + SET {}, + BATCH SET {}, + GET {}, + BATCH GET {} + TOMBSTONE {} + GET USER STATE {} + GET USER DATA {} + GET USER STATE VERSIONS {} +=================================================== +============ Database operation timing ============ +=================================================== + TIME READ {} ms + TIME WRITE {} ms", + snapshot[METRIC_SET], + snapshot[METRIC_BATCH_SET], + snapshot[METRIC_GET], + snapshot[METRIC_BATCH_GET], + snapshot[METRIC_TOMBSTONE], + snapshot[METRIC_GET_USER_STATE], + snapshot[METRIC_GET_USER_DATA], + snapshot[METRIC_GET_USER_STATE_VERSIONS], + snapshot[METRIC_READ_TIME], + snapshot[METRIC_WRITE_TIME] + ); + + match level { + // Currently logs cannot be captured unless they are + // println!. Normally Level::Trace should use the trace! macro. + log::Level::Trace => println!("{}", msg), + log::Level::Debug => debug!("{}", msg), + log::Level::Info => info!("{}", msg), + log::Level::Warn => warn!("{}", msg), + _ => error!("{}", msg), + } + } + + /// Start an in-memory transaction of changes + pub async fn begin_transaction(&self) -> bool { + // disable the cache cleaning since we're in a write transaction + // and will want to keep cache'd objects for the life of the transaction + if let Some(cache) = &self.cache { + cache.disable_clean(); + } + + self.transaction.begin_transaction().await + } + + /// Commit a transaction in the database + pub async fn commit_transaction(&self) -> Result<(), StorageError> { + // The transaction is now complete (or reverted) and therefore we can re-enable + // the cache cleaning status + if let Some(cache) = &self.cache { + cache.enable_clean(); + } + + // this retrieves all the trans operations, and "de-activates" the transaction flag + let ops = self.transaction.commit_transaction().await?; + + let _epoch = match ops.last() { + Some(DbRecord::Azks(azks)) => Ok(azks.latest_epoch), + other => Err(StorageError::Transaction(format!( + "The last record in the transaction log is NOT an Azks record {:?}", + other + ))), + }?; + + self.tic_toc(METRIC_WRITE_TIME, self.db.batch_set(ops)) + .await?; + self.increment_metric(METRIC_BATCH_SET).await; + Ok(()) + } + + /// Rollback a transaction + pub async fn rollback_transaction(&self) -> Result<(), StorageError> { + // The transaction is being reverted and therefore we can re-enable + // the cache cleaning status + if let Some(cache) = &self.cache { + cache.enable_clean(); + } + + self.transaction.rollback_transaction().await + } + + /// Retrieve a flag determining if there is a transaction active + pub async fn is_transaction_active(&self) -> bool { + self.transaction.is_transaction_active().await + } + + /// Store a record in the database + pub async fn set(&self, record: DbRecord) -> Result<(), StorageError> { + // we're in a transaction, set the item in the transaction + if self.is_transaction_active().await { + self.transaction.set(&record).await; + return Ok(()); + } + + // update the cache + if let Some(cache) = &self.cache { + cache.put(&record).await; + } + + // write to the database + self.tic_toc(METRIC_WRITE_TIME, self.db.set(record)).await?; + self.increment_metric(METRIC_SET).await; + Ok(()) + } + + /// Set a batch of records in the database + pub async fn batch_set(&self, records: Vec) -> Result<(), StorageError> { + if records.is_empty() { + // nothing to do, save the cycles + return Ok(()); + } + + // we're in a transaction, set the items in the transaction + if self.is_transaction_active().await { + for record in records.iter() { + self.transaction.set(record).await; + } + return Ok(()); + } + + // update the cache + if let Some(cache) = &self.cache { + let _ = cache.batch_put(&records).await; + } + + // Write to the database + self.tic_toc(METRIC_WRITE_TIME, self.db.batch_set(records)) + .await?; + self.increment_metric(METRIC_BATCH_SET).await; + Ok(()) + } + + /// Retrieve a stored record directly from the data layer, ignoring any caching or transaction processes + pub async fn get_direct( + &self, + id: &St::StorageKey, + ) -> Result { + // cache miss, read direct from db + let record = self + .tic_toc(METRIC_READ_TIME, self.db.get::(id)) + .await?; + self.increment_metric(METRIC_GET).await; + Ok(record) + } + + /// Retrieve a stored record from the database + pub async fn get(&self, id: &St::StorageKey) -> Result { + // we're in a transaction, meaning the object _might_ be newer and therefore we should try and read if from the transaction + // log instead of the raw storage layer + if self.is_transaction_active().await { + if let Some(result) = self.transaction.get::(id).await { + return Ok(result); + } + } + + // check for a cache hit + if let Some(cache) = &self.cache { + if let Some(result) = cache.hit_test::(id).await { + return Ok(result); + } + } + + // cache miss, read direct from db + let record = self + .tic_toc(METRIC_READ_TIME, self.db.get::(id)) + .await?; + if let Some(cache) = &self.cache { + // cache the result + cache.put(&record).await; + } + self.increment_metric(METRIC_GET).await; + Ok(record) + } + + /// Retrieve a batch of records by id from the database + pub async fn batch_get( + &self, + ids: &[St::StorageKey], + ) -> Result, StorageError> { + let mut map = Vec::new(); + + if ids.is_empty() { + // nothing to retrieve, save the cycles + return Ok(map); + } + + let mut key_set: HashSet = ids.iter().cloned().collect::>(); + + let trans_active = self.is_transaction_active().await; + // first check the transaction log & cache records + for id in ids.iter() { + if trans_active { + // we're in a transaction, meaning the object _might_ be newer and therefore we should try and read if from the transaction + // log instead of the raw storage layer + if let Some(result) = self.transaction.get::(id).await { + map.push(result); + key_set.remove(id); + continue; + } + } + + // check if item is cached + if let Some(cache) = &self.cache { + if let Some(result) = cache.hit_test::(id).await { + map.push(result); + key_set.remove(id); + continue; + } + } + } + + if !key_set.is_empty() { + // these are items to be retrieved from the backing database (not in pending transaction or in the object cache) + let keys = key_set.into_iter().collect::>(); + let mut results = self + .tic_toc(METRIC_READ_TIME, self.db.batch_get::(&keys)) + .await?; + map.append(&mut results); + self.increment_metric(METRIC_BATCH_GET).await; + } + Ok(map) + } + + /// Flush the caching of objects (if present) + pub async fn flush_cache(&self) { + if let Some(cache) = &self.cache { + cache.flush().await; + } + } + + /// Tombstone a set of records adhereing to the caching + transactional + /// settings of the storage manager + pub async fn tombstone_value_states(&self, keys: &[ValueStateKey]) -> Result<(), StorageError> { + if keys.is_empty() { + return Ok(()); + } + + let data = self.batch_get::(keys).await?; + let mut new_data = vec![]; + for record in data { + if let DbRecord::ValueState(value_state) = record { + new_data.push(DbRecord::ValueState(ValueState { + epoch: value_state.epoch, + label: value_state.label, + plaintext_val: crate::AkdValue(crate::TOMBSTONE.to_vec()), + username: value_state.username, + version: value_state.version, + })); + } + } + if !new_data.is_empty() { + debug!("Tombstoning {} entries", new_data.len()); + self.tic_toc(METRIC_WRITE_TIME, self.batch_set(new_data)) + .await?; + self.increment_metric(METRIC_TOMBSTONE).await; + } + + Ok(()) + } + + /// Retrieve the specified user state object based on the retrieval flag from the database + pub async fn get_user_state( + &self, + username: &AkdLabel, + flag: ValueStateRetrievalFlag, + ) -> Result { + let maybe_db_state = match self + .tic_toc(METRIC_READ_TIME, self.db.get_user_state(username, flag)) + .await + { + Err(StorageError::NotFound(_)) => Ok(None), + Ok(something) => Ok(Some(something)), + Err(other) => Err(other), + }?; + self.increment_metric(METRIC_GET_USER_STATE).await; + + // in the event we are in a transaction, there may be an updated object in the + // transactional storage. Therefore we should update the db retrieved value if + // we can with what's in the transaction log + if self.is_transaction_active().await { + if let Some(transaction_value) = self.transaction.get_user_state(username, flag).await { + if let Some(db_value) = &maybe_db_state { + if let Some(record) = Self::compare_db_and_transaction_records( + db_value.epoch, + transaction_value, + flag, + ) { + return Ok(record); + } + } else { + // no db record, but there is a transaction record so use that + return Ok(transaction_value); + } + } + } + + if let Some(state) = maybe_db_state { + // cache the item for future access + if let Some(cache) = &self.cache { + cache.put(&DbRecord::ValueState(state.clone())).await; + } + + Ok(state) + } else { + Err(StorageError::NotFound(format!("ValueState {:?}", username))) + } + } + + /// Retrieve all values states for a given user + pub async fn get_user_data(&self, username: &AkdLabel) -> Result { + let maybe_db_data = match self + .tic_toc(METRIC_READ_TIME, self.db.get_user_data(username)) + .await + { + Err(StorageError::NotFound(_)) => Ok(None), + Ok(something) => Ok(Some(something)), + Err(other) => Err(other), + }?; + self.increment_metric(METRIC_GET_USER_DATA).await; + + if self.is_transaction_active().await { + // there are transaction-based values in the current transaction, they should override database-retrieved values + let mut map = maybe_db_data + .map(|data| { + data.states + .into_iter() + .map(|state| (state.epoch, state)) + .collect::>() + }) + .unwrap_or_else(|| HashMap::new()); + + let transaction_records = self + .transaction + .get_users_data(&[username.clone()]) + .await + .remove(username) + .unwrap_or_else(|| vec![]); + for transaction_record in transaction_records.into_iter() { + map.insert(transaction_record.epoch, transaction_record); + } + + return Ok(KeyData { + states: map.into_iter().map(|(_, v)| v).collect::>(), + }); + } + + if let Some(data) = maybe_db_data { + Ok(data) + } else { + Err(StorageError::NotFound(format!( + "ValueState records for {:?}", + username + ))) + } + } + + /// Retrieve the user -> state version mapping in bulk. This is the same as get_user_state in a loop, but with less data retrieved from the storage layer + pub async fn get_user_state_versions( + &self, + usernames: &[AkdLabel], + flag: ValueStateRetrievalFlag, + ) -> Result, StorageError> { + let mut data = self + .tic_toc( + METRIC_READ_TIME, + self.db.get_user_state_versions(usernames, flag), + ) + .await?; + self.increment_metric(METRIC_GET_USER_STATE_VERSIONS).await; + + // in the event we are in a transaction, there may be an updated object in the + // transactional storage. Therefore we should update the db retrieved value if + // we can with what's in the transaction log + if self.is_transaction_active().await { + let transaction_records = self.transaction.get_users_states(usernames, flag).await; + for (label, value_state) in transaction_records.into_iter() { + if let Some((epoch, _)) = data.get(&label) { + // there is an existing DB record, check if we should updated it from the transaction log + if let Some(updated_record) = + Self::compare_db_and_transaction_records(*epoch, value_state, flag) + { + data.insert(label, (*epoch, updated_record.plaintext_val)); + } + } else { + // there is no db-equivalent record, but there IS a record in the transaction log. + // Take the transaction log value + data.insert(label, (value_state.epoch, value_state.plaintext_val)); + } + } + } + + Ok(data) + } + + fn compare_db_and_transaction_records( + state_epoch: u64, + transaction_value: ValueState, + flag: ValueStateRetrievalFlag, + ) -> Option { + match flag { + ValueStateRetrievalFlag::SpecificVersion(_) => { + return Some(transaction_value); + } + ValueStateRetrievalFlag::SpecificEpoch(_) => { + return Some(transaction_value); + } + ValueStateRetrievalFlag::LeqEpoch(_) => { + if transaction_value.epoch >= state_epoch { + // the transaction has either the same epoch or an epoch in the future, and therefore should + // override the db value + return Some(transaction_value); + } + } + ValueStateRetrievalFlag::MaxEpoch => { + if transaction_value.epoch >= state_epoch { + // the transaction has either the same epoch or an epoch in the future, and therefore should + // override the db value + return Some(transaction_value); + } + } + ValueStateRetrievalFlag::MinEpoch => { + if transaction_value.epoch <= state_epoch { + // the transaction has either the same epoch or an older epoch, and therefore should + // override the db value + return Some(transaction_value); + } + } + } + None + } + + async fn increment_metric(&self, _metric: Metric) { + #[cfg(feature = "runtime_metrics")] + { + let mut guard = self.metrics.write().await; + (*guard)[_metric] = (*guard)[_metric] + 1; + } + } + + async fn tic_toc(&self, _metric: Metric, f: impl std::future::Future) -> T { + #[cfg(feature = "runtime_metrics")] + { + let tic = std::time::Instant::now(); + let out = f.await; + let delta = std::time::Instant::now().duration_since(tic); + + let mut guard = self.metrics.write().await; + (*guard)[_metric] = (*guard)[_metric] + delta.as_millis(); + + out + } + #[cfg(not(feature = "runtime_metrics"))] + { + f.await + } + } +} diff --git a/akd/src/storage/tests.rs b/akd/src/storage/tests.rs index e73c7165..2dea7c67 100644 --- a/akd/src/storage/tests.rs +++ b/akd/src/storage/tests.rs @@ -10,7 +10,7 @@ use crate::errors::StorageError; use crate::node_label::byte_arr_from_u64; use crate::storage::types::*; -use crate::storage::Storage; +use crate::storage::Database; use crate::tree_node::*; use crate::NodeLabel; use rand::distributions::Alphanumeric; @@ -28,13 +28,6 @@ mod memory_storage_tests { use crate::storage::memory::AsyncInMemoryDatabase; use serial_test::serial; - #[tokio::test] - #[serial] - async fn test_v2_in_memory_db_with_caching() { - let db = crate::storage::memory::AsyncInMemoryDbWithCache::new(); - crate::storage::tests::run_test_cases_for_storage_impl(&db).await; - } - #[tokio::test] #[serial] async fn test_v2_in_memory_db() { @@ -47,7 +40,7 @@ mod memory_storage_tests { /// Run the storage-layer test suite for a given storage implementation. /// This is public because it can be used by other implemented storage layers /// for consistency checks (e.g. mysql, memcached, etc) -pub async fn run_test_cases_for_storage_impl(db: &S) { +pub async fn run_test_cases_for_storage_impl(db: &S) { test_get_and_set_item(db).await; test_user_data(db).await; test_transactions(db).await; @@ -56,7 +49,7 @@ pub async fn run_test_cases_for_storage_impl(db: &S) { } // *** New Test Helper Functions *** // -async fn test_get_and_set_item(storage: &Ns) { +async fn test_get_and_set_item(storage: &Ns) { // === Azks storage === // let azks = Azks { latest_epoch: 34, @@ -146,7 +139,7 @@ async fn test_get_and_set_item(storage: &Ns) { } } -async fn test_batch_get_items(storage: &Ns) { +async fn test_batch_get_items(storage: &Ns) { let mut rand_users: Vec> = vec![]; for _ in 0..20 { let str: String = thread_rng() @@ -309,7 +302,9 @@ async fn test_batch_get_items(storage: &Ns) { } } -async fn test_transactions(storage: &S) { +async fn test_transactions(db: &S) { + let storage = crate::storage::storage::StorageManager::new_no_cache(db); + let mut rand_users: Vec> = vec![]; for _ in 0..20 { let str: String = thread_rng() @@ -389,7 +384,7 @@ async fn test_transactions(storage: &S) { } } -async fn test_user_data(storage: &S) { +async fn test_user_data(storage: &S) { let rand_user = thread_rng() .sample_iter(&Alphanumeric) .take(30) @@ -586,7 +581,7 @@ async fn test_user_data(storage: &S) { assert_eq!(4, data.unwrap().states.len()); } -async fn test_tombstoning_data( +async fn test_tombstoning_data( storage: &S, ) -> Result<(), crate::errors::AkdError> { let rand_user = thread_rng() diff --git a/akd/src/storage/transaction.rs b/akd/src/storage/transaction.rs index 0ca5d632..5e511da2 100644 --- a/akd/src/storage/transaction.rs +++ b/akd/src/storage/transaction.rs @@ -9,6 +9,8 @@ use crate::errors::StorageError; use crate::storage::types::DbRecord; +use crate::storage::types::ValueState; +use crate::storage::types::ValueStateRetrievalFlag; use crate::storage::Storable; use log::{debug, error, info, trace, warn}; @@ -65,6 +67,8 @@ impl Transaction { *r = 0; *w = 0; + drop(r); + drop(w); match level { log::Level::Trace => trace!("{}", msg), @@ -148,6 +152,7 @@ impl Transaction { let guard = self.state.read().await; let out = guard.mods.get(&bin_id).cloned(); + #[cfg(feature = "runtime_metrics")] if out.is_some() { *(self.num_reads.write().await) += 1; } @@ -163,9 +168,120 @@ impl Transaction { let mut guard = self.state.write().await; guard.mods.insert(bin_id, record.clone()); - *(self.num_writes.write().await) += 1; + #[cfg(feature = "runtime_metrics")] + { + *(self.num_writes.write().await) += 1; + } + debug!("END transaction set"); } + + /// Retrieve all of the user data for a given username + /// + /// Note: This is a FULL SCAN operation of the entire transaction log + pub async fn get_users_data( + &self, + usernames: &[crate::AkdLabel], + ) -> HashMap> { + debug!("BEGIN transaction user version scan"); + let mut results = HashMap::new(); + + let mut set = std::collections::HashSet::with_capacity(usernames.len()); + for username in usernames.iter() { + if !set.contains(username) { + set.insert(username.clone()); + } + } + + let guard = self.state.read().await; + for (_key, record) in guard.mods.iter() { + if let DbRecord::ValueState(value_state) = record { + if set.contains(&value_state.username) { + if results.contains_key(&value_state.username) { + results + .get_mut(&value_state.username) + .map(|item: &mut Vec| item.push(value_state.clone())); + } else { + results.insert(value_state.username.clone(), vec![value_state.clone()]); + } + } + } + } + + // sort all the value lists by epoch + for (_k, v) in results.iter_mut() { + v.sort_unstable_by(|a, b| a.epoch.cmp(&b.epoch)); + } + + debug!("END transaction user version scan"); + results + } + + /// Retrieve the user state given the specified value state retrieval mode + /// + /// Note: This is a FULL SCAN operation of the entire transaction log + pub async fn get_user_state( + &self, + username: &crate::AkdLabel, + flag: ValueStateRetrievalFlag, + ) -> Option { + let intermediate = self + .get_users_data(&[username.clone()]) + .await + .remove(username) + .unwrap_or_else(|| vec![]); + let out = Self::find_appropriate_item(intermediate, flag); + #[cfg(feature = "runtime_metrics")] + if out.is_some() { + *(self.num_reads.write().await) += 1; + } + out + } + + /// Retrieve the batch of specified users user_state's based on the filtering flag provided + /// + /// Note: This is a FULL SCAN operation of the entire transaction log + pub async fn get_users_states( + &self, + usernames: &[crate::AkdLabel], + flag: ValueStateRetrievalFlag, + ) -> HashMap { + let mut result_map = HashMap::new(); + let intermediate = self.get_users_data(usernames).await; + + for (key, value_list) in intermediate.into_iter() { + if let Some(found) = Self::find_appropriate_item(value_list, flag) { + result_map.insert(key, found); + } + } + #[cfg(feature = "runtime_metrics")] + { + *(self.num_reads.write().await) += 1; + } + result_map + } + + /// Find the appropriate item of the cached value states for a given user. This assumes that the incoming vector + /// is already sorted in ascending epoch order + fn find_appropriate_item( + intermediate: Vec, + flag: ValueStateRetrievalFlag, + ) -> Option { + match flag { + ValueStateRetrievalFlag::SpecificVersion(version) => intermediate + .into_iter() + .find(|item| item.version == version), + ValueStateRetrievalFlag::SpecificEpoch(epoch) => { + intermediate.into_iter().find(|item| item.epoch == epoch) + } + ValueStateRetrievalFlag::LeqEpoch(epoch) => intermediate + .into_iter() + .rev() + .find(|item| item.epoch <= epoch), + ValueStateRetrievalFlag::MaxEpoch => intermediate.into_iter().last(), + ValueStateRetrievalFlag::MinEpoch => intermediate.into_iter().nth(0), + } + } } #[cfg(test)] diff --git a/akd/src/tests.rs b/akd/src/tests.rs index 15262b78..c41548b5 100644 --- a/akd/src/tests.rs +++ b/akd/src/tests.rs @@ -16,8 +16,9 @@ use crate::{ errors::AkdError, storage::{ memory::AsyncInMemoryDatabase, + storage::StorageManager, types::{AkdLabel, AkdValue, DbRecord}, - Storage, + Database as Storage, }, }; use winter_crypto::{Digest, Hasher}; @@ -29,8 +30,9 @@ type Sha3 = winter_crypto::hashers::Sha3_256; #[tokio::test] async fn test_empty_tree_root_hash() -> Result<(), AkdError> { let db = AsyncInMemoryDatabase::new(); + let storage = StorageManager::new_no_cache(&db); let vrf = HardCodedAkdVRF {}; - let akd = Directory::<_, _, Blake3>::new(&db, &vrf, false).await?; + let akd = Directory::<_, _, Blake3>::new(&storage, &vrf, false).await?; let current_azks = akd.retrieve_current_azks().await?; let hash = akd.get_root_hash(¤t_azks).await?; @@ -46,8 +48,9 @@ async fn test_empty_tree_root_hash() -> Result<(), AkdError> { #[tokio::test] async fn test_simple_publish() -> Result<(), AkdError> { let db = AsyncInMemoryDatabase::new(); + let storage = StorageManager::new_no_cache(&db); let vrf = HardCodedAkdVRF {}; - let akd = Directory::<_, _, Blake3>::new(&db, &vrf, false).await?; + let akd = Directory::<_, _, Blake3>::new(&storage, &vrf, false).await?; // Make sure you can publish and that something so simple // won't throw errors. akd.publish(vec![( @@ -64,8 +67,9 @@ async fn test_simple_publish() -> Result<(), AkdError> { #[tokio::test] async fn test_simple_lookup() -> Result<(), AkdError> { let db = AsyncInMemoryDatabase::new(); + let storage = StorageManager::new_no_cache(&db); let vrf = HardCodedAkdVRF {}; - let akd = Directory::<_, _, Blake3>::new(&db, &vrf, false).await?; + let akd = Directory::<_, _, Blake3>::new(&storage, &vrf, false).await?; // Add two labels and corresponding values to the akd akd.publish(vec![ ( @@ -104,8 +108,9 @@ async fn test_small_key_history() -> Result<(), AkdError> { // The value of this label is updated two times. // Then the test verifies the key history. let db = AsyncInMemoryDatabase::new(); + let storage = StorageManager::new_no_cache(&db); let vrf = HardCodedAkdVRF {}; - let akd = Directory::<_, _, Blake3>::new(&db, &vrf, false).await?; + let akd = Directory::<_, _, Blake3>::new(&storage, &vrf, false).await?; // Publish the first value for the label "hello" // Epoch here will be 1 akd.publish(vec![( @@ -148,8 +153,9 @@ async fn test_small_key_history() -> Result<(), AkdError> { #[tokio::test] async fn test_simple_key_history() -> Result<(), AkdError> { let db = AsyncInMemoryDatabase::new(); + let storage = StorageManager::new_no_cache(&db); let vrf = HardCodedAkdVRF {}; - let akd = Directory::<_, _, Blake3>::new(&db, &vrf, false).await?; + let akd = Directory::<_, _, Blake3>::new(&storage, &vrf, false).await?; // Epoch 1: Add labels "hello" and "hello2" akd.publish(vec![ ( @@ -321,8 +327,9 @@ async fn test_malicious_key_history() -> Result<(), AkdError> { // delay in marking the first version for "hello" as stale, which should // be caught by key history verifications for "hello". let db = AsyncInMemoryDatabase::new(); + let storage = StorageManager::new_no_cache(&db); let vrf = HardCodedAkdVRF {}; - let akd = Directory::<_, _, Blake3>::new(&db, &vrf, false).await?; + let akd = Directory::<_, _, Blake3>::new(&storage, &vrf, false).await?; // Publish the first value for the label "hello" // Epoch here will be 1 akd.publish(vec![( @@ -399,8 +406,9 @@ async fn test_malicious_key_history() -> Result<(), AkdError> { #[tokio::test] async fn test_simple_audit() -> Result<(), AkdError> { let db = AsyncInMemoryDatabase::new(); + let storage = StorageManager::new_no_cache(&db); let vrf = HardCodedAkdVRF {}; - let akd = Directory::<_, _, Blake3>::new(&db, &vrf, false).await?; + let akd = Directory::<_, _, Blake3>::new(&storage, &vrf, false).await?; akd.publish(vec![ ( @@ -558,8 +566,9 @@ async fn test_simple_audit() -> Result<(), AkdError> { #[tokio::test] async fn test_read_during_publish() -> Result<(), AkdError> { let db = AsyncInMemoryDatabase::new(); + let storage = StorageManager::new_no_cache(&db); let vrf = HardCodedAkdVRF {}; - let akd = Directory::<_, _, Blake3>::new(&db, &vrf, false).await?; + let akd = Directory::<_, _, Blake3>::new(&storage, &vrf, false).await?; // Publish once akd.publish(vec![ @@ -665,17 +674,18 @@ async fn test_read_during_publish() -> Result<(), AkdError> { #[tokio::test] async fn test_directory_read_only_mode() -> Result<(), AkdError> { let db = AsyncInMemoryDatabase::new(); + let storage = StorageManager::new_no_cache(&db); let vrf = HardCodedAkdVRF {}; // There is no AZKS object in the storage layer, directory construction should fail - let akd = Directory::<_, _, Blake3>::new(&db, &vrf, true).await; + let akd = Directory::<_, _, Blake3>::new(&storage, &vrf, true).await; assert!(matches!(akd, Err(_))); // now create the AZKS - let akd = Directory::<_, _, Blake3>::new(&db, &vrf, false).await; + let akd = Directory::<_, _, Blake3>::new(&storage, &vrf, false).await; assert!(matches!(akd, Ok(_))); // create another read-only dir now that the AZKS exists in the storage layer, and try to publish which should fail - let akd = Directory::<_, _, Blake3>::new(&db, &vrf, true).await?; + let akd = Directory::<_, _, Blake3>::new(&storage, &vrf, true).await?; assert!(matches!(akd.publish(vec![]).await, Err(_))); Ok(()) @@ -687,9 +697,10 @@ async fn test_directory_read_only_mode() -> Result<(), AkdError> { #[tokio::test] async fn test_directory_polling_azks_change() -> Result<(), AkdError> { let db = AsyncInMemoryDatabase::new(); + let storage = StorageManager::new_no_cache(&db); let vrf = HardCodedAkdVRF {}; // writer will write the AZKS record - let writer = Directory::<_, _, Blake3>::new(&db, &vrf, false).await?; + let writer = Directory::<_, _, Blake3>::new(&storage, &vrf, false).await?; writer .publish(vec![ @@ -705,7 +716,7 @@ async fn test_directory_polling_azks_change() -> Result<(), AkdError> { .await?; // reader will not write the AZKS but will be "polling" for AZKS changes - let reader = Directory::<_, _, Blake3>::new(&db, &vrf, true).await?; + let reader = Directory::<_, _, Blake3>::new(&storage, &vrf, true).await?; // start the poller let (tx, mut rx) = tokio::sync::mpsc::channel(10); @@ -749,9 +760,10 @@ async fn test_directory_polling_azks_change() -> Result<(), AkdError> { #[tokio::test] async fn test_limited_key_history() -> Result<(), AkdError> { let db = AsyncInMemoryDatabase::new(); + let storage = StorageManager::new_no_cache(&db); let vrf = HardCodedAkdVRF {}; // epoch 0 - let akd = Directory::<_, _, Blake3>::new(&db, &vrf, false).await?; + let akd = Directory::<_, _, Blake3>::new(&storage, &vrf, false).await?; // epoch 1 akd.publish(vec![ @@ -887,9 +899,10 @@ async fn test_limited_key_history() -> Result<(), AkdError> { #[tokio::test] async fn test_tombstoned_key_history() -> Result<(), AkdError> { let db = AsyncInMemoryDatabase::new(); + let storage = StorageManager::new_no_cache(&db); let vrf = HardCodedAkdVRF {}; // epoch 0 - let akd = Directory::<_, _, Blake3>::new(&db, &vrf, false).await?; + let akd = Directory::<_, _, Blake3>::new(&storage, &vrf, false).await?; // epoch 1 akd.publish(vec![( @@ -986,9 +999,10 @@ async fn test_tombstoned_key_history() -> Result<(), AkdError> { #[tokio::test] async fn test_simple_lookup_for_small_tree_blake() -> Result<(), AkdError> { let db = AsyncInMemoryDatabase::new(); + let storage = StorageManager::new_no_cache(&db); let vrf = HardCodedAkdVRF {}; // epoch 0 - let akd = Directory::<_, _, Blake3>::new(&db, &vrf, false).await?; + let akd = Directory::<_, _, Blake3>::new(&storage, &vrf, false).await?; // Create a set with 2 updates, (label, value) pairs // ("hello10", "hello10") @@ -1034,9 +1048,10 @@ async fn test_simple_lookup_for_small_tree_blake() -> Result<(), AkdError> { #[tokio::test] async fn test_simple_lookup_for_small_tree_sha256() -> Result<(), AkdError> { let db = AsyncInMemoryDatabase::new(); + let storage = StorageManager::new_no_cache(&db); let vrf = HardCodedAkdVRF {}; // epoch 0 - let akd = Directory::<_, _, Sha3>::new(&db, &vrf, false).await?; + let akd = Directory::<_, _, Sha3>::new(&storage, &vrf, false).await?; // Create a set with 2 updates, (label, value) pairs // ("hello10", "hello10") diff --git a/akd/src/tree_node.rs b/akd/src/tree_node.rs index e1929d05..aacecc6b 100644 --- a/akd/src/tree_node.rs +++ b/akd/src/tree_node.rs @@ -11,8 +11,9 @@ use crate::errors::{AkdError, StorageError, TreeNodeError}; #[cfg(feature = "serde_serialization")] use crate::serialization::{bytes_deserialize_hex, bytes_serialize_hex}; use crate::serialization::{from_digest, to_digest}; +use crate::storage::storage::StorageManager; use crate::storage::types::{DbRecord, StorageType}; -use crate::storage::{Storable, Storage}; +use crate::storage::{Database as Storage, Storable}; use crate::{node_label::*, Direction, EMPTY_LABEL}; use async_recursion::async_recursion; use log::debug; @@ -171,13 +172,13 @@ impl TreeNodeWithPreviousValue { pub(crate) async fn write_to_storage( &self, - storage: &S, + storage: &StorageManager, ) -> Result<(), StorageError> { storage.set(DbRecord::TreeNode(self.clone())).await } pub(crate) async fn get_appropriate_tree_node_from_storage( - storage: &S, + storage: &StorageManager, key: &NodeKey, target_epoch: u64, ) -> Result { @@ -191,7 +192,7 @@ impl TreeNodeWithPreviousValue { } pub(crate) async fn batch_get_appropriate_tree_node_from_storage( - storage: &S, + storage: &StorageManager, keys: &[NodeKey], target_epoch: u64, ) -> Result, StorageError> { @@ -269,7 +270,7 @@ impl TreeNode { // Storage operations pub(crate) async fn write_to_storage( &self, - storage: &S, + storage: &StorageManager, ) -> Result<(), StorageError> { self.write_to_storage_impl(storage, false).await } @@ -278,7 +279,7 @@ impl TreeNode { /// will be used as None without the cost of finding this information in the cache or worse yet in the database. async fn write_to_storage_impl( &self, - storage: &S, + storage: &StorageManager, is_new_node: bool, ) -> Result<(), StorageError> { // MOTIVATION: @@ -325,7 +326,7 @@ impl TreeNode { } pub(crate) async fn get_from_storage( - storage: &S, + storage: &StorageManager, key: &NodeKey, target_epoch: u64, ) -> Result { @@ -338,7 +339,7 @@ impl TreeNode { } pub(crate) async fn batch_get_from_storage( - storage: &S, + storage: &StorageManager, keys: &[NodeKey], target_epoch: u64, ) -> Result, StorageError> { @@ -381,7 +382,7 @@ impl TreeNode { #[allow(clippy::too_many_arguments)] /// Creates a new TreeNode and writes it to the storage. async fn new( - storage: &S, + storage: &StorageManager, label: NodeLabel, parent: NodeLabel, node_type: NodeType, @@ -411,7 +412,7 @@ impl TreeNode { #[cfg(test)] pub(crate) async fn insert_single_leaf_and_hash( &mut self, - storage: &S, + storage: &StorageManager, new_leaf: Self, epoch: u64, num_nodes: &mut u64, @@ -430,7 +431,7 @@ impl TreeNode { /// in an amortized way, at a later time. pub(crate) async fn insert_leaf( &mut self, - storage: &S, + storage: &StorageManager, new_leaf: Self, epoch: u64, num_nodes: &mut u64, @@ -449,7 +450,7 @@ impl TreeNode { #[async_recursion] pub(crate) async fn insert_single_leaf_helper( &mut self, - storage: &S, + storage: &StorageManager, new_leaf: Self, epoch: u64, num_nodes: &mut u64, @@ -509,7 +510,7 @@ impl TreeNode { H: Hasher, >( &mut self, - storage: &S, + storage: &StorageManager, mut new_leaf: Self, epoch: u64, hashing: bool, @@ -551,7 +552,7 @@ impl TreeNode { H: Hasher, >( &mut self, - storage: &S, + storage: &StorageManager, mut new_leaf: Self, epoch: u64, num_nodes: &mut u64, @@ -631,7 +632,7 @@ impl TreeNode { H: Hasher, >( &mut self, - storage: &S, + storage: &StorageManager, new_leaf: Self, epoch: u64, num_nodes: &mut u64, @@ -675,7 +676,7 @@ impl TreeNode { /// Updates the node hash and saves it in storage. pub(crate) async fn update_node_hash( &mut self, - storage: &S, + storage: &StorageManager, epoch: u64, exclude_ep: Option, ) -> Result<(), AkdError> { @@ -715,7 +716,7 @@ impl TreeNode { /// without updating its own hash. pub(crate) async fn set_child( &mut self, - storage: &S, + storage: &StorageManager, child: &mut InsertionNode<'_>, epoch: u64, ) -> Result<(), StorageError> { @@ -794,7 +795,7 @@ impl TreeNode { /// Loads (from storage) the left or right child of a node using given direction pub(crate) async fn get_child_state( &self, - storage: &S, + storage: &StorageManager, direction: Direction, current_epoch: u64, ) -> Result, AkdError> { @@ -907,7 +908,7 @@ pub(crate) fn optional_child_state_hash( /// Create an empty root node. pub async fn create_empty_root( - storage: &S, + storage: &StorageManager, ep: Option, least_descendant_ep: Option, ) -> Result { @@ -937,7 +938,7 @@ pub async fn create_empty_root( /// Create a specific leaf node. pub async fn create_leaf_node( - storage: &S, + storage: &StorageManager, label: NodeLabel, value: &H::Digest, parent: NodeLabel, @@ -971,10 +972,12 @@ mod tests { type Blake3 = Blake3_256; type InMemoryDb = crate::storage::memory::AsyncInMemoryDatabase; + use crate::storage::storage::StorageManager; #[tokio::test] async fn test_least_descendant_ep() -> Result<(), AkdError> { - let db = InMemoryDb::new(); + let database = InMemoryDb::new(); + let db = StorageManager::new_no_cache(&database); let mut root = create_empty_root::(&db, Option::Some(0u64), Option::Some(0u64)) .await?; @@ -1076,7 +1079,8 @@ mod tests { // insert_single_leaf tests #[tokio::test] async fn test_insert_single_leaf_root() -> Result<(), AkdError> { - let db = InMemoryDb::new(); + let database = InMemoryDb::new(); + let db = StorageManager::new_no_cache(&database); let mut root = create_empty_root::(&db, Option::Some(0u64), Option::Some(0u64)) @@ -1147,7 +1151,8 @@ mod tests { #[tokio::test] async fn test_insert_single_leaf_below_root() -> Result<(), AkdError> { - let db = InMemoryDb::new(); + let database = InMemoryDb::new(); + let db = StorageManager::new_no_cache(&database); let mut root = create_empty_root::(&db, Option::Some(0u64), Option::Some(0u64)) .await?; @@ -1230,7 +1235,8 @@ mod tests { #[tokio::test] async fn test_insert_single_leaf_below_root_both_sides() -> Result<(), AkdError> { - let db = InMemoryDb::new(); + let database = InMemoryDb::new(); + let db = StorageManager::new_no_cache(&database); let mut root = create_empty_root::(&db, Option::Some(0u64), Option::Some(0u64)) .await?; @@ -1336,7 +1342,8 @@ mod tests { #[tokio::test] async fn test_insert_single_leaf_full_tree() -> Result<(), AkdError> { - let db = InMemoryDb::new(); + let database = InMemoryDb::new(); + let db = StorageManager::new_no_cache(&database); let mut root = create_empty_root::(&db, Option::Some(0u64), Option::Some(0u64)) .await?; From 83c7f202e12a0bd52087047b32cbf240d7471149 Mon Sep 17 00:00:00 2001 From: Sean Lawlor Date: Wed, 23 Nov 2022 14:06:23 -0500 Subject: [PATCH 04/13] Refactoring the other libraries to support the new storage model --- akd/src/lib.rs | 2 +- akd/src/storage/caches/basic.rs | 4 +- akd/src/storage/storage.rs | 3 +- akd_client/src/tests.rs | 21 +- akd_local_auditor/src/common_test.rs | 4 +- akd_mysql/Cargo.toml | 3 +- akd_mysql/src/mysql.rs | 658 +++++------------- akd_mysql/src/mysql_db_tests.rs | 1 - .../examples/example_tests.rs | 5 +- .../src/fixture_generator/generator.rs | 5 +- akd_test_tools/src/test_suites.rs | 4 +- integration_tests/Cargo.toml | 5 +- integration_tests/src/memory_tests.rs | 10 +- integration_tests/src/mysql_tests.rs | 20 +- integration_tests/src/test_util.rs | 11 +- poc/src/directory_host.rs | 6 +- poc/src/main.rs | 24 +- 17 files changed, 236 insertions(+), 550 deletions(-) diff --git a/akd/src/lib.rs b/akd/src/lib.rs index 0532e669..68da009a 100644 --- a/akd/src/lib.rs +++ b/akd/src/lib.rs @@ -315,7 +315,7 @@ //! # use akd::directory::Directory; //! # //! # let db = AsyncInMemoryDatabase::new(); -//! # let storage_manager = StorageManager::new_no_cache(&db); +//! # let storage_manager = StorageManager::new_no_cache(&db); //! # let vrf = HardCodedAkdVRF{}; //! # use akd::EpochHash; //! # use akd::storage::types::{AkdLabel, AkdValue}; diff --git a/akd/src/storage/caches/basic.rs b/akd/src/storage/caches/basic.rs index 5e43c057..a4905731 100644 --- a/akd/src/storage/caches/basic.rs +++ b/akd/src/storage/caches/basic.rs @@ -54,8 +54,8 @@ impl TimedCache { "Cache hit since last: {}, cached size: {} items", hit_count, cache_size ); - match level { - log::Level::Trace => trace!("{}", msg), + match _level { + log::Level::Trace => println!("{}", msg), log::Level::Debug => debug!("{}", msg), log::Level::Info => info!("{}", msg), log::Level::Warn => warn!("{}", msg), diff --git a/akd/src/storage/storage.rs b/akd/src/storage/storage.rs index c54a7226..430b3ec3 100644 --- a/akd/src/storage/storage.rs +++ b/akd/src/storage/storage.rs @@ -49,7 +49,8 @@ const NUM_METRICS: usize = 10; pub struct StorageManager { cache: Option, transaction: Transaction, - db: Db, + /// The underlying database managed by this storage manager + pub db: Db, metrics: Arc>, } diff --git a/akd_client/src/tests.rs b/akd_client/src/tests.rs index 291e5923..c7a0e332 100644 --- a/akd_client/src/tests.rs +++ b/akd_client/src/tests.rs @@ -19,7 +19,7 @@ use alloc::vec; use alloc::vec::Vec; use akd::errors::{AkdError, StorageError}; -use akd::storage::Storage; +use akd::storage::StorageManager; use akd::{AkdLabel, AkdValue}; use winter_crypto::Hasher; @@ -56,7 +56,9 @@ fn make_unparsable_json(serialized_json: &str) -> String { #[tokio::test] async fn test_simple_lookup() -> Result<(), AkdError> { - let db = InMemoryDb::new(); + let database = InMemoryDb::new(); + let db = StorageManager::new_no_cache(&database); + let vrf = HardCodedAkdVRF {}; let akd = Directory::new(&db, &vrf, false).await?; @@ -163,7 +165,8 @@ async fn test_simple_lookup() -> Result<(), AkdError> { #[tokio::test] async fn test_simple_lookup_for_small_tree() -> Result<(), AkdError> { - let db = InMemoryDb::new(); + let database = InMemoryDb::new(); + let db = StorageManager::new_no_cache(&database); let vrf = HardCodedAkdVRF {}; let akd = Directory::new(&db, &vrf, false).await?; @@ -220,7 +223,8 @@ async fn test_simple_lookup_for_small_tree() -> Result<(), AkdError> { #[tokio::test] async fn test_history_proof_multiple_epochs() -> Result<(), AkdError> { - let db = InMemoryDb::new(); + let database = InMemoryDb::new(); + let db = StorageManager::new_no_cache(&database); let vrf = HardCodedAkdVRF {}; let akd = Directory::new(&db, &vrf, false).await?; let vrf_pk = akd.get_public_key().await.unwrap(); @@ -374,7 +378,8 @@ async fn test_history_proof_multiple_epochs() -> Result<(), AkdError> { #[tokio::test] async fn test_history_proof_single_epoch() -> Result<(), AkdError> { - let db = InMemoryDb::new(); + let database = InMemoryDb::new(); + let db = StorageManager::new_no_cache(&database); let vrf = HardCodedAkdVRF {}; let akd = Directory::new(&db, &vrf, false).await?; let vrf_pk = akd.get_public_key().await.unwrap(); @@ -408,7 +413,8 @@ async fn test_history_proof_single_epoch() -> Result<(), AkdError> { #[tokio::test] async fn test_tombstoned_key_history() -> Result<(), AkdError> { - let db = InMemoryDb::new(); + let database = InMemoryDb::new(); + let db = StorageManager::new_no_cache(&database); let vrf = HardCodedAkdVRF {}; // epoch 0 let akd = Directory::new(&db, &vrf, false).await?; @@ -533,7 +539,8 @@ async fn test_malicious_key_history() -> Result<(), AkdError> { // insertion of a new label "hello2". Meanwhile, the server has a one epoch // delay in marking the first version for "hello" as stale, which should // be caught by key history verifications for "hello". - let db = InMemoryDb::new(); + let database = InMemoryDb::new(); + let db = StorageManager::new_no_cache(&database); let vrf = HardCodedAkdVRF {}; let akd = Directory::new(&db, &vrf, false).await?; // Publish the first value for the label "hello" diff --git a/akd_local_auditor/src/common_test.rs b/akd_local_auditor/src/common_test.rs index c6283248..5eec605a 100644 --- a/akd_local_auditor/src/common_test.rs +++ b/akd_local_auditor/src/common_test.rs @@ -12,6 +12,7 @@ use crate::{Digest, Hasher}; use akd::directory::Directory; use akd::ecvrf::HardCodedAkdVRF; use akd::storage::memory::AsyncInMemoryDatabase; +use akd::storage::StorageManager; use akd::AkdLabel; use akd::AkdValue; @@ -60,8 +61,9 @@ pub async fn generate_audit_proofs( expensive: bool, ) -> Result, akd::errors::AkdError> { let db = AsyncInMemoryDatabase::new(); + let storage_manager = StorageManager::new_no_cache(&db); let vrf = HardCodedAkdVRF {}; - let akd = Directory::<_, _, Hasher>::new(&db, &vrf, false).await?; + let akd = Directory::<_, _, Hasher>::new(&storage_manager, &vrf, false).await?; let mut proofs = vec![]; // gather the hash + azks for epoch "0" (init) let mut azks = akd.retrieve_current_azks().await?; diff --git a/akd_mysql/Cargo.toml b/akd_mysql/Cargo.toml index 088ab2a2..1aa57fcd 100644 --- a/akd_mysql/Cargo.toml +++ b/akd_mysql/Cargo.toml @@ -9,7 +9,8 @@ keywords = ["key-transparency", "akd", "mysql", "akd-mysql"] repository = "https://github.com/novifinancial/akd" [features] -bench = [] +# Collect runtime metrics on db access calls + timing +runtime_metrics = [] [dependencies] bincode = "1" diff --git a/akd_mysql/src/mysql.rs b/akd_mysql/src/mysql.rs index 0e4fb93f..604b0e1d 100644 --- a/akd_mysql/src/mysql.rs +++ b/akd_mysql/src/mysql.rs @@ -12,7 +12,7 @@ use akd::errors::StorageError; use akd::storage::types::{ AkdLabel, AkdValue, DbRecord, KeyData, StorageType, ValueState, ValueStateRetrievalFlag, }; -use akd::storage::{Storable, Storage}; +use akd::storage::{Database, Storable}; use akd::tree_node::TreeNodeWithPreviousValue; use akd::NodeLabel; use async_trait::async_trait; @@ -21,16 +21,13 @@ use mysql_async::prelude::*; use mysql_async::*; use std::cmp::Ordering; -use std::collections::{HashMap, HashSet}; +use std::collections::HashMap; use std::convert::TryInto; use std::process::Command; use std::sync::Arc; -use tokio::time::{Duration, Instant}; +use tokio::time::Instant; type MySqlError = mysql_async::Error; -type LocalTransaction = akd::storage::transaction::Transaction; - -use akd::storage::caches::*; const TABLE_AZKS: &str = crate::mysql_storables::TABLE_AZKS; const TABLE_HISTORY_TREE_NODES: &str = crate::mysql_storables::TABLE_HISTORY_TREE_NODES; @@ -59,30 +56,14 @@ enum BatchMode { MySql documentation: https://docs.rs/mysql_async/0.23.1/mysql_async/ */ -/// Memory cache options for SQL query result caching -pub enum MySqlCacheOptions { - /// Do not utilize any cache - None, - /// Utilize the default caching settings - Default, - /// Customize the caching options (cache item duration, memory_limit_bytes) - Specific(std::time::Duration, Option), -} - /// Represents an _asynchronous_ connection to a MySQL database pub struct AsyncMySqlDatabase { opts: Opts, pool: Arc>, is_healthy: Arc>, - cache: Option, - trans: LocalTransaction, - num_reads: Arc>, read_call_stats: Arc>>, - num_writes: Arc>, write_call_stats: Arc>>, - time_read: Arc>, - time_write: Arc>, tunable_insert_depth: usize, } @@ -115,15 +96,9 @@ impl Clone for AsyncMySqlDatabase { opts: self.opts.clone(), pool: self.pool.clone(), is_healthy: self.is_healthy.clone(), - cache: self.cache.clone(), - trans: LocalTransaction::new(), - num_reads: self.num_reads.clone(), read_call_stats: self.read_call_stats.clone(), - num_writes: self.num_writes.clone(), write_call_stats: self.write_call_stats.clone(), - time_read: self.time_read.clone(), - time_write: self.time_write.clone(), tunable_insert_depth: self.tunable_insert_depth, } @@ -139,7 +114,6 @@ impl<'a> AsyncMySqlDatabase { user: Option, password: Option, port: Option, - cache_options: MySqlCacheOptions, depth: usize, ) -> Self { let dport = port.unwrap_or(3306u16); @@ -158,27 +132,12 @@ impl<'a> AsyncMySqlDatabase { // prior to the directory let pool = Self::new_connection_pool(&opts, &healthy).await.unwrap(); - let cache = match cache_options { - MySqlCacheOptions::None => None, - MySqlCacheOptions::Default => Some(TimedCache::new(None, None)), - MySqlCacheOptions::Specific(timing, memory_limit_bytes) => { - Some(TimedCache::new(Some(timing), memory_limit_bytes)) - } - }; - Self { opts, pool: Arc::new(tokio::sync::RwLock::new(pool)), is_healthy: healthy, - cache, - trans: LocalTransaction::new(), - - num_reads: Arc::new(tokio::sync::RwLock::new(0)), read_call_stats: Arc::new(tokio::sync::RwLock::new(HashMap::new())), - num_writes: Arc::new(tokio::sync::RwLock::new(0)), write_call_stats: Arc::new(tokio::sync::RwLock::new(HashMap::new())), - time_read: Arc::new(tokio::sync::RwLock::new(Duration::from_millis(0))), - time_write: Arc::new(tokio::sync::RwLock::new(Duration::from_millis(0))), tunable_insert_depth: depth, } @@ -417,13 +376,10 @@ impl<'a> AsyncMySqlDatabase { record: DbRecord, trans: Option>, ) -> Result<()> { - *(self.num_writes.write().await) += 1; self.record_call_stats('w', "internal_set".to_string(), "".to_string()) .await; debug!("BEGIN MySQL set"); - let tic = Instant::now(); - let statement_text = record.set_statement(); let params = record .set_params() @@ -440,8 +396,6 @@ impl<'a> AsyncMySqlDatabase { } }; self.check_for_infra_error(out)?; - let toc = Instant::now() - tic; - *(self.time_write.write().await) += toc; debug!("END MySQL set"); Ok(()) @@ -457,7 +411,6 @@ impl<'a> AsyncMySqlDatabase { return Ok(trans); } - *(self.num_writes.write().await) += records.len() as u64; self.record_call_stats('w', "internal_batch_set".to_string(), "".to_string()) .await; @@ -502,8 +455,6 @@ impl<'a> AsyncMySqlDatabase { } } - let tic = Instant::now(); - debug!("MySQL batch - {} full inserts", params.len()); // insert the batches of size = MYSQL_EXTENDED_INSERT_DEPTH if !params.is_empty() { @@ -520,8 +471,6 @@ impl<'a> AsyncMySqlDatabase { self.check_for_infra_error(out)?; } - let toc = Instant::now() - tic; - *(self.time_write.write().await) += toc; debug!("END MySQL set batch"); Ok(trans) } @@ -548,17 +497,22 @@ impl<'a> AsyncMySqlDatabase { Ok(()) } - async fn record_call_stats(&self, call_type: char, caller_name: String, data_type: String) { - let mut stats; - if call_type == 'r' { - stats = self.read_call_stats.write().await; - } else if call_type == 'w' { - stats = self.write_call_stats.write().await; - } else { - panic!("Unknown call type to record call stats for.") + async fn record_call_stats(&self, _call_type: char, _caller_name: String, _data_type: String) { + #[cfg(feature = "runtime_metrics")] + { + let mut stats; + if _call_type == 'r' { + stats = self.read_call_stats.write().await; + } else if _call_type == 'w' { + stats = self.write_call_stats.write().await; + } else { + panic!("Unknown call type to record call stats for.") + } + let call_count = (*stats) + .entry(_caller_name + "~" + &_data_type) + .or_insert(0); + *call_count += 1; } - let call_count = (*stats).entry(caller_name + "~" + &data_type).or_insert(0); - *call_count += 1; } fn try_dockers() -> std::io::Result { @@ -629,155 +583,63 @@ impl<'a> AsyncMySqlDatabase { // docker may have thrown an error, just fail false } -} -#[async_trait] -impl Storage for AsyncMySqlDatabase { - async fn log_metrics(&self, level: log::Level) { - if let Some(cache) = &self.cache { - cache.log_metrics(level).await - } + async fn get_direct( + &self, + id: &St::StorageKey, + ) -> core::result::Result { + self.record_call_stats( + 'r', + "get_direct:".to_string(), + format!("{:?}", St::data_type()), + ) + .await; - self.trans.log_metrics(level).await; - - let mut tree_size = "Tree size: Query err".to_string(); - let mut value_state_size = "Value state count: Query err".to_string(); - if let Ok(mut conn) = self.get_connection().await { - let query_text = format!("SELECT COUNT(*) FROM {}", TABLE_HISTORY_TREE_NODES); - if let Ok(results) = conn.query_iter(query_text).await { - if let Ok(mapped) = results - .map_and_drop(|row| { - let count: u64 = mysql_async::from_row(row); - count - }) - .await - { - if let Some(count) = mapped.first() { - tree_size = format!("Tree size: {}", count); - } + debug!("BEGIN MySQL get {:?}", id); + let result = async { + let mut conn = self.get_connection().await?; + let statement = DbRecord::get_specific_statement::(); + let params = DbRecord::get_specific_params::(id); + let out = match params { + Some(p) => match conn.exec_first(statement, p).await { + Err(err) => Err(err), + Ok(result) => Ok(result), + }, + None => match conn.query_first(statement).await { + Err(err) => Err(err), + Ok(result) => Ok(result), + }, + }; - let query_text = format!("SELECT COUNT(`epoch`) FROM {}", TABLE_USER); - if let Ok(results) = conn.query_iter(query_text).await { - if let Ok(mapped) = results - .map_and_drop(|row| { - let count: u64 = mysql_async::from_row(row); - count - }) - .await - { - if let Some(count) = mapped.first() { - value_state_size = format!("Value state count: {}", count); - } - } - } - } + let result = self.check_for_infra_error(out)?; + if let Some(mut row) = result { + // return result + let record = DbRecord::from_row::(&mut row)?; + return Ok::, MySqlError>(Some(record)); } - } - - let mut r = self.num_reads.write().await; - let mut rcs = self.read_call_stats.write().await; - let mut w = self.num_writes.write().await; - let mut wcs = self.write_call_stats.write().await; - let mut tr = self.time_read.write().await; - let mut tw = self.time_write.write().await; - - // Sort call stats for consistency. - let mut rcs_vec = (*rcs).iter().collect::>(); - let mut wcs_vec = (*wcs).iter().collect::>(); - rcs_vec.sort_by_key(|rc| rc.0); - wcs_vec.sort_by_key(|wc| wc.0); - - let msg = format!( - "MySQL writes: {}, MySQL reads: {}, Time read: {} s, Time write: {} s\n\t{}\n\t{}\nRead call stats: {:?}\nWrite call stats: {:?}\n", - *w, - *r, - (*tr).as_secs_f64(), - (*tw).as_secs_f64(), - tree_size, - value_state_size, - rcs_vec, - wcs_vec, - ); - - *r = 0; - *rcs = HashMap::new(); - *w = 0; - *wcs = HashMap::new(); - *tr = Duration::from_millis(0); - *tw = Duration::from_millis(0); - - match level { - // Currently logs cannot be captured unless they are - // println!. Normally Level::Trace should use the trace! macro. - log::Level::Trace => println!("{}", msg), - log::Level::Debug => debug!("{}", msg), - log::Level::Info => info!("{}", msg), - log::Level::Warn => warn!("{}", msg), - _ => error!("{}", msg), - } - } - - /// Start a transaction in the storage layer - async fn begin_transaction(&self) -> bool { - // disable the cache cleaning since we're in a write transaction - // and will want to keep cache'd objects for the life of the transaction - if let Some(cache) = &self.cache { - cache.disable_clean(); - } - - self.trans.begin_transaction().await - } - - /// Commit a transaction in the storage layer - async fn commit_transaction(&self) -> core::result::Result<(), StorageError> { - // The transaction is now complete (or reverted) and therefore we can re-enable - // the cache cleaning status - if let Some(cache) = &self.cache { - cache.enable_clean(); - } - - // this retrieves all the trans operations, and "de-activates" the transaction flag - let ops = self.trans.commit_transaction().await?; + Ok::, MySqlError>(None) + }; - let _epoch = match ops.last() { - Some(DbRecord::Azks(azks)) => Ok(azks.latest_epoch), - other => Err(StorageError::Transaction(format!( - "The last record in the transaction log is NOT an Azks record {:?}", - other + debug!("END MySQL get"); + match result.await { + Ok(Some(r)) => Ok(r), + Ok(None) => Err(StorageError::NotFound(format!( + "{:?} {:?}", + St::data_type(), + id ))), - }?; - - self.batch_set(ops).await - } - - /// Rollback a transaction - async fn rollback_transaction(&self) -> core::result::Result<(), StorageError> { - // The transaction is being reverted and therefore we can re-enable - // the cache cleaning status - if let Some(cache) = &self.cache { - cache.enable_clean(); + Err(error) => { + error!("MySQL error {}", error); + Err(StorageError::Other(format!("MySQL Error {}", error))) + } } - - self.trans.rollback_transaction().await - } - - /// Retrieve a flag determining if there is a transaction active - async fn is_transaction_active(&self) -> bool { - self.trans.is_transaction_active().await } +} +#[async_trait] +impl Database for AsyncMySqlDatabase { /// Storage a record in the data layer async fn set(&self, record: DbRecord) -> core::result::Result<(), StorageError> { - // we're in a transaction, set the item in the transaction - if self.is_transaction_active().await { - self.trans.set(&record).await; - return Ok(()); - } - - if let Some(cache) = &self.cache { - cache.put(&record).await; - } - match self.internal_set(record, None).await { Ok(_) => Ok(()), Err(error) => { @@ -793,18 +655,6 @@ impl Storage for AsyncMySqlDatabase { return Ok(()); } - // we're in a transaction, set the items in the transaction - if self.is_transaction_active().await { - for record in records.iter() { - self.trans.set(record).await; - } - return Ok(()); - } - - if let Some(cache) = &self.cache { - let _ = cache.batch_put(&records).await; - } - // generate batches by type let mut groups = std::collections::HashMap::new(); for record in records { @@ -882,92 +732,7 @@ impl Storage for AsyncMySqlDatabase { &self, id: &St::StorageKey, ) -> core::result::Result { - // we're in a transaction, meaning the object _might_ be newer and therefore we should try and read if from the transaction - // log instead of the raw storage layer - if self.is_transaction_active().await { - if let Some(result) = self.trans.get::(id).await { - return Ok(result); - } - } - - // check for a cache hit - if let Some(cache) = &self.cache { - if let Some(result) = cache.hit_test::(id).await { - return Ok(result); - } - } - - // cache miss, log a real sql read op - let record = self.get_direct::(id).await?; - if let Some(cache) = &self.cache { - // cache the result - cache.put(&record).await; - } - Ok(record) - } - - async fn get_direct( - &self, - id: &St::StorageKey, - ) -> core::result::Result { - *(self.num_reads.write().await) += 1; - self.record_call_stats( - 'r', - "get_direct:".to_string(), - format!("{:?}", St::data_type()), - ) - .await; - - debug!("BEGIN MySQL get {:?}", id); - let result = async { - let tic = Instant::now(); - - let mut conn = self.get_connection().await?; - let statement = DbRecord::get_specific_statement::(); - let params = DbRecord::get_specific_params::(id); - let out = match params { - Some(p) => match conn.exec_first(statement, p).await { - Err(err) => Err(err), - Ok(result) => Ok(result), - }, - None => match conn.query_first(statement).await { - Err(err) => Err(err), - Ok(result) => Ok(result), - }, - }; - - let toc = Instant::now() - tic; - *(self.time_read.write().await) += toc; - - let result = self.check_for_infra_error(out)?; - if let Some(mut row) = result { - // return result - let record = DbRecord::from_row::(&mut row)?; - return Ok::, MySqlError>(Some(record)); - } - Ok::, MySqlError>(None) - }; - - debug!("END MySQL get"); - match result.await { - Ok(Some(r)) => Ok(r), - Ok(None) => Err(StorageError::NotFound(format!( - "{:?} {:?}", - St::data_type(), - id - ))), - Err(error) => { - error!("MySQL error {}", error); - Err(StorageError::Other(format!("MySQL Error {}", error))) - } - } - } - - /// Flush the caching of objects (if present) - async fn flush_cache(&self) { - if let Some(cache) = &self.cache { - cache.flush().await; - } + self.get_direct::(id).await } /// Retrieve a batch of records by id @@ -982,162 +747,120 @@ impl Storage for AsyncMySqlDatabase { return Ok(map); } - let mut key_set: HashSet = ids.iter().cloned().collect::>(); - - let trans_active = self.is_transaction_active().await; - // first check the transaction log & cache records - for id in ids.iter() { - if trans_active { - // we're in a transaction, meaning the object _might_ be newer and therefore we should try and read if from the transaction - // log instead of the raw storage layer - if let Some(result) = self.trans.get::(id).await { - map.push(result); - key_set.remove(id); - continue; - } - } - - // check if item is cached - if let Some(cache) = &self.cache { - if let Some(result) = cache.hit_test::(id).await { - map.push(result); - key_set.remove(id); - continue; - } - } - } + let result = async { + let key_set_vec: Vec<_> = ids.to_vec(); - if !key_set.is_empty() { - // these are items to be retrieved from the backing database (not in pending transaction or in the object cache) - let result = async { - let tic = Instant::now(); + debug!("BEGIN MySQL get batch"); + let mut conn = self.get_connection().await?; - let key_set_vec: Vec<_> = key_set.into_iter().collect(); + let results = if let Some(create_table_cmd) = + DbRecord::get_batch_create_temp_table::() + { + // Create the temp table of ids + let out = conn.query_drop(create_table_cmd).await; + self.check_for_infra_error(out)?; - debug!("BEGIN MySQL get batch"); - let mut conn = self.get_connection().await?; + // Fill temp table with the requested ids + let mut tx = conn.start_transaction(TxOpts::default()).await?; + tx.query_drop("SET autocommit=0").await?; + tx.query_drop("SET unique_checks=0").await?; + tx.query_drop("SET foreign_key_checks=0").await?; + + let mut fallout: Option> = None; + let mut params = vec![]; + for batch in key_set_vec.chunks(self.tunable_insert_depth) { + if batch.len() < self.tunable_insert_depth { + fallout = Some(batch.to_vec()); + } else if let Some(p) = DbRecord::get_multi_row_specific_params::(batch) { + params.push(p); + } else { + return Err(MySqlError::Other( + "Unable to generate type-specific MySQL parameters".into(), + )); + } + } - let results = if let Some(create_table_cmd) = - DbRecord::get_batch_create_temp_table::() - { - // Create the temp table of ids - let out = conn.query_drop(create_table_cmd).await; + // insert the batches of size = MYSQL_EXTENDED_INSERT_DEPTH + if !params.is_empty() { + let fill_statement = + DbRecord::get_batch_fill_temp_table::(Some(self.tunable_insert_depth)); + let out = tx.exec_batch(fill_statement, params).await; self.check_for_infra_error(out)?; + // We would need the statement for it. (Possibly) No need for close here. + // See https://docs.rs/mysql_async/0.28.1/mysql_async/struct.Opts.html#caveats. + // tx.close().await?; + } - // Fill temp table with the requested ids - let mut tx = conn.start_transaction(TxOpts::default()).await?; - tx.query_drop("SET autocommit=0").await?; - tx.query_drop("SET unique_checks=0").await?; - tx.query_drop("SET foreign_key_checks=0").await?; - - let mut fallout: Option> = None; - let mut params = vec![]; - for batch in key_set_vec.chunks(self.tunable_insert_depth) { - if batch.len() < self.tunable_insert_depth { - fallout = Some(batch.to_vec()); - } else if let Some(p) = DbRecord::get_multi_row_specific_params::(batch) - { - params.push(p); - } else { - return Err(MySqlError::Other( - "Unable to generate type-specific MySQL parameters".into(), - )); - } - } - - // insert the batches of size = MYSQL_EXTENDED_INSERT_DEPTH - if !params.is_empty() { - let fill_statement = DbRecord::get_batch_fill_temp_table::(Some( - self.tunable_insert_depth, - )); - let out = tx.exec_batch(fill_statement, params).await; + // insert the remainder as a final statement + if let Some(remainder) = fallout { + let remainder_stmt = + DbRecord::get_batch_fill_temp_table::(Some(remainder.len())); + let params_batch = DbRecord::get_multi_row_specific_params::(&remainder); + if let Some(pb) = params_batch { + let out = tx.exec_drop(remainder_stmt, pb).await; self.check_for_infra_error(out)?; - // We would need the statement for it. (Possibly) No need for close here. - // See https://docs.rs/mysql_async/0.28.1/mysql_async/struct.Opts.html#caveats. - // tx.close().await?; - } - - // insert the remainder as a final statement - if let Some(remainder) = fallout { - let remainder_stmt = - DbRecord::get_batch_fill_temp_table::(Some(remainder.len())); - let params_batch = - DbRecord::get_multi_row_specific_params::(&remainder); - if let Some(pb) = params_batch { - let out = tx.exec_drop(remainder_stmt, pb).await; - self.check_for_infra_error(out)?; - } else { - return Err(MySqlError::Other( - "Unable to generate type-specific MySQL parameters".into(), - )); - } + } else { + return Err(MySqlError::Other( + "Unable to generate type-specific MySQL parameters".into(), + )); } + } - tx.query_drop("SET autocommit=1").await?; - tx.query_drop("SET unique_checks=1").await?; - tx.query_drop("SET foreign_key_checks=1").await?; - tx.commit().await?; + tx.query_drop("SET autocommit=1").await?; + tx.query_drop("SET unique_checks=1").await?; + tx.query_drop("SET foreign_key_checks=1").await?; + tx.commit().await?; - // Query the records which intersect (INNER JOIN) with the temp table of ids - let query = DbRecord::get_batch_statement::(); - let out = conn.query_iter(query).await; - let result = self.check_for_infra_error(out)?; + // Query the records which intersect (INNER JOIN) with the temp table of ids + let query = DbRecord::get_batch_statement::(); + let out = conn.query_iter(query).await; + let result = self.check_for_infra_error(out)?; - let out = result - .reduce_and_drop(vec![], |mut acc, mut row| { - if let Ok(result) = DbRecord::from_row::(&mut row) { - acc.push(result); - } - acc - }) - .await?; - - // drop the temp table of ids - let t_out = conn - .query_drop(format!("DROP TEMPORARY TABLE `{}`", TEMP_IDS_TABLE)) - .await; - self.check_for_infra_error(t_out)?; + let out = result + .reduce_and_drop(vec![], |mut acc, mut row| { + if let Ok(result) = DbRecord::from_row::(&mut row) { + acc.push(result); + } + acc + }) + .await?; - out - } else { - // no results (i.e. AZKS table doesn't support "get by batch ids") - vec![] - }; + // drop the temp table of ids + let t_out = conn + .query_drop(format!("DROP TEMPORARY TABLE `{}`", TEMP_IDS_TABLE)) + .await; + self.check_for_infra_error(t_out)?; - debug!("END MySQL get batch"); - let toc = Instant::now() - tic; - *(self.time_read.write().await) += toc; + out + } else { + // no results (i.e. AZKS table doesn't support "get by batch ids") + vec![] + }; - if let Some(cache) = &self.cache { - // insert retrieved records into the cache for faster future access - for el in results.iter() { - cache.put(el).await; - } - } + debug!("END MySQL get batch"); - Ok::, mysql_async::Error>(results) - }; + Ok::, mysql_async::Error>(results) + }; - *(self.num_reads.write().await) += 1; - self.record_call_stats( - 'r', - "batch_get".to_string(), - format!("{:?}", St::data_type()), - ) - .await; + self.record_call_stats( + 'r', + "batch_get".to_string(), + format!("{:?}", St::data_type()), + ) + .await; - match result.await { - Ok(result_vec) => { - for item in result_vec.into_iter() { - map.push(item); - } - } - Err(error) => { - error!("MySQL error {}", error); - return Err(StorageError::Other(format!("MySQL Error {}", error))); + match result.await { + Ok(result_vec) => { + for item in result_vec.into_iter() { + map.push(item); } } + Err(error) => { + error!("MySQL error {}", error); + return Err(StorageError::Other(format!("MySQL Error {}", error))); + } } + Ok(map) } @@ -1188,15 +911,12 @@ impl Storage for AsyncMySqlDatabase { ) -> core::result::Result { // This is the same as previous logic under "get_all" - *(self.num_reads.write().await) += 1; self.record_call_stats('r', "get_user_data".to_string(), "".to_string()) .await; // DO NOT log the user info, it's PII in the future debug!("BEGIN MySQL get user data"); let result = async { - let tic = Instant::now(); - let mut conn = self.get_connection().await?; let statement_text = "SELECT `username`, `epoch`, `version`, `node_label_val`, `node_label_len`, `data` FROM `" @@ -1243,33 +963,10 @@ impl Storage for AsyncMySqlDatabase { .await .map(|a| a.into_iter().flatten().collect::>()); - let toc = Instant::now() - tic; - *(self.time_read.write().await) += toc; let selected_records = self.check_for_infra_error(out)?; - if let Some(cache) = &self.cache { - for record in selected_records.iter() { - cache.put(&DbRecord::ValueState(record.clone())).await; - } - } - if self.is_transaction_active().await { - let mut updated = vec![]; - for record in selected_records.into_iter() { - if let Some(DbRecord::ValueState(value)) = self - .trans - .get::(&record.get_id()) - .await - { - updated.push(value); - } else { - updated.push(record); - } - } - Ok::(KeyData { states: updated }) - } else { - Ok::(KeyData { - states: selected_records, - }) - } + Ok::(KeyData { + states: selected_records, + }) }; debug!("END MySQL get user data"); @@ -1287,14 +984,11 @@ impl Storage for AsyncMySqlDatabase { username: &AkdLabel, flag: ValueStateRetrievalFlag, ) -> core::result::Result { - *(self.num_reads.write().await) += 1; self.record_call_stats('r', "get_user_state".to_string(), "".to_string()) .await; debug!("BEGIN MySQL get user state (flag {:?})", flag); let result = async { - let tic = Instant::now(); - let mut conn = self.get_connection().await?; let mut statement_text = "SELECT `username`, `epoch`, `version`, `node_label_val`, `node_label_len`, `data` FROM `" @@ -1361,30 +1055,8 @@ impl Storage for AsyncMySqlDatabase { .await .map(|a| a.into_iter().flatten().collect::>()); - let toc = Instant::now() - tic; - *(self.time_read.write().await) += toc; let selected_record = self.check_for_infra_error(out)?; - let item = selected_record.into_iter().next(); - if let Some(value_in_item) = &item { - if let Some(cache) = &self.cache { - cache - .put(&DbRecord::ValueState(value_in_item.clone())) - .await; - } - } - // check the transaction log for an updated record - if self.is_transaction_active().await { - if let Some(found_item) = &item { - if let Some(DbRecord::ValueState(value)) = self - .trans - .get::(&found_item.get_id()) - .await - { - return Ok::, MySqlError>(Some(value)); - } - } - } Ok::, MySqlError>(item) }; debug!("END MySQL get user state"); @@ -1403,7 +1075,6 @@ impl Storage for AsyncMySqlDatabase { keys: &[AkdLabel], flag: ValueStateRetrievalFlag, ) -> core::result::Result, StorageError> { - *(self.num_reads.write().await) += 1; self.record_call_stats('r', "get_user_state_versions".to_string(), "".to_string()) .await; @@ -1411,8 +1082,6 @@ impl Storage for AsyncMySqlDatabase { debug!("BEGIN MySQL get user state versions (flag {:?})", flag); let result = async { - let tic = Instant::now(); - let mut conn = self.get_connection().await?; debug!("Creating the temporary search username's table"); @@ -1577,9 +1246,6 @@ impl Storage for AsyncMySqlDatabase { let nout = conn.query_drop("DROP TEMPORARY TABLE `search_users`").await; self.check_for_infra_error(nout)?; - let toc = Instant::now() - tic; - *(self.time_read.write().await) += toc; - for item in out.into_iter() { results.insert(item.0, item.1); } diff --git a/akd_mysql/src/mysql_db_tests.rs b/akd_mysql/src/mysql_db_tests.rs index b3420983..1876914a 100644 --- a/akd_mysql/src/mysql_db_tests.rs +++ b/akd_mysql/src/mysql_db_tests.rs @@ -36,7 +36,6 @@ async fn test_mysql_db() { Option::from("root"), Option::from("example"), Option::from(8001), - MySqlCacheOptions::None, 200, ) .await; diff --git a/akd_test_tools/src/fixture_generator/examples/example_tests.rs b/akd_test_tools/src/fixture_generator/examples/example_tests.rs index 2f0486fd..abdc5832 100644 --- a/akd_test_tools/src/fixture_generator/examples/example_tests.rs +++ b/akd_test_tools/src/fixture_generator/examples/example_tests.rs @@ -12,7 +12,7 @@ use std::fs::File; use akd::{ directory::Directory, ecvrf::HardCodedAkdVRF, - storage::{memory::AsyncInMemoryDatabase, Storage, StorageUtil}, + storage::{memory::AsyncInMemoryDatabase, Database, StorageManager, StorageUtil}, Blake3, }; @@ -34,7 +34,8 @@ async fn test_use_fixture() { let db = AsyncInMemoryDatabase::new(); db.batch_set(initial_state.records).await.unwrap(); let vrf = HardCodedAkdVRF {}; - let akd = Directory::<_, _, Blake3>::new(&db, &vrf, false) + let storage_manager = StorageManager::new_no_cache(&db); + let akd = Directory::<_, _, Blake3>::new(&storage_manager, &vrf, false) .await .unwrap(); diff --git a/akd_test_tools/src/fixture_generator/generator.rs b/akd_test_tools/src/fixture_generator/generator.rs index b83cfbc0..44eaa569 100644 --- a/akd_test_tools/src/fixture_generator/generator.rs +++ b/akd_test_tools/src/fixture_generator/generator.rs @@ -15,7 +15,7 @@ use std::io::Write; use akd::directory::Directory; use akd::storage::types::{AkdLabel, AkdValue, DbRecord}; -use akd::storage::StorageUtil; +use akd::storage::{StorageManager, StorageUtil}; use akd::Blake3; use clap::Parser; use rand::{rngs::OsRng, Rng}; @@ -121,7 +121,8 @@ pub(crate) async fn generate(args: Args) { // initialize directory let db = akd::storage::memory::AsyncInMemoryDatabase::new(); let vrf = akd::ecvrf::HardCodedAkdVRF {}; - let akd = Directory::<_, _, Blake3>::new(&db, &vrf, false) + let storage_manager = StorageManager::new_no_cache(&db); + let akd = Directory::<_, _, Blake3>::new(&storage_manager, &vrf, false) .await .unwrap(); diff --git a/akd_test_tools/src/test_suites.rs b/akd_test_tools/src/test_suites.rs index a05ab1ec..04daa6ea 100644 --- a/akd_test_tools/src/test_suites.rs +++ b/akd_test_tools/src/test_suites.rs @@ -18,8 +18,8 @@ use rand::{thread_rng, Rng}; /// The suite of tests to run against a fully-instantated and storage-backed directory. /// This will publish 3 epochs of ```num_users``` records and /// perform 10 random lookup proofs + 2 random history proofs + and audit proof from epochs 1u64 -> 2u64 -pub async fn directory_test_suite( - mysql_db: &S, +pub async fn directory_test_suite( + mysql_db: &akd::storage::StorageManager, num_users: usize, vrf: &V, ) { diff --git a/integration_tests/Cargo.toml b/integration_tests/Cargo.toml index ea852453..bb96c121 100644 --- a/integration_tests/Cargo.toml +++ b/integration_tests/Cargo.toml @@ -8,12 +8,11 @@ publish = false [features] high_parallelism = ["akd/high_parallelism"] memory_pressure = ["akd/memory_pressure"] -limited_memory_test = [] [dependencies] winter-crypto = "0.2" winter-math = "0.2" -akd = { path = "../akd", features = ["public-tests", "rand", "serde_serialization"] } +akd = { path = "../akd", features = ["public-tests", "rand", "serde_serialization", "runtime_metrics"] } [dev-dependencies] log = { version = "0.4.8", features = ["kv_unstable"] } @@ -26,5 +25,5 @@ thread-id = "3" multi_log = "0.1" hex = "0.4.3" -akd_mysql = { path = "../akd_mysql" } +akd_mysql = { path = "../akd_mysql", features = ["runtime_metrics"] } akd_test_tools = { path = "../akd_test_tools" } diff --git a/integration_tests/src/memory_tests.rs b/integration_tests/src/memory_tests.rs index 33143053..10ce3a16 100644 --- a/integration_tests/src/memory_tests.rs +++ b/integration_tests/src/memory_tests.rs @@ -5,7 +5,7 @@ // License, Version 2.0 found in the LICENSE-APACHE file in the root directory // of this source tree. -use akd::ecvrf::HardCodedAkdVRF; +use akd::{ecvrf::HardCodedAkdVRF, storage::StorageManager}; use log::info; type InMemoryDb = akd::storage::memory::AsyncInMemoryDatabase; @@ -20,7 +20,13 @@ async fn test_directory_operations() { let db = InMemoryDb::new(); let vrf = HardCodedAkdVRF {}; - akd_test_tools::test_suites::directory_test_suite::<_, HardCodedAkdVRF>(&db, 500, &vrf).await; + let storage_manager = StorageManager::new_no_cache(&db); + akd_test_tools::test_suites::directory_test_suite::<_, HardCodedAkdVRF>( + &storage_manager, + 500, + &vrf, + ) + .await; info!("\n\n******** Finished In-Memory Directory Operations Integration Test ********\n\n"); } diff --git a/integration_tests/src/mysql_tests.rs b/integration_tests/src/mysql_tests.rs index 108fe462..abd7265a 100644 --- a/integration_tests/src/mysql_tests.rs +++ b/integration_tests/src/mysql_tests.rs @@ -5,7 +5,8 @@ // License, Version 2.0 found in the LICENSE-APACHE file in the root directory // of this source tree. -use akd::{ecvrf::HardCodedAkdVRF, storage::Storage}; +use akd::ecvrf::HardCodedAkdVRF; +use akd::storage::StorageManager; use akd_mysql::mysql::*; use log::{error, info, warn}; @@ -36,11 +37,6 @@ async fn test_directory_operations() { Option::from("root"), Option::from("example"), Option::from(8001), - // clean every 1s and limit to 1KB - #[cfg(feature = "limited_memory_test")] - MySqlCacheOptions::Specific(std::time::Duration::from_secs(1), Some(1024)), - #[cfg(not(feature = "limited_memory_test"))] - MySqlCacheOptions::Default, 200, ) .await; @@ -51,12 +47,15 @@ async fn test_directory_operations() { } let vrf = HardCodedAkdVRF {}; + let storage_manager = StorageManager::new(&mysql_db, None, None); akd_test_tools::test_suites::directory_test_suite::<_, HardCodedAkdVRF>( - &mysql_db, 50, &vrf, + &storage_manager, + 50, + &vrf, ) .await; - mysql_db.log_metrics(log::Level::Trace).await; + storage_manager.log_metrics(log::Level::Trace).await; // clean the test infra if let Err(mysql_async::Error::Server(error)) = mysql_db.drop_tables().await { @@ -99,7 +98,6 @@ async fn test_lookups() { Option::from("root"), Option::from("example"), Option::from(8001), - MySqlCacheOptions::Default, 200, ) .await; @@ -110,7 +108,9 @@ async fn test_lookups() { } let vrf = HardCodedAkdVRF {}; - crate::test_util::test_lookups::<_, HardCodedAkdVRF>(&mysql_db, &vrf, 50, 5, 100).await; + let storage_manager = StorageManager::new(&mysql_db, None, None); + crate::test_util::test_lookups::<_, HardCodedAkdVRF>(&storage_manager, &vrf, 50, 5, 100) + .await; // clean the test infra if let Err(mysql_async::Error::Server(error)) = mysql_db.drop_tables().await { diff --git a/integration_tests/src/test_util.rs b/integration_tests/src/test_util.rs index 73184c82..c5c5ecae 100644 --- a/integration_tests/src/test_util.rs +++ b/integration_tests/src/test_util.rs @@ -9,6 +9,7 @@ extern crate thread_id; use akd::ecvrf::VRFKeyStorage; use akd::storage::types::{AkdLabel, AkdValue}; +use akd::storage::StorageManager; use akd::Directory; use log::{info, Level, Metadata, Record}; use once_cell::sync::OnceCell; @@ -129,10 +130,10 @@ impl log::Log for FileLogger { /// perform 10 random lookup proofs + 2 random history proofs + and audit proof from epochs 1u64 -> 2u64 #[allow(unused)] pub(crate) async fn directory_test_suite< - S: akd::storage::Storage + Sync + Send, + S: akd::storage::Database + Sync + Send, V: VRFKeyStorage, >( - mysql_db: &S, + mysql_db: &StorageManager, num_users: usize, vrf: &V, ) { @@ -253,8 +254,8 @@ pub(crate) async fn directory_test_suite< } } -pub(crate) async fn test_lookups( - mysql_db: &S, +pub(crate) async fn test_lookups( + mysql_db: &StorageManager, vrf: &V, num_users: u64, num_epochs: u64, @@ -370,7 +371,7 @@ pub(crate) async fn test_lookups(mysql_db: &S) { +async fn reset_mysql_db(mysql_db: &StorageManager) { mysql_db.log_metrics(Level::Trace).await; mysql_db.flush_cache().await; } diff --git a/poc/src/directory_host.rs b/poc/src/directory_host.rs index ebfbae15..fe8c3a58 100644 --- a/poc/src/directory_host.rs +++ b/poc/src/directory_host.rs @@ -8,7 +8,7 @@ use akd::ecvrf::VRFKeyStorage; use akd::errors::AkdError; use akd::storage::types::*; -use akd::storage::Storage; +use akd::storage::{Database, StorageManager}; use akd::{Directory, EpochHash}; use log::{debug, error, info}; use std::marker::{Send, Sync}; @@ -38,7 +38,7 @@ async fn get_root_hash( o_epoch: Option, ) -> Option> where - S: Storage + Sync + Send, + S: Database + Sync + Send, H: Hasher, V: VRFKeyStorage, { @@ -54,7 +54,7 @@ where pub(crate) async fn init_host(rx: &mut Receiver, directory: &mut Directory) where - S: Storage + Sync + Send, + S: Database + Sync + Send, H: Hasher, V: VRFKeyStorage, { diff --git a/poc/src/main.rs b/poc/src/main.rs index 4040c424..eb933485 100644 --- a/poc/src/main.rs +++ b/poc/src/main.rs @@ -9,10 +9,10 @@ // of this source tree. use akd::ecvrf::HardCodedAkdVRF; -use akd::storage::Storage; +use akd::storage::{Database, StorageManager}; use akd::Blake3; use akd::Directory; -use akd_mysql::mysql::{AsyncMySqlDatabase, MySqlCacheOptions}; +use akd_mysql::mysql::AsyncMySqlDatabase; use clap::{ArgEnum, Parser}; use commands::Command; use log::{debug, error, info, warn}; @@ -140,7 +140,8 @@ async fn main() { let vrf = HardCodedAkdVRF {}; if cli.memory_db { let db = akd::storage::memory::AsyncInMemoryDatabase::new(); - let mut directory = Directory::<_, _, Blake3>::new(&db, &vrf, false) + let storage_manager = StorageManager::new_no_cache(&db); + let mut directory = Directory::<_, _, Blake3>::new(&storage_manager, &vrf, false) .await .unwrap(); if let Some(()) = pre_process_input(&cli, &tx, None).await { @@ -158,20 +159,21 @@ async fn main() { Option::from("root"), Option::from("example"), Option::from(8001), - MySqlCacheOptions::Default, // enable caching cli.mysql_insert_depth, ) .await; if let Some(()) = pre_process_input(&cli, &tx, Some(&mysql_db)).await { return; } - let mut directory = Directory::<_, _, Blake3>::new(&mysql_db, &vrf, false) + let storage_manager = + StorageManager::new(&mysql_db, Some(Duration::from_secs(10 * 60)), None); + let mut directory = Directory::<_, _, Blake3>::new(&storage_manager, &vrf, false) .await .unwrap(); tokio::spawn(async move { directory_host::init_host::<_, Blake3, HardCodedAkdVRF>(&mut rx, &mut directory).await }); - process_input(&cli, &tx, Some(&mysql_db)).await; + process_input(&cli, &tx, Some(&storage_manager)).await; } } @@ -200,7 +202,7 @@ async fn pre_process_input( async fn process_input( cli: &Cli, tx: &Sender, - db: Option<&AsyncMySqlDatabase>, + db: Option<&StorageManager>, ) { if let Some(other_mode) = &cli.other_mode { match other_mode { @@ -399,7 +401,7 @@ async fn process_input( OtherMode::Flush => { println!("======= One-off flushing of the database ======= "); if let Some(mysql_db) = db { - if let Err(error) = mysql_db.delete_data().await { + if let Err(error) = mysql_db.db.delete_data().await { error!("Error flushing database: {}", error); } else { info!("Database flushed."); @@ -409,7 +411,7 @@ async fn process_input( OtherMode::Drop => { println!("======= Dropping database ======= "); if let Some(mysql_db) = db { - if let Err(error) = mysql_db.drop_tables().await { + if let Err(error) = mysql_db.db.drop_tables().await { error!("Error dropping database: {}", error); } else { info!("Database dropped."); @@ -443,7 +445,7 @@ async fn process_input( Command::Flush => { println!("Flushing the database..."); if let Some(mysql_db) = db { - if let Err(error) = mysql_db.delete_data().await { + if let Err(error) = mysql_db.db.delete_data().await { println!("Error flushing database: {}", error); } else { println!( @@ -459,7 +461,7 @@ async fn process_input( } println!("===== Auditable Key Directory Information ====="); if let Some(mysql) = db { - println!(" Database properties ({})", mysql); + println!(" Database properties ({})", mysql.db); } else { println!(" Connected to an in-memory database"); } From 1e255409d07edcd9550a6d4329bae730f5676901 Mon Sep 17 00:00:00 2001 From: Sean Lawlor Date: Wed, 23 Nov 2022 15:36:39 -0500 Subject: [PATCH 05/13] Removing features and moving to always enabled functionality --- .github/workflows/ci.yml | 6 + .github/workflows/mysql.yml | 6 - akd/Cargo.toml | 9 +- .../{caches => cache}/high_parallelism.rs | 44 +-- akd/src/storage/{caches => cache}/mod.rs | 18 +- akd/src/storage/cache/tests.rs | 130 ++++++++ akd/src/storage/caches/basic.rs | 285 ------------------ akd/src/storage/mod.rs | 2 +- akd/src/storage/storage.rs | 22 +- integration_tests/Cargo.toml | 2 - 10 files changed, 181 insertions(+), 343 deletions(-) rename akd/src/storage/{caches => cache}/high_parallelism.rs (90%) rename akd/src/storage/{caches => cache}/mod.rs (69%) create mode 100644 akd/src/storage/cache/tests.rs delete mode 100644 akd/src/storage/caches/basic.rs diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml index ce98e922..7e1b8ca0 100644 --- a/.github/workflows/ci.yml +++ b/.github/workflows/ci.yml @@ -36,6 +36,12 @@ jobs: command: test args: --package akd + - name: Test the base library, enabling runtime metrics processing + uses: actions-rs/cargo@v1 + with: + command: test + args: --package akd --features runtime_metrics + - name: Test the local auditor, with default features uses: actions-rs/cargo@v1 with: diff --git a/.github/workflows/mysql.yml b/.github/workflows/mysql.yml index 391a153e..1e70d6ba 100644 --- a/.github/workflows/mysql.yml +++ b/.github/workflows/mysql.yml @@ -62,12 +62,6 @@ jobs: command: test args: --manifest-path Cargo.toml -p akd_integration_tests - - name: Run integration tests with high-parallelism and memory pressure support - uses: actions-rs/cargo@v1 - with: - command: test - args: --manifest-path Cargo.toml -p akd_integration_tests --features high_parallelism,memory_pressure,limited_memory_test - - name: Cleanup docker container run: docker-compose -f docker-compose.yml down -v diff --git a/akd/Cargo.toml b/akd/Cargo.toml index c6271758..6d518c69 100644 --- a/akd/Cargo.toml +++ b/akd/Cargo.toml @@ -21,13 +21,6 @@ public-tests = ["rand", "bincode", "colored", "once_cell", "serde_serialization" public_auditing = ["protobuf", "thiserror"] default = [] serde_serialization = ["serde", "ed25519-dalek/serde"] -# Represents that we're going to support a higher-level of parallelism and therefore should avoid -# usages of a RwLock around a HashMap which doesn't scale well for parallel access (i.e. in the storage -# caches) -high_parallelism = ["dashmap"] -# Add support for handling memory pressure process on the caches. This costs some performance, however -# provides the ability to protect against OOMing -memory_pressure = [] # Collect runtime metrics on db access calls + timing runtime_metrics = [] @@ -35,6 +28,7 @@ runtime_metrics = [] ## Required dependencies ## hex = "0.4" async-trait = "0.1" +dashmap = { version = "5" } tokio = { version = "1.10", features = ["full"] } async-recursion = "0.3" log = { version = "0.4.8", features = ["kv_unstable"] } @@ -44,7 +38,6 @@ winter-math = "0.2" keyed_priority_queue = "0.3" ## Optional Dependencies ## -dashmap = { version = "5", optional = true } bincode = { version = "1", optional = true } serde = { version = "1", features = ["derive"], optional = true } rand = { version = "0.7", optional = true } diff --git a/akd/src/storage/caches/high_parallelism.rs b/akd/src/storage/cache/high_parallelism.rs similarity index 90% rename from akd/src/storage/caches/high_parallelism.rs rename to akd/src/storage/cache/high_parallelism.rs index 5bed6ff2..8abc8d94 100644 --- a/akd/src/storage/caches/high_parallelism.rs +++ b/akd/src/storage/cache/high_parallelism.rs @@ -9,19 +9,17 @@ //! objects use super::{ - CachedItem, CACHE_CLEAN_FREQUENCY_MS, DEFAULT_ITEM_LIFETIME_MS, DEFAULT_MEMORY_LIMIT_BYTES, + CachedItem, CACHE_CLEAN_FREQUENCY_MS, DEFAULT_ITEM_LIFETIME_MS, }; use crate::storage::DbRecord; -#[cfg(feature = "memory_pressure")] use crate::storage::SizeOf; use crate::storage::Storable; use dashmap::DashMap; #[cfg(not(feature = "runtime_metrics"))] use log::debug; -#[cfg(all(feature = "memory_pressure", not(feature = "runtime_metrics")))] use log::info; #[cfg(feature = "runtime_metrics")] -use log::{debug, error, info, warn}; +use log::{debug, error, warn}; use std::sync::atomic::{AtomicBool, AtomicU64, Ordering}; use std::sync::Arc; @@ -35,7 +33,7 @@ pub struct TimedCache { last_clean: Arc>, can_clean: Arc, item_lifetime: Duration, - memory_limit_bytes: usize, + memory_limit_bytes: Option, hit_count: Arc, } @@ -78,8 +76,7 @@ impl Clone for TimedCache { impl TimedCache { async fn clean(&self) { - let can_clean = self.can_clean.load(Ordering::Relaxed); - if !can_clean { + if !self.can_clean.load(Ordering::Relaxed) { // cleaning is disabled return; } @@ -94,10 +91,7 @@ impl TimedCache { debug!("BEGIN clean cache"); let now = Instant::now(); - #[cfg(not(feature = "memory_pressure"))] - self.map.retain(|_, v| v.expiration >= now); - #[cfg(feature = "memory_pressure")] - { + if let Some(memory_limit_bytes) = self.memory_limit_bytes { let mut retained_size = 0; let mut retained_elements = 0f64; let mut removed = 0; @@ -113,12 +107,12 @@ impl TimedCache { }); info!("Removed {} expired elements from the cache", removed); debug!("Retained cache size is {} bytes", retained_size); - if retained_size > self.memory_limit_bytes { + if retained_size > memory_limit_bytes { debug!("BEGIN cache memory pressure clean"); info!("Retained cache size has exceeded the predefined limit, cleaning old entries"); // calculate the percentage we'd need to trim off to get to 100% utilization and take another 5% let percent_clean = - 0.05 + 1.0 - (self.memory_limit_bytes as f64) / (retained_size as f64); + 0.05 + 1.0 - (memory_limit_bytes as f64) / (retained_size as f64); // convert that to the number of items to delete based on the size of the dictionary let num_clean = (retained_elements * percent_clean).round() as usize; // sort the dict based on the oldest entries @@ -138,6 +132,8 @@ impl TimedCache { } debug!("END cache memory pressure clean") } + } else { + self.map.retain(|_, v| v.expiration >= now); } debug!("END clean cache"); @@ -157,20 +153,20 @@ impl TimedCache { } /// Create a new timed cache instance. You can supply an optional item lifetime parameter - /// or take the default (30s) + /// or take the default (30s) and an optional memory-pressure limit, where the cache will be + /// cleaned if too much memory is being utilized pub fn new(o_lifetime: Option, o_memory_limit_bytes: Option) -> Self { let lifetime = match o_lifetime { - Some(life) if life > Duration::from_secs(1) => life, + Some(life) if life > Duration::from_millis(1) => life, _ => Duration::from_millis(DEFAULT_ITEM_LIFETIME_MS), }; - let memory_limit_bytes: usize = o_memory_limit_bytes.unwrap_or(DEFAULT_MEMORY_LIMIT_BYTES); Self { azks: Arc::new(tokio::sync::RwLock::new(None)), map: Arc::new(DashMap::new()), last_clean: Arc::new(tokio::sync::RwLock::new(Instant::now())), can_clean: Arc::new(AtomicBool::new(true)), item_lifetime: lifetime, - memory_limit_bytes, + memory_limit_bytes: o_memory_limit_bytes, hit_count: Arc::new(AtomicU64::new(0u64)), } } @@ -271,6 +267,20 @@ impl TimedCache { debug!("END cache flush"); } + /// Retrieve all of the cached items + pub async fn get_all(&self) -> Vec { + self.clean().await; + + let mut items = vec![]; + if let Some(record) = self.azks.read().await.clone() { + items.push(record); + } + for kv in self.map.iter() { + items.push(kv.value().data.clone()); + } + items + } + /// Disable cache-cleaning (i.e. during a transaction) pub fn disable_clean(&self) { debug!("Disabling MySQL object cache cleaning"); diff --git a/akd/src/storage/caches/mod.rs b/akd/src/storage/cache/mod.rs similarity index 69% rename from akd/src/storage/caches/mod.rs rename to akd/src/storage/cache/mod.rs index 4b74ea85..84cd4768 100644 --- a/akd/src/storage/caches/mod.rs +++ b/akd/src/storage/cache/mod.rs @@ -5,18 +5,22 @@ // License, Version 2.0 found in the LICENSE-APACHE file in the root directory // of this source tree. -//! This module handles various types of caches supported in the AKD crate which are -//! helpful for caching storage results for faster re-access +//! This module handles the caching implementation and testing for a time-based cache +//! which supports memory pressure shedding use crate::storage::DbRecord; use std::time::Instant; +#[cfg(test)] +mod tests; + /// item's live for 30s pub(crate) const DEFAULT_ITEM_LIFETIME_MS: u64 = 30000; /// clean the cache every 15s +#[cfg(not(test))] pub(crate) const CACHE_CLEAN_FREQUENCY_MS: u64 = 15000; -/// Default memory limit in bytes ~ 1GB -pub(crate) const DEFAULT_MEMORY_LIMIT_BYTES: usize = 1024 * 1024 * 1024; +#[cfg(test)] +pub(crate) const CACHE_CLEAN_FREQUENCY_MS: u64 = 50; pub(crate) struct CachedItem { pub(crate) expiration: Instant, @@ -33,14 +37,8 @@ impl super::SizeOf for CachedItem { // -------- sub modules -------- // -#[cfg(not(feature = "high_parallelism"))] -pub mod basic; -#[cfg(feature = "high_parallelism")] pub mod high_parallelism; // -------- cache exports -------- // -#[cfg(not(feature = "high_parallelism"))] -pub use basic::TimedCache; -#[cfg(feature = "high_parallelism")] pub use high_parallelism::TimedCache; diff --git a/akd/src/storage/cache/tests.rs b/akd/src/storage/cache/tests.rs new file mode 100644 index 00000000..59d78c03 --- /dev/null +++ b/akd/src/storage/cache/tests.rs @@ -0,0 +1,130 @@ +// Copyright (c) Meta Platforms, Inc. and affiliates. +// +// This source code is licensed under both the MIT license found in the +// LICENSE-MIT file in the root directory of this source tree and the Apache +// License, Version 2.0 found in the LICENSE-APACHE file in the root directory +// of this source tree. + +//! Caching tests + +use super::*; +use std::time::Duration; + +use crate::storage::DbRecord; +use crate::{NodeLabel, AkdValue, AkdLabel}; +use crate::storage::types::{ValueState, ValueStateKey}; + +#[tokio::test] +async fn test_cache_put_and_expires() { + let cache = TimedCache::new(Some(Duration::from_millis(10)), None); + + let value_state = DbRecord::ValueState(ValueState { + epoch: 1, + version: 1, + label: NodeLabel { + label_len: 1, + label_val: [0u8; 32], + }, + plaintext_val: AkdValue::from_utf8_str("some value"), + username: AkdLabel::from_utf8_str("user"), + }); + let key = ValueStateKey(AkdLabel::from_utf8_str("user").0.to_vec(), 1); + cache.put(&value_state).await; + + let got = cache.hit_test::(&key).await; + assert!(got.is_some()); + assert_eq!(Some(value_state), got); + + tokio::time::sleep(tokio::time::Duration::from_millis(100)).await; + let got = cache.hit_test::(&key).await; + assert_eq!(None, got); +} + +#[tokio::test] +async fn test_cache_overwrite() { + let cache = TimedCache::new(Some(Duration::from_millis(1000)), None); + + let value_state = ValueState { + epoch: 1, + version: 1, + label: NodeLabel { + label_len: 1, + label_val: [0u8; 32], + }, + plaintext_val: AkdValue::from_utf8_str("some value"), + username: AkdLabel::from_utf8_str("user"), + }; + let key = ValueStateKey(AkdLabel::from_utf8_str("user").0.to_vec(), 1); + + let value_state_2 = ValueState { + epoch: 1, + version: 2, + label: NodeLabel { + label_len: 2, + label_val: [0u8; 32], + }, + plaintext_val: AkdValue::from_utf8_str("some value"), + username: AkdLabel::from_utf8_str("user"), + }; + cache.put(&DbRecord::ValueState(value_state)).await; + cache.put(&DbRecord::ValueState(value_state_2.clone())).await; + + let got = cache.hit_test::(&key).await; + assert_eq!(Some(DbRecord::ValueState(value_state_2)), got); +} + + +#[tokio::test] +async fn test_cache_memory_pressure() { + let cache = TimedCache::new(Some(Duration::from_millis(1000)), Some(10)); + + let value_state = DbRecord::ValueState(ValueState { + epoch: 1, + version: 1, + label: NodeLabel { + label_len: 1, + label_val: [0u8; 32], + }, + plaintext_val: AkdValue::from_utf8_str("some value"), + username: AkdLabel::from_utf8_str("user"), + }); + let key = ValueStateKey(AkdLabel::from_utf8_str("user").0.to_vec(), 1); + cache.put(&value_state).await; + + // we only do an "automated" clean every 50ms in test, which is when memory pressure is evaluated. + // 100ms will make sure the clean op will run on the next `hit_test` op + tokio::time::sleep(tokio::time::Duration::from_millis(100)).await; + // This get should return none, even though the cache expiration time is 1s. This is because + // we should exceed 10 bytes of storage utilization so the cache should clean the item. + let got = cache.hit_test::(&key).await; + assert_eq!(None, got); +} + +#[tokio::test] +async fn test_many_memory_pressure() { + let cache = TimedCache::new(Some(Duration::from_millis(1000)), Some(1024*5)); + + let value_states = (1..100).map(|i| { + ValueState { + epoch: i as u64, + version: i as u64, + label: NodeLabel { + label_len: 1, + label_val: [0u8; 32], + }, + plaintext_val: AkdValue::from_utf8_str("test"), + username: AkdLabel::from_utf8_str("user"), + } + }) + .map(|vs| DbRecord::ValueState(vs)) + .collect::>(); + + cache.batch_put(&value_states).await; + + // we only do an "automated" clean every 50ms in test, which is when memory pressure is evaluated. + // 100ms will make sure the clean op will run on the next `hit_test` op + tokio::time::sleep(tokio::time::Duration::from_millis(100)).await; + + let all = cache.get_all().await; + assert!(all.len() < 99); +} \ No newline at end of file diff --git a/akd/src/storage/caches/basic.rs b/akd/src/storage/caches/basic.rs deleted file mode 100644 index a4905731..00000000 --- a/akd/src/storage/caches/basic.rs +++ /dev/null @@ -1,285 +0,0 @@ -// Copyright (c) Meta Platforms, Inc. and affiliates. -// -// This source code is licensed under both the MIT license found in the -// LICENSE-MIT file in the root directory of this source tree and the Apache -// License, Version 2.0 found in the LICENSE-APACHE file in the root directory -// of this source tree. - -//! This module implements a basic async timed cache. It additionally counts some -//! metrics related to access counts which can be helpful for profiling/debugging - -use super::{ - CachedItem, CACHE_CLEAN_FREQUENCY_MS, DEFAULT_ITEM_LIFETIME_MS, DEFAULT_MEMORY_LIMIT_BYTES, -}; -use crate::storage::DbRecord; -#[cfg(feature = "memory_pressure")] -use crate::storage::SizeOf; -use crate::storage::Storable; -#[cfg(not(feature = "runtime_metrics"))] -use log::debug; -#[cfg(all(feature = "memory_pressure", not(feature = "runtime_metrics")))] -use log::info; -#[cfg(feature = "runtime_metrics")] -use log::{debug, error, info, warn}; -use std::collections::HashMap; -use std::sync::atomic::{AtomicBool, AtomicU64, Ordering}; -use std::sync::Arc; -use std::time::Duration; -use std::time::Instant; - -/// Implements a basic cahce with timing information which automatically flushes -/// expired entries and removes them -pub struct TimedCache { - azks: Arc>>, - map: Arc, CachedItem>>>, - last_clean: Arc>, - can_clean: Arc, - item_lifetime: Duration, - hit_count: Arc, - memory_limit_bytes: usize, -} - -impl TimedCache { - /// Log cache access metrics along with size information - pub async fn log_metrics(&self, _level: log::Level) { - #[cfg(feature = "runtime_metrics")] - { - let hit_count = self.hit_count.swap(0, Ordering::Relaxed); - - let guard = self.map.read().await; - let cache_size = (*guard).keys().len(); - drop(guard); - - let msg = format!( - "Cache hit since last: {}, cached size: {} items", - hit_count, cache_size - ); - match _level { - log::Level::Trace => println!("{}", msg), - log::Level::Debug => debug!("{}", msg), - log::Level::Info => info!("{}", msg), - log::Level::Warn => warn!("{}", msg), - _ => error!("{}", msg), - } - } - } -} - -impl Clone for TimedCache { - fn clone(&self) -> Self { - TimedCache { - azks: self.azks.clone(), - map: self.map.clone(), - last_clean: self.last_clean.clone(), - can_clean: self.can_clean.clone(), - item_lifetime: self.item_lifetime, - hit_count: self.hit_count.clone(), - memory_limit_bytes: self.memory_limit_bytes, - } - } -} - -impl TimedCache { - async fn clean(&self) { - let can_clean = self.can_clean.load(Ordering::Relaxed); - if !can_clean { - // cleaning is disabled - return; - } - - let do_clean = { - // we need the {} brackets in order to release the read lock, since we _may_ acquire a write lock shortly later - *(self.last_clean.read().await) + Duration::from_millis(CACHE_CLEAN_FREQUENCY_MS) - < Instant::now() - }; - if do_clean { - debug!("BEGIN clean cache"); - let mut write = self.map.write().await; - let now = Instant::now(); - #[cfg(not(feature = "memory_pressure"))] - write.retain(|_, v| v.expiration >= now); - #[cfg(feature = "memory_pressure")] - { - let mut retained_size = 0; - let mut retained_elements = 0f64; - write.retain(|k, v| { - if v.expiration >= now { - retained_size += k.len() + v.size_of(); - retained_elements += 1.0; - true - } else { - false - } - }); - debug!("Retained cache size is {} bytes", retained_size); - if retained_size > self.memory_limit_bytes { - debug!("BEGIN cache memory pressure clean"); - info!("Retained cache size has exceeded the predefined limit, cleaning old entries"); - // calculate the percentage we'd need to trim off to get to 100% utilization and take another 5% - let percent_clean = - 0.05 + 1.0 - (self.memory_limit_bytes as f64) / (retained_size as f64); - // convert that to the number of items to delete based on the size of the dictionary - let num_clean = (retained_elements * percent_clean).round() as usize; - // sort the dict based on the oldest entries - let mut keys_and_expiration = write - .iter() - .map(|(k, v)| (k.clone(), v.expiration)) - .collect::>(); - keys_and_expiration.sort_by(|(_, a), (_, b)| a.cmp(b)); - // take those old entries, and remove them - for key in keys_and_expiration - .into_iter() - .take(num_clean) - .map(|(k, _)| k) - { - write.remove(&key); - } - debug!("END cache memory pressure clean") - } - } - debug!("END clean cache"); - - // update last clean time - *(self.last_clean.write().await) = Instant::now(); - } - } - - #[cfg(feature = "memory_pressure")] - /// Measure the size of the underlying hashmap and storage utilized - pub async fn measure(&self) -> usize { - let read = self.map.read().await; - read.iter() - .map(|(key, item)| key.len() + item.size_of()) - .sum() - } - - /// Create a new timed cache instance. You can supply an optional item lifetime parameter - /// or take the default (30s) - pub fn new(o_lifetime: Option, o_memory_limit_bytes: Option) -> Self { - let lifetime = match o_lifetime { - Some(life) if life > Duration::from_secs(1) => life, - _ => Duration::from_millis(DEFAULT_ITEM_LIFETIME_MS), - }; - let memory_limit_bytes: usize = o_memory_limit_bytes.unwrap_or(DEFAULT_MEMORY_LIMIT_BYTES); - Self { - azks: Arc::new(tokio::sync::RwLock::new(None)), - map: Arc::new(tokio::sync::RwLock::new(HashMap::new())), - last_clean: Arc::new(tokio::sync::RwLock::new(Instant::now())), - can_clean: Arc::new(AtomicBool::new(true)), - item_lifetime: lifetime, - hit_count: Arc::new(AtomicU64::new(0)), - memory_limit_bytes, - } - } - - /// Perform a hit-test of the cache for a given key. If successful, Some(record) will be returned - pub async fn hit_test(&self, key: &St::StorageKey) -> Option { - self.clean().await; - - debug!("BEGIN cache retrieve {:?}", key); - - let full_key = St::get_full_binary_key_id(key); - - // special case for AZKS - if full_key - == crate::append_only_zks::Azks::get_full_binary_key_id( - &crate::append_only_zks::DEFAULT_AZKS_KEY, - ) - { - // someone's requesting the AZKS object, return it from the special "cache" storage - let record = self.azks.read().await.clone(); - debug!("END cache retrieve"); - #[cfg(feature = "runtime_metrics")] - if record.is_some() { - self.hit_count.fetch_add(1, Ordering::Relaxed); - } - // AZKS objects cannot expire, they need to be manually flushed, so we don't need - // to check the expiration as below - return record; - } - - let guard = self.map.read().await; - let ptr: &HashMap<_, _> = &*guard; - debug!("END cache retrieve"); - if let Some(result) = ptr.get(&full_key) { - #[cfg(feature = "runtime_metrics")] - self.hit_count.fetch_add(1, Ordering::Relaxed); - - let ignore_clean = !self.can_clean.load(Ordering::Relaxed); - // if we've disabled cache cleaning, we're in the middle - // of an in-memory transaction and should ignore expiration - // of cache items until this flag is disabled again - if ignore_clean || result.expiration > Instant::now() { - return Some(result.data.clone()); - } - } - None - } - - /// Put an item into the cache - pub async fn put(&self, record: &DbRecord) { - self.clean().await; - - debug!("BEGIN cache put"); - let key = record.get_full_binary_id(); - - // special case for AZKS - if let DbRecord::Azks(azks_ref) = &record { - let mut guard = self.azks.write().await; - *guard = Some(DbRecord::Azks(azks_ref.clone())); - } else { - let mut guard = self.map.write().await; - // overwrite any existing items since a flush is requested - let item = CachedItem { - expiration: Instant::now() + self.item_lifetime, - data: record.clone(), - }; - (*guard).insert(key, item); - } - debug!("END cache put"); - } - - /// Put a batch of items into the cache, utilizing a single write lock - pub async fn batch_put(&self, records: &[DbRecord]) { - self.clean().await; - - debug!("BEGIN cache put batch"); - let mut guard = self.map.write().await; - for record in records.iter() { - if let DbRecord::Azks(azks_ref) = &record { - let mut azks_guard = self.azks.write().await; - *azks_guard = Some(DbRecord::Azks(azks_ref.clone())); - } else { - let key = record.get_full_binary_id(); - let item = CachedItem { - expiration: Instant::now() + self.item_lifetime, - data: record.clone(), - }; - (*guard).insert(key, item); - } - } - debug!("END cache put batch"); - } - - /// Flush the cache - pub async fn flush(&self) { - debug!("BEGIN cache flush"); - let mut guard = self.map.write().await; - (*guard).clear(); - let mut azks_guard = self.azks.write().await; - *azks_guard = None; - debug!("END cache flush"); - } - - /// Disable cache-cleaning (i.e. during a transaction) - pub fn disable_clean(&self) { - debug!("Disabling MySQL object cache cleaning"); - self.can_clean.store(false, Ordering::Relaxed); - } - - /// Re-enable cache cleaning (i.e. when a transaction is over) - pub fn enable_clean(&self) { - debug!("Enabling MySQL object cache cleaning"); - self.can_clean.store(true, Ordering::Relaxed); - } -} diff --git a/akd/src/storage/mod.rs b/akd/src/storage/mod.rs index 7128895d..41ebd845 100644 --- a/akd/src/storage/mod.rs +++ b/akd/src/storage/mod.rs @@ -17,7 +17,7 @@ use std::collections::HashMap; use std::hash::Hash; use std::marker::Send; -pub mod caches; +pub mod cache; pub mod transaction; pub mod types; diff --git a/akd/src/storage/storage.rs b/akd/src/storage/storage.rs index 430b3ec3..7d747423 100644 --- a/akd/src/storage/storage.rs +++ b/akd/src/storage/storage.rs @@ -9,7 +9,7 @@ //! to manage interactions with the data layer to optimize things like caching and //! transaction management -use crate::storage::caches::TimedCache; +use crate::storage::cache::TimedCache; use crate::storage::transaction::Transaction; use crate::storage::types::DbRecord; use crate::storage::types::KeyData; @@ -25,6 +25,7 @@ use log::{debug, error, info, warn}; use std::collections::HashMap; use std::collections::HashSet; use std::sync::Arc; +use std::sync::atomic::{AtomicU64, Ordering}; use std::time::Duration; use super::types::ValueStateRetrievalFlag; @@ -52,7 +53,7 @@ pub struct StorageManager { /// The underlying database managed by this storage manager pub db: Db, - metrics: Arc>, + metrics: [Arc; NUM_METRICS], } impl Clone for StorageManager { @@ -76,7 +77,7 @@ impl StorageManager { cache: None, transaction: Transaction::new(), db: db.clone(), - metrics: Arc::new(tokio::sync::RwLock::new([0; NUM_METRICS])), + metrics: [0; NUM_METRICS].map(|_| Arc::new(AtomicU64::new(0))), } } @@ -90,7 +91,7 @@ impl StorageManager { cache: Some(TimedCache::new(cache_item_lifetime, cache_limit_bytes)), transaction: Transaction::new(), db: db.clone(), - metrics: Arc::new(tokio::sync::RwLock::new([0; NUM_METRICS])), + metrics: [0; NUM_METRICS].map(|_| Arc::new(AtomicU64::new(0))), } } @@ -102,12 +103,7 @@ impl StorageManager { self.transaction.log_metrics(level).await; - let mut metric_guard = self.metrics.write().await; - let snapshot = metric_guard.clone(); - // clear the metrics - *metric_guard = [0; NUM_METRICS]; - // free the metric's guard to not block metric collection processing in parallel - drop(metric_guard); + let snapshot = self.metrics.iter().map(|metric| metric.load(Ordering::Relaxed)).collect::>(); let msg = format!( "=================================================== @@ -549,8 +545,7 @@ impl StorageManager { async fn increment_metric(&self, _metric: Metric) { #[cfg(feature = "runtime_metrics")] { - let mut guard = self.metrics.write().await; - (*guard)[_metric] = (*guard)[_metric] + 1; + (&self.metrics[_metric]).fetch_add(1, Ordering::Relaxed); } } @@ -561,8 +556,7 @@ impl StorageManager { let out = f.await; let delta = std::time::Instant::now().duration_since(tic); - let mut guard = self.metrics.write().await; - (*guard)[_metric] = (*guard)[_metric] + delta.as_millis(); + (&self.metrics[_metric]).fetch_add(delta.as_millis() as u64, Ordering::Relaxed); out } diff --git a/integration_tests/Cargo.toml b/integration_tests/Cargo.toml index bb96c121..a2a87d3d 100644 --- a/integration_tests/Cargo.toml +++ b/integration_tests/Cargo.toml @@ -6,8 +6,6 @@ edition = "2018" publish = false [features] -high_parallelism = ["akd/high_parallelism"] -memory_pressure = ["akd/memory_pressure"] [dependencies] winter-crypto = "0.2" From 1220401a047846c367e30ccfada5274db7e103bd Mon Sep 17 00:00:00 2001 From: Sean Lawlor Date: Wed, 23 Nov 2022 15:39:52 -0500 Subject: [PATCH 06/13] cargo fmt --- akd/src/storage/cache/high_parallelism.rs | 4 +--- akd/src/storage/cache/tests.rs | 24 +++++++++++------------ akd/src/storage/storage.rs | 8 ++++++-- 3 files changed, 19 insertions(+), 17 deletions(-) diff --git a/akd/src/storage/cache/high_parallelism.rs b/akd/src/storage/cache/high_parallelism.rs index 8abc8d94..2fe20a74 100644 --- a/akd/src/storage/cache/high_parallelism.rs +++ b/akd/src/storage/cache/high_parallelism.rs @@ -8,9 +8,7 @@ //! This module implements a higher-parallelism, async temporary cache for database //! objects -use super::{ - CachedItem, CACHE_CLEAN_FREQUENCY_MS, DEFAULT_ITEM_LIFETIME_MS, -}; +use super::{CachedItem, CACHE_CLEAN_FREQUENCY_MS, DEFAULT_ITEM_LIFETIME_MS}; use crate::storage::DbRecord; use crate::storage::SizeOf; use crate::storage::Storable; diff --git a/akd/src/storage/cache/tests.rs b/akd/src/storage/cache/tests.rs index 59d78c03..a483a5d5 100644 --- a/akd/src/storage/cache/tests.rs +++ b/akd/src/storage/cache/tests.rs @@ -10,9 +10,9 @@ use super::*; use std::time::Duration; -use crate::storage::DbRecord; -use crate::{NodeLabel, AkdValue, AkdLabel}; use crate::storage::types::{ValueState, ValueStateKey}; +use crate::storage::DbRecord; +use crate::{AkdLabel, AkdValue, NodeLabel}; #[tokio::test] async fn test_cache_put_and_expires() { @@ -67,13 +67,14 @@ async fn test_cache_overwrite() { username: AkdLabel::from_utf8_str("user"), }; cache.put(&DbRecord::ValueState(value_state)).await; - cache.put(&DbRecord::ValueState(value_state_2.clone())).await; + cache + .put(&DbRecord::ValueState(value_state_2.clone())) + .await; let got = cache.hit_test::(&key).await; assert_eq!(Some(DbRecord::ValueState(value_state_2)), got); } - #[tokio::test] async fn test_cache_memory_pressure() { let cache = TimedCache::new(Some(Duration::from_millis(1000)), Some(10)); @@ -102,10 +103,10 @@ async fn test_cache_memory_pressure() { #[tokio::test] async fn test_many_memory_pressure() { - let cache = TimedCache::new(Some(Duration::from_millis(1000)), Some(1024*5)); + let cache = TimedCache::new(Some(Duration::from_millis(1000)), Some(1024 * 5)); - let value_states = (1..100).map(|i| { - ValueState { + let value_states = (1..100) + .map(|i| ValueState { epoch: i as u64, version: i as u64, label: NodeLabel { @@ -114,10 +115,9 @@ async fn test_many_memory_pressure() { }, plaintext_val: AkdValue::from_utf8_str("test"), username: AkdLabel::from_utf8_str("user"), - } - }) - .map(|vs| DbRecord::ValueState(vs)) - .collect::>(); + }) + .map(|vs| DbRecord::ValueState(vs)) + .collect::>(); cache.batch_put(&value_states).await; @@ -127,4 +127,4 @@ async fn test_many_memory_pressure() { let all = cache.get_all().await; assert!(all.len() < 99); -} \ No newline at end of file +} diff --git a/akd/src/storage/storage.rs b/akd/src/storage/storage.rs index 7d747423..36a67361 100644 --- a/akd/src/storage/storage.rs +++ b/akd/src/storage/storage.rs @@ -24,8 +24,8 @@ use crate::AkdValue; use log::{debug, error, info, warn}; use std::collections::HashMap; use std::collections::HashSet; -use std::sync::Arc; use std::sync::atomic::{AtomicU64, Ordering}; +use std::sync::Arc; use std::time::Duration; use super::types::ValueStateRetrievalFlag; @@ -103,7 +103,11 @@ impl StorageManager { self.transaction.log_metrics(level).await; - let snapshot = self.metrics.iter().map(|metric| metric.load(Ordering::Relaxed)).collect::>(); + let snapshot = self + .metrics + .iter() + .map(|metric| metric.load(Ordering::Relaxed)) + .collect::>(); let msg = format!( "=================================================== From 89ddbc52f8d13c6d4ac29412190522be3ed8bf5e Mon Sep 17 00:00:00 2001 From: Sean Lawlor Date: Wed, 23 Nov 2022 15:45:33 -0500 Subject: [PATCH 07/13] Rename Storage trait to Database --- akd/src/append_only_zks.rs | 32 ++++++++-------- akd/src/auditor.rs | 2 +- akd/src/directory.rs | 16 ++++---- akd/src/serialization.rs | 2 +- akd/src/storage/{storage.rs => manager.rs} | 0 akd/src/storage/mod.rs | 4 +- akd/src/storage/tests.rs | 2 +- akd/src/tests.rs | 6 +-- akd/src/tree_node.rs | 44 +++++++++++----------- 9 files changed, 54 insertions(+), 54 deletions(-) rename akd/src/storage/{storage.rs => manager.rs} (100%) diff --git a/akd/src/append_only_zks.rs b/akd/src/append_only_zks.rs index c80a8d50..64949e54 100644 --- a/akd/src/append_only_zks.rs +++ b/akd/src/append_only_zks.rs @@ -7,11 +7,11 @@ //! An implementation of an append-only zero knowledge set use crate::serialization::to_digest; -use crate::storage::storage::StorageManager; +use crate::storage::manager::StorageManager; use crate::{ errors::TreeNodeError, proof_structs::{AppendOnlyProof, MembershipProof, NonMembershipProof, SingleAppendOnlyProof}, - storage::{Database as Storage, SizeOf, Storable}, + storage::{Database, SizeOf, Storable}, tree_node::*, }; @@ -86,7 +86,7 @@ impl Clone for Azks { impl Azks { /// Creates a new azks - pub async fn new( + pub async fn new( storage: &StorageManager, ) -> Result { create_empty_root::(storage, Option::Some(0), Option::Some(0)).await?; @@ -101,7 +101,7 @@ impl Azks { /// Inserts a single leaf and is only used for testing, since batching is more efficient. /// We just want to make sure batch insertions work correctly and this function is useful for that. #[cfg(test)] - pub async fn insert_leaf( + pub async fn insert_leaf( &mut self, storage: &StorageManager, node: Node, @@ -134,7 +134,7 @@ impl Azks { } /// Insert a batch of new leaves - pub async fn batch_insert_leaves( + pub async fn batch_insert_leaves( &mut self, storage: &StorageManager, insertion_set: Vec>, @@ -143,7 +143,7 @@ impl Azks { .await } - async fn preload_nodes_for_insertion( + async fn preload_nodes_for_insertion( &self, storage: &StorageManager, insertion_set: &[Node], @@ -160,7 +160,7 @@ impl Azks { } /// Preloads given nodes using breadth-first search. - pub async fn bfs_preload_nodes( + pub async fn bfs_preload_nodes( &self, storage: &StorageManager, nodes_to_load: HashSet, @@ -200,7 +200,7 @@ impl Azks { /// An azks is built both by the [crate::directory::Directory] and the auditor. /// However, both constructions have very minor differences, and the append_only_usage /// bool keeps track of this. - pub async fn batch_insert_leaves_helper( + pub async fn batch_insert_leaves_helper( &mut self, storage: &StorageManager, insertion_set: Vec>, @@ -284,7 +284,7 @@ impl Azks { /// Returns the Merkle membership proof for the trie as it stood at epoch // Assumes the verifier has access to the root at epoch - pub async fn get_membership_proof( + pub async fn get_membership_proof( &self, storage: &StorageManager, label: NodeLabel, @@ -301,7 +301,7 @@ impl Azks { /// In a compressed trie, the proof consists of the longest prefix /// of the label that is included in the trie, as well as its children, to show that /// none of the children is equal to the given label. - pub async fn get_non_membership_proof( + pub async fn get_non_membership_proof( &self, storage: &StorageManager, label: NodeLabel, @@ -361,7 +361,7 @@ impl Azks { /// **RESTRICTIONS**: Note that `start_epoch` and `end_epoch` are valid only when the following are true /// * `start_epoch` <= `end_epoch` /// * `start_epoch` and `end_epoch` are both existing epochs of this AZKS - pub async fn get_append_only_proof( + pub async fn get_append_only_proof( &self, storage: &StorageManager, start_epoch: u64, @@ -431,7 +431,7 @@ impl Azks { } } - async fn gather_audit_proof_nodes( + async fn gather_audit_proof_nodes( &self, nodes: Vec, storage: &StorageManager, @@ -463,7 +463,7 @@ impl Azks { } #[async_recursion] - async fn get_append_only_proof_helper( + async fn get_append_only_proof_helper( &self, storage: &StorageManager, node: TreeNode, @@ -527,7 +527,7 @@ impl Azks { // FIXME: these functions below should be moved into higher-level API /// Gets the root hash for this azks - pub async fn get_root_hash( + pub async fn get_root_hash( &self, storage: &StorageManager, ) -> Result { @@ -538,7 +538,7 @@ impl Azks { /// Gets the root hash of the tree at a epoch. /// Since this is accessing the root node and the root node exists at all epochs that /// the azks does, this would never be called at an epoch before the birth of the root node. - pub async fn get_root_hash_at_epoch( + pub async fn get_root_hash_at_epoch( &self, storage: &StorageManager, epoch: u64, @@ -573,7 +573,7 @@ impl Azks { /// This function returns the node label for the node whose label is the longest common /// prefix for the queried label. It also returns a membership proof for said label. /// This is meant to be used in both, getting membership proofs and getting non-membership proofs. - pub async fn get_membership_proof_and_node( + pub async fn get_membership_proof_and_node( &self, storage: &StorageManager, label: NodeLabel, diff --git a/akd/src/auditor.rs b/akd/src/auditor.rs index f303e0c1..fa72c102 100644 --- a/akd/src/auditor.rs +++ b/akd/src/auditor.rs @@ -14,7 +14,7 @@ use winter_crypto::Hasher; use crate::{ errors::{AkdError, AuditorError, AzksError}, proof_structs::{AppendOnlyProof, SingleAppendOnlyProof}, - storage::{memory::AsyncInMemoryDatabase, storage::StorageManager}, + storage::{manager::StorageManager, memory::AsyncInMemoryDatabase}, Azks, }; diff --git a/akd/src/directory.rs b/akd/src/directory.rs index 64dace03..c579c0f9 100644 --- a/akd/src/directory.rs +++ b/akd/src/directory.rs @@ -11,9 +11,9 @@ use crate::append_only_zks::Azks; use crate::ecvrf::{VRFKeyStorage, VRFPublicKey}; use crate::errors::{AkdError, DirectoryError, StorageError}; use crate::proof_structs::*; -use crate::storage::storage::StorageManager; +use crate::storage::manager::StorageManager; use crate::storage::types::{AkdLabel, AkdValue, DbRecord, ValueState, ValueStateRetrievalFlag}; -use crate::storage::Database as Storage; +use crate::storage::Database; use crate::{helper_structs::LookupInfo, EpochHash, Node}; use log::{debug, error, info}; @@ -43,7 +43,7 @@ impl AkdLabel { } /// The representation of a auditable key directory -pub struct Directory { +pub struct Directory { storage: StorageManager, vrf: V, hasher: PhantomData, @@ -58,7 +58,7 @@ pub struct Directory { } // Manual implementation of Clone, see: https://github.com/rust-lang/rust/issues/41481 -impl Clone for Directory { +impl Clone for Directory { fn clone(&self) -> Self { Self { storage: self.storage.clone(), @@ -70,7 +70,7 @@ impl Clone for Directory< } } -impl Directory { +impl Directory { /// Creates a new (stateless) instance of a auditable key directory. /// Takes as input a pointer to the storage being used for this instance. /// The state is stored in the storage. @@ -822,7 +822,7 @@ fn get_random_str(rng: &mut R) -> String { type KeyHistoryHelper = (Vec, Vec>); /// Gets hashes for key history proofs -pub async fn get_key_history_hashes( +pub async fn get_key_history_hashes( akd_dir: &Directory, history_proof: &HistoryProof, ) -> Result, AkdError> { @@ -861,7 +861,7 @@ pub async fn get_key_history_hashes( @@ -884,7 +884,7 @@ pub enum PublishCorruption { MarkVersionStale(AkdLabel, u64), } -impl Directory { +impl Directory { /// Updates the directory to include the updated key-value pairs with possible issues. pub async fn publish_malicious_update( &self, diff --git a/akd/src/serialization.rs b/akd/src/serialization.rs index bc092725..39cb4af7 100644 --- a/akd/src/serialization.rs +++ b/akd/src/serialization.rs @@ -79,8 +79,8 @@ mod tests { use crate::ecvrf::HardCodedAkdVRF; use crate::errors::AkdError; use crate::proof_structs::{AppendOnlyProof, HistoryProof, LookupProof}; + use crate::storage::manager::StorageManager; use crate::storage::memory::AsyncInMemoryDatabase; - use crate::storage::storage::StorageManager; use crate::storage::types::{AkdLabel, AkdValue}; use crate::Blake3; diff --git a/akd/src/storage/storage.rs b/akd/src/storage/manager.rs similarity index 100% rename from akd/src/storage/storage.rs rename to akd/src/storage/manager.rs diff --git a/akd/src/storage/mod.rs b/akd/src/storage/mod.rs index 41ebd845..ec946f3a 100644 --- a/akd/src/storage/mod.rs +++ b/akd/src/storage/mod.rs @@ -24,10 +24,10 @@ pub mod types; /* Various implementations supported by the library are imported here and usable at various checkpoints */ +pub mod manager; pub mod memory; -pub mod storage; -pub use storage::StorageManager; +pub use manager::StorageManager; #[cfg(any(test, feature = "public-tests"))] pub mod tests; diff --git a/akd/src/storage/tests.rs b/akd/src/storage/tests.rs index 2dea7c67..b207ddb4 100644 --- a/akd/src/storage/tests.rs +++ b/akd/src/storage/tests.rs @@ -303,7 +303,7 @@ async fn test_batch_get_items(storage: &Ns) { } async fn test_transactions(db: &S) { - let storage = crate::storage::storage::StorageManager::new_no_cache(db); + let storage = crate::storage::manager::StorageManager::new_no_cache(db); let mut rand_users: Vec> = vec![]; for _ in 0..20 { diff --git a/akd/src/tests.rs b/akd/src/tests.rs index c41548b5..4099bb4b 100644 --- a/akd/src/tests.rs +++ b/akd/src/tests.rs @@ -15,10 +15,10 @@ use crate::{ ecvrf::{HardCodedAkdVRF, VRFKeyStorage}, errors::AkdError, storage::{ + manager::StorageManager, memory::AsyncInMemoryDatabase, - storage::StorageManager, types::{AkdLabel, AkdValue, DbRecord}, - Database as Storage, + Database, }, }; use winter_crypto::{Digest, Hasher}; @@ -1093,7 +1093,7 @@ async fn test_simple_lookup_for_small_tree_sha256() -> Result<(), AkdError> { =========== Test Helpers =========== */ -async fn async_poll_helper_proof( +async fn async_poll_helper_proof( reader: &Directory, value: AkdValue, ) -> Result<(), AkdError> { diff --git a/akd/src/tree_node.rs b/akd/src/tree_node.rs index aacecc6b..ce34127b 100644 --- a/akd/src/tree_node.rs +++ b/akd/src/tree_node.rs @@ -11,9 +11,9 @@ use crate::errors::{AkdError, StorageError, TreeNodeError}; #[cfg(feature = "serde_serialization")] use crate::serialization::{bytes_deserialize_hex, bytes_serialize_hex}; use crate::serialization::{from_digest, to_digest}; -use crate::storage::storage::StorageManager; +use crate::storage::manager::StorageManager; use crate::storage::types::{DbRecord, StorageType}; -use crate::storage::{Database as Storage, Storable}; +use crate::storage::{Database, Storable}; use crate::{node_label::*, Direction, EMPTY_LABEL}; use async_recursion::async_recursion; use log::debug; @@ -170,14 +170,14 @@ impl TreeNodeWithPreviousValue { } } - pub(crate) async fn write_to_storage( + pub(crate) async fn write_to_storage( &self, storage: &StorageManager, ) -> Result<(), StorageError> { storage.set(DbRecord::TreeNode(self.clone())).await } - pub(crate) async fn get_appropriate_tree_node_from_storage( + pub(crate) async fn get_appropriate_tree_node_from_storage( storage: &StorageManager, key: &NodeKey, target_epoch: u64, @@ -191,7 +191,7 @@ impl TreeNodeWithPreviousValue { } } - pub(crate) async fn batch_get_appropriate_tree_node_from_storage( + pub(crate) async fn batch_get_appropriate_tree_node_from_storage( storage: &StorageManager, keys: &[NodeKey], target_epoch: u64, @@ -268,7 +268,7 @@ impl crate::storage::SizeOf for TreeNode { impl TreeNode { // Storage operations - pub(crate) async fn write_to_storage( + pub(crate) async fn write_to_storage( &self, storage: &StorageManager, ) -> Result<(), StorageError> { @@ -277,7 +277,7 @@ impl TreeNode { /// Internal function to be used for storage operations. If a node is new (i.e., is_new_node=true), the node's previous version /// will be used as None without the cost of finding this information in the cache or worse yet in the database. - async fn write_to_storage_impl( + async fn write_to_storage_impl( &self, storage: &StorageManager, is_new_node: bool, @@ -325,7 +325,7 @@ impl TreeNode { left_shifted.write_to_storage(storage).await } - pub(crate) async fn get_from_storage( + pub(crate) async fn get_from_storage( storage: &StorageManager, key: &NodeKey, target_epoch: u64, @@ -338,7 +338,7 @@ impl TreeNode { .await } - pub(crate) async fn batch_get_from_storage( + pub(crate) async fn batch_get_from_storage( storage: &StorageManager, keys: &[NodeKey], target_epoch: u64, @@ -381,7 +381,7 @@ impl TreeNode { // FIXME: Figure out how to better group arguments. #[allow(clippy::too_many_arguments)] /// Creates a new TreeNode and writes it to the storage. - async fn new( + async fn new( storage: &StorageManager, label: NodeLabel, parent: NodeLabel, @@ -410,7 +410,7 @@ impl TreeNode { /// This function is only used in testing, since in general, we want to update the hashes of nodes /// in a batch to prevent repeated work. #[cfg(test)] - pub(crate) async fn insert_single_leaf_and_hash( + pub(crate) async fn insert_single_leaf_and_hash( &mut self, storage: &StorageManager, new_leaf: Self, @@ -429,7 +429,7 @@ impl TreeNode { /// TreeNode is used but not the hash stored in updated parts of this Trei. /// This is used for batch inserting leaves, so that hashes can be updated /// in an amortized way, at a later time. - pub(crate) async fn insert_leaf( + pub(crate) async fn insert_leaf( &mut self, storage: &StorageManager, new_leaf: Self, @@ -448,7 +448,7 @@ impl TreeNode { /// This is used to both batch insert leaves in a Patricia Trei as well as /// for the single leaf insertions for testing. #[async_recursion] - pub(crate) async fn insert_single_leaf_helper( + pub(crate) async fn insert_single_leaf_helper( &mut self, storage: &StorageManager, new_leaf: Self, @@ -506,7 +506,7 @@ impl TreeNode { /// This handler is used to handle the case when the tree is just starting out and /// at least one of the root's (left or right) children is None. pub(crate) async fn insert_single_leaf_helper_root_handler< - S: Storage + Sync + Send, + S: Database + Sync + Send, H: Hasher, >( &mut self, @@ -548,7 +548,7 @@ impl TreeNode { #[async_recursion] #[allow(clippy::too_many_arguments)] pub(crate) async fn insert_single_leaf_helper_base_case_handler< - S: Storage + Sync + Send, + S: Database + Sync + Send, H: Hasher, >( &mut self, @@ -628,7 +628,7 @@ impl TreeNode { #[allow(clippy::too_many_arguments)] #[async_recursion] pub(crate) async fn insert_single_leaf_helper_recursive_case_handler< - S: Storage + Sync + Send, + S: Database + Sync + Send, H: Hasher, >( &mut self, @@ -674,7 +674,7 @@ impl TreeNode { } /// Updates the node hash and saves it in storage. - pub(crate) async fn update_node_hash( + pub(crate) async fn update_node_hash( &mut self, storage: &StorageManager, epoch: u64, @@ -714,7 +714,7 @@ impl TreeNode { /// Inserts a child into this node, adding the state to the state at this epoch, /// without updating its own hash. - pub(crate) async fn set_child( + pub(crate) async fn set_child( &mut self, storage: &StorageManager, child: &mut InsertionNode<'_>, @@ -793,7 +793,7 @@ impl TreeNode { ///// getrs for child nodes //// /// Loads (from storage) the left or right child of a node using given direction - pub(crate) async fn get_child_state( + pub(crate) async fn get_child_state( &self, storage: &StorageManager, direction: Direction, @@ -907,7 +907,7 @@ pub(crate) fn optional_child_state_hash( } /// Create an empty root node. -pub async fn create_empty_root( +pub async fn create_empty_root( storage: &StorageManager, ep: Option, least_descendant_ep: Option, @@ -937,7 +937,7 @@ pub async fn create_empty_root( } /// Create a specific leaf node. -pub async fn create_leaf_node( +pub async fn create_leaf_node( storage: &StorageManager, label: NodeLabel, value: &H::Digest, @@ -972,7 +972,7 @@ mod tests { type Blake3 = Blake3_256; type InMemoryDb = crate::storage::memory::AsyncInMemoryDatabase; - use crate::storage::storage::StorageManager; + use crate::storage::manager::StorageManager; #[tokio::test] async fn test_least_descendant_ep() -> Result<(), AkdError> { From de60184263c62e9b831213a7573428ea69ba10b9 Mon Sep 17 00:00:00 2001 From: Sean Lawlor Date: Wed, 23 Nov 2022 15:49:12 -0500 Subject: [PATCH 08/13] Fix type annotation in transaction.rs so clippy can run properly --- akd/src/storage/cache/tests.rs | 2 +- akd/src/storage/manager.rs | 10 +++++----- akd/src/storage/transaction.rs | 12 ++++++------ 3 files changed, 12 insertions(+), 12 deletions(-) diff --git a/akd/src/storage/cache/tests.rs b/akd/src/storage/cache/tests.rs index a483a5d5..1e799ce8 100644 --- a/akd/src/storage/cache/tests.rs +++ b/akd/src/storage/cache/tests.rs @@ -116,7 +116,7 @@ async fn test_many_memory_pressure() { plaintext_val: AkdValue::from_utf8_str("test"), username: AkdLabel::from_utf8_str("user"), }) - .map(|vs| DbRecord::ValueState(vs)) + .map(DbRecord::ValueState) .collect::>(); cache.batch_put(&value_states).await; diff --git a/akd/src/storage/manager.rs b/akd/src/storage/manager.rs index 36a67361..addbd0e5 100644 --- a/akd/src/storage/manager.rs +++ b/akd/src/storage/manager.rs @@ -237,7 +237,7 @@ impl StorageManager { // update the cache if let Some(cache) = &self.cache { - let _ = cache.batch_put(&records).await; + cache.batch_put(&records).await; } // Write to the database @@ -444,14 +444,14 @@ impl StorageManager { .map(|state| (state.epoch, state)) .collect::>() }) - .unwrap_or_else(|| HashMap::new()); + .unwrap_or_else(HashMap::new); let transaction_records = self .transaction .get_users_data(&[username.clone()]) .await .remove(username) - .unwrap_or_else(|| vec![]); + .unwrap_or_default(); for transaction_record in transaction_records.into_iter() { map.insert(transaction_record.epoch, transaction_record); } @@ -549,7 +549,7 @@ impl StorageManager { async fn increment_metric(&self, _metric: Metric) { #[cfg(feature = "runtime_metrics")] { - (&self.metrics[_metric]).fetch_add(1, Ordering::Relaxed); + self.metrics[_metric].fetch_add(1, Ordering::Relaxed); } } @@ -560,7 +560,7 @@ impl StorageManager { let out = f.await; let delta = std::time::Instant::now().duration_since(tic); - (&self.metrics[_metric]).fetch_add(delta.as_millis() as u64, Ordering::Relaxed); + self.metrics[_metric].fetch_add(delta.as_millis() as u64, Ordering::Relaxed); out } diff --git a/akd/src/storage/transaction.rs b/akd/src/storage/transaction.rs index 5e511da2..47e6878a 100644 --- a/akd/src/storage/transaction.rs +++ b/akd/src/storage/transaction.rs @@ -184,7 +184,7 @@ impl Transaction { usernames: &[crate::AkdLabel], ) -> HashMap> { debug!("BEGIN transaction user version scan"); - let mut results = HashMap::new(); + let mut results: HashMap> = HashMap::new(); let mut set = std::collections::HashSet::with_capacity(usernames.len()); for username in usernames.iter() { @@ -198,9 +198,9 @@ impl Transaction { if let DbRecord::ValueState(value_state) = record { if set.contains(&value_state.username) { if results.contains_key(&value_state.username) { - results - .get_mut(&value_state.username) - .map(|item: &mut Vec| item.push(value_state.clone())); + if let Some(item) = results.get_mut(&value_state.username) { + item.push(value_state.clone()) + } } else { results.insert(value_state.username.clone(), vec![value_state.clone()]); } @@ -229,7 +229,7 @@ impl Transaction { .get_users_data(&[username.clone()]) .await .remove(username) - .unwrap_or_else(|| vec![]); + .unwrap_or_default(); let out = Self::find_appropriate_item(intermediate, flag); #[cfg(feature = "runtime_metrics")] if out.is_some() { @@ -279,7 +279,7 @@ impl Transaction { .rev() .find(|item| item.epoch <= epoch), ValueStateRetrievalFlag::MaxEpoch => intermediate.into_iter().last(), - ValueStateRetrievalFlag::MinEpoch => intermediate.into_iter().nth(0), + ValueStateRetrievalFlag::MinEpoch => intermediate.into_iter().next(), } } } From e564fb3e888687938d455cb0b682798306705d47 Mon Sep 17 00:00:00 2001 From: Sean Lawlor Date: Wed, 23 Nov 2022 18:37:56 -0500 Subject: [PATCH 09/13] Small bug in setting cached values on a db batch set write --- akd/src/storage/manager.rs | 33 +++++++------- akd/src/storage/transaction.rs | 19 ++++++++ integration_tests/src/memory_tests.rs | 21 ++++++++- integration_tests/src/mysql_tests.rs | 63 ++++++++++++++++++++++++++- 4 files changed, 116 insertions(+), 20 deletions(-) diff --git a/akd/src/storage/manager.rs b/akd/src/storage/manager.rs index addbd0e5..2622506a 100644 --- a/akd/src/storage/manager.rs +++ b/akd/src/storage/manager.rs @@ -110,22 +110,23 @@ impl StorageManager { .collect::>(); let msg = format!( - "=================================================== + " +=================================================== ============ Database operation counts ============ =================================================== - SET {}, - BATCH SET {}, - GET {}, - BATCH GET {} - TOMBSTONE {} - GET USER STATE {} - GET USER DATA {} - GET USER STATE VERSIONS {} + SET {}, + BATCH SET {}, + GET {}, + BATCH GET {} + TOMBSTONE {} + GET USER STATE {} + GET USER DATA {} + GET USER STATE VERSIONS {} =================================================== ============ Database operation timing ============ =================================================== - TIME READ {} ms - TIME WRITE {} ms", + TIME READ {} ms + TIME WRITE {} ms", snapshot[METRIC_SET], snapshot[METRIC_BATCH_SET], snapshot[METRIC_GET], @@ -179,8 +180,7 @@ impl StorageManager { ))), }?; - self.tic_toc(METRIC_WRITE_TIME, self.db.batch_set(ops)) - .await?; + self.tic_toc(METRIC_WRITE_TIME, self.batch_set(ops)).await?; self.increment_metric(METRIC_BATCH_SET).await; Ok(()) } @@ -229,9 +229,7 @@ impl StorageManager { // we're in a transaction, set the items in the transaction if self.is_transaction_active().await { - for record in records.iter() { - self.transaction.set(record).await; - } + self.transaction.batch_set(&records).await; return Ok(()); } @@ -367,8 +365,7 @@ impl StorageManager { } if !new_data.is_empty() { debug!("Tombstoning {} entries", new_data.len()); - self.tic_toc(METRIC_WRITE_TIME, self.batch_set(new_data)) - .await?; + self.batch_set(new_data).await?; self.increment_metric(METRIC_TOMBSTONE).await; } diff --git a/akd/src/storage/transaction.rs b/akd/src/storage/transaction.rs index 47e6878a..a717f1ab 100644 --- a/akd/src/storage/transaction.rs +++ b/akd/src/storage/transaction.rs @@ -160,6 +160,25 @@ impl Transaction { out } + /// Set a batch of values into the cache + pub async fn batch_set(&self, records: &[DbRecord]) { + debug!("BEGIN transaction set"); + + let mut guard = self.state.write().await; + for record in records { + guard + .mods + .insert(record.get_full_binary_id(), record.clone()); + } + + #[cfg(feature = "runtime_metrics")] + { + *(self.num_writes.write().await) += 1; + } + + debug!("END transaction set"); + } + /// Set a value in the transaction to be committed at transaction commit time pub async fn set(&self, record: &DbRecord) { debug!("BEGIN transaction set"); diff --git a/integration_tests/src/memory_tests.rs b/integration_tests/src/memory_tests.rs index 10ce3a16..a9d69f26 100644 --- a/integration_tests/src/memory_tests.rs +++ b/integration_tests/src/memory_tests.rs @@ -11,7 +11,6 @@ use log::info; type InMemoryDb = akd::storage::memory::AsyncInMemoryDatabase; #[tokio::test] -#[serial_test::serial] async fn test_directory_operations() { crate::test_util::log_init(log::Level::Info); @@ -30,3 +29,23 @@ async fn test_directory_operations() { info!("\n\n******** Finished In-Memory Directory Operations Integration Test ********\n\n"); } + +#[tokio::test] +async fn test_directory_operations_with_caching() { + crate::test_util::log_init(log::Level::Info); + + info!("\n\n******** Starting In-Memory Directory Operations (w/caching) Integration Test ********\n\n"); + + let db = InMemoryDb::new(); + + let vrf = HardCodedAkdVRF {}; + let storage_manager = StorageManager::new(&db, None, None); + akd_test_tools::test_suites::directory_test_suite::<_, HardCodedAkdVRF>( + &storage_manager, + 500, + &vrf, + ) + .await; + + info!("\n\n******** Finished In-Memory Directory Operations (w/caching) Integration Test ********\n\n"); +} diff --git a/integration_tests/src/mysql_tests.rs b/integration_tests/src/mysql_tests.rs index abd7265a..c4038ea8 100644 --- a/integration_tests/src/mysql_tests.rs +++ b/integration_tests/src/mysql_tests.rs @@ -47,7 +47,7 @@ async fn test_directory_operations() { } let vrf = HardCodedAkdVRF {}; - let storage_manager = StorageManager::new(&mysql_db, None, None); + let storage_manager = StorageManager::new_no_cache(&mysql_db); akd_test_tools::test_suites::directory_test_suite::<_, HardCodedAkdVRF>( &storage_manager, 50, @@ -71,6 +71,67 @@ async fn test_directory_operations() { info!("\n\n******** Completed MySQL Directory Operations Integration Test ********\n\n"); } +#[tokio::test] +#[serial_test::serial] +async fn test_directory_operations_with_caching() { + crate::test_util::log_init(log::Level::Info); + + info!("\n\n******** Starting MySQL Directory Operations (w/caching) Integration Test ********\n\n"); + + if AsyncMySqlDatabase::test_guard() { + // create the "test" database + if let Err(error) = AsyncMySqlDatabase::create_test_db( + "localhost", + Option::from("root"), + Option::from("example"), + Option::from(8001), + ) + .await + { + panic!("Error creating test database: {}", error); + } + + // connect to the newly created test db + let mysql_db = AsyncMySqlDatabase::new( + "localhost", + "test_db", + Option::from("root"), + Option::from("example"), + Option::from(8001), + 200, + ) + .await; + + // delete all data from the db + if let Err(error) = mysql_db.delete_data().await { + error!("Error cleaning mysql prior to test suite: {}", error); + } + + let vrf = HardCodedAkdVRF {}; + let storage_manager = StorageManager::new(&mysql_db, None, None); + akd_test_tools::test_suites::directory_test_suite::<_, HardCodedAkdVRF>( + &storage_manager, + 50, + &vrf, + ) + .await; + + storage_manager.log_metrics(log::Level::Trace).await; + + // clean the test infra + if let Err(mysql_async::Error::Server(error)) = mysql_db.drop_tables().await { + error!( + "ERROR: Failed to clean MySQL test database with error {}", + error + ); + } + } else { + warn!("WARN: Skipping MySQL test due to test guard noting that the docker container appears to not be running."); + } + + info!("\n\n******** Completed MySQL Directory Operations (w/caching) Integration Test ********\n\n"); +} + #[tokio::test] #[serial_test::serial] async fn test_lookups() { From b8bffa6749a489f56396ec3c148858332a8e9d1e Mon Sep 17 00:00:00 2001 From: Sean Lawlor Date: Wed, 23 Nov 2022 18:58:51 -0500 Subject: [PATCH 10/13] removing old feature flag --- akd/src/append_only_zks.rs | 2 +- akd/src/storage/cache/high_parallelism.rs | 1 - 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/akd/src/append_only_zks.rs b/akd/src/append_only_zks.rs index ca582ac0..8bad9afa 100644 --- a/akd/src/append_only_zks.rs +++ b/akd/src/append_only_zks.rs @@ -20,7 +20,7 @@ use crate::{errors::*, node_label::*, tree_node::TreeNode, ARITY, *}; use async_recursion::async_recursion; use log::{debug, info}; use std::marker::{Send, Sync}; -use tokio::time::Instant; +use std::time::Instant; use winter_crypto::Hasher; use keyed_priority_queue::{Entry, KeyedPriorityQueue}; diff --git a/akd/src/storage/cache/high_parallelism.rs b/akd/src/storage/cache/high_parallelism.rs index 2fe20a74..2536a085 100644 --- a/akd/src/storage/cache/high_parallelism.rs +++ b/akd/src/storage/cache/high_parallelism.rs @@ -141,7 +141,6 @@ impl TimedCache { } } - #[cfg(feature = "memory_pressure")] /// Measure the size of the underlying hashmap and storage utilized pub fn measure(&self) -> usize { self.map From 4de32442702cc332ec7c13f888c681e880757d3d Mon Sep 17 00:00:00 2001 From: Sean Lawlor Date: Wed, 23 Nov 2022 19:02:22 -0500 Subject: [PATCH 11/13] Resolving #217 --- akd/src/tests.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/akd/src/tests.rs b/akd/src/tests.rs index d0f78d73..160b8159 100644 --- a/akd/src/tests.rs +++ b/akd/src/tests.rs @@ -899,7 +899,7 @@ async fn test_directory_read_only_mode() -> Result<(), AkdError> { #[tokio::test] async fn test_directory_polling_azks_change() -> Result<(), AkdError> { let db = AsyncInMemoryDatabase::new(); - let storage = StorageManager::new_no_cache(&db); + let storage = StorageManager::new(&db, None, None); let vrf = HardCodedAkdVRF {}; // writer will write the AZKS record let writer = Directory::<_, _, Blake3>::new(&storage, &vrf, false).await?; From bd21e9cb641368fcff7b0c917f4f47069b38d735 Mon Sep 17 00:00:00 2001 From: Sean Lawlor Date: Thu, 24 Nov 2022 08:52:01 -0500 Subject: [PATCH 12/13] Exposing cache clean frequency option --- akd/src/storage/cache/high_parallelism.rs | 18 ++++++++++++++---- akd/src/storage/cache/mod.rs | 9 +++------ akd/src/storage/cache/tests.rs | 20 ++++++++++++++++---- akd/src/storage/manager.rs | 7 ++++++- akd/src/tests.rs | 2 +- integration_tests/src/memory_tests.rs | 2 +- integration_tests/src/mysql_tests.rs | 4 ++-- poc/src/main.rs | 8 ++++++-- 8 files changed, 49 insertions(+), 21 deletions(-) diff --git a/akd/src/storage/cache/high_parallelism.rs b/akd/src/storage/cache/high_parallelism.rs index 2536a085..1806b5fe 100644 --- a/akd/src/storage/cache/high_parallelism.rs +++ b/akd/src/storage/cache/high_parallelism.rs @@ -8,7 +8,7 @@ //! This module implements a higher-parallelism, async temporary cache for database //! objects -use super::{CachedItem, CACHE_CLEAN_FREQUENCY_MS, DEFAULT_ITEM_LIFETIME_MS}; +use super::{CachedItem, DEFAULT_CACHE_CLEAN_FREQUENCY_MS, DEFAULT_ITEM_LIFETIME_MS}; use crate::storage::DbRecord; use crate::storage::SizeOf; use crate::storage::Storable; @@ -33,6 +33,7 @@ pub struct TimedCache { item_lifetime: Duration, memory_limit_bytes: Option, hit_count: Arc, + clean_frequency: Duration, } impl TimedCache { @@ -68,6 +69,7 @@ impl Clone for TimedCache { item_lifetime: self.item_lifetime, memory_limit_bytes: self.memory_limit_bytes, hit_count: self.hit_count.clone(), + clean_frequency: self.clean_frequency, } } } @@ -81,8 +83,7 @@ impl TimedCache { let do_clean = { // we need the {} brackets in order to release the read lock, since we _may_ acquire a write lock shortly later - *(self.last_clean.read().await) + Duration::from_millis(CACHE_CLEAN_FREQUENCY_MS) - < Instant::now() + *(self.last_clean.read().await) + self.clean_frequency < Instant::now() }; if do_clean { let mut last_clean_write = self.last_clean.write().await; @@ -152,11 +153,19 @@ impl TimedCache { /// Create a new timed cache instance. You can supply an optional item lifetime parameter /// or take the default (30s) and an optional memory-pressure limit, where the cache will be /// cleaned if too much memory is being utilized - pub fn new(o_lifetime: Option, o_memory_limit_bytes: Option) -> Self { + pub fn new( + o_lifetime: Option, + o_memory_limit_bytes: Option, + o_clean_frequency: Option, + ) -> Self { let lifetime = match o_lifetime { Some(life) if life > Duration::from_millis(1) => life, _ => Duration::from_millis(DEFAULT_ITEM_LIFETIME_MS), }; + let clean_frequency = match o_clean_frequency { + Some(frequency) if frequency > Duration::from_millis(1) => frequency, + _ => Duration::from_millis(DEFAULT_CACHE_CLEAN_FREQUENCY_MS), + }; Self { azks: Arc::new(tokio::sync::RwLock::new(None)), map: Arc::new(DashMap::new()), @@ -165,6 +174,7 @@ impl TimedCache { item_lifetime: lifetime, memory_limit_bytes: o_memory_limit_bytes, hit_count: Arc::new(AtomicU64::new(0u64)), + clean_frequency, } } diff --git a/akd/src/storage/cache/mod.rs b/akd/src/storage/cache/mod.rs index 84cd4768..b9554b0c 100644 --- a/akd/src/storage/cache/mod.rs +++ b/akd/src/storage/cache/mod.rs @@ -14,13 +14,10 @@ use std::time::Instant; #[cfg(test)] mod tests; -/// item's live for 30s +/// items live for 30s by default pub(crate) const DEFAULT_ITEM_LIFETIME_MS: u64 = 30000; -/// clean the cache every 15s -#[cfg(not(test))] -pub(crate) const CACHE_CLEAN_FREQUENCY_MS: u64 = 15000; -#[cfg(test)] -pub(crate) const CACHE_CLEAN_FREQUENCY_MS: u64 = 50; +/// clean the cache every 15s by default +pub(crate) const DEFAULT_CACHE_CLEAN_FREQUENCY_MS: u64 = 15000; pub(crate) struct CachedItem { pub(crate) expiration: Instant, diff --git a/akd/src/storage/cache/tests.rs b/akd/src/storage/cache/tests.rs index 1e799ce8..1117d027 100644 --- a/akd/src/storage/cache/tests.rs +++ b/akd/src/storage/cache/tests.rs @@ -16,7 +16,11 @@ use crate::{AkdLabel, AkdValue, NodeLabel}; #[tokio::test] async fn test_cache_put_and_expires() { - let cache = TimedCache::new(Some(Duration::from_millis(10)), None); + let cache = TimedCache::new( + Some(Duration::from_millis(10)), + None, + Some(Duration::from_millis(50)), + ); let value_state = DbRecord::ValueState(ValueState { epoch: 1, @@ -42,7 +46,7 @@ async fn test_cache_put_and_expires() { #[tokio::test] async fn test_cache_overwrite() { - let cache = TimedCache::new(Some(Duration::from_millis(1000)), None); + let cache = TimedCache::new(Some(Duration::from_millis(1000)), None, None); let value_state = ValueState { epoch: 1, @@ -77,7 +81,11 @@ async fn test_cache_overwrite() { #[tokio::test] async fn test_cache_memory_pressure() { - let cache = TimedCache::new(Some(Duration::from_millis(1000)), Some(10)); + let cache = TimedCache::new( + Some(Duration::from_millis(1000)), + Some(10), + Some(Duration::from_millis(50)), + ); let value_state = DbRecord::ValueState(ValueState { epoch: 1, @@ -103,7 +111,11 @@ async fn test_cache_memory_pressure() { #[tokio::test] async fn test_many_memory_pressure() { - let cache = TimedCache::new(Some(Duration::from_millis(1000)), Some(1024 * 5)); + let cache = TimedCache::new( + Some(Duration::from_millis(1000)), + Some(1024 * 5), + Some(Duration::from_millis(50)), + ); let value_states = (1..100) .map(|i| ValueState { diff --git a/akd/src/storage/manager.rs b/akd/src/storage/manager.rs index 2622506a..1ccdc9aa 100644 --- a/akd/src/storage/manager.rs +++ b/akd/src/storage/manager.rs @@ -86,9 +86,14 @@ impl StorageManager { db: &Db, cache_item_lifetime: Option, cache_limit_bytes: Option, + cache_clean_frequency: Option, ) -> Self { Self { - cache: Some(TimedCache::new(cache_item_lifetime, cache_limit_bytes)), + cache: Some(TimedCache::new( + cache_item_lifetime, + cache_limit_bytes, + cache_clean_frequency, + )), transaction: Transaction::new(), db: db.clone(), metrics: [0; NUM_METRICS].map(|_| Arc::new(AtomicU64::new(0))), diff --git a/akd/src/tests.rs b/akd/src/tests.rs index 160b8159..32b6caaf 100644 --- a/akd/src/tests.rs +++ b/akd/src/tests.rs @@ -899,7 +899,7 @@ async fn test_directory_read_only_mode() -> Result<(), AkdError> { #[tokio::test] async fn test_directory_polling_azks_change() -> Result<(), AkdError> { let db = AsyncInMemoryDatabase::new(); - let storage = StorageManager::new(&db, None, None); + let storage = StorageManager::new(&db, None, None, None); let vrf = HardCodedAkdVRF {}; // writer will write the AZKS record let writer = Directory::<_, _, Blake3>::new(&storage, &vrf, false).await?; diff --git a/integration_tests/src/memory_tests.rs b/integration_tests/src/memory_tests.rs index a9d69f26..35a15ed3 100644 --- a/integration_tests/src/memory_tests.rs +++ b/integration_tests/src/memory_tests.rs @@ -39,7 +39,7 @@ async fn test_directory_operations_with_caching() { let db = InMemoryDb::new(); let vrf = HardCodedAkdVRF {}; - let storage_manager = StorageManager::new(&db, None, None); + let storage_manager = StorageManager::new(&db, None, None, None); akd_test_tools::test_suites::directory_test_suite::<_, HardCodedAkdVRF>( &storage_manager, 500, diff --git a/integration_tests/src/mysql_tests.rs b/integration_tests/src/mysql_tests.rs index c4038ea8..3709022c 100644 --- a/integration_tests/src/mysql_tests.rs +++ b/integration_tests/src/mysql_tests.rs @@ -108,7 +108,7 @@ async fn test_directory_operations_with_caching() { } let vrf = HardCodedAkdVRF {}; - let storage_manager = StorageManager::new(&mysql_db, None, None); + let storage_manager = StorageManager::new(&mysql_db, None, None, None); akd_test_tools::test_suites::directory_test_suite::<_, HardCodedAkdVRF>( &storage_manager, 50, @@ -169,7 +169,7 @@ async fn test_lookups() { } let vrf = HardCodedAkdVRF {}; - let storage_manager = StorageManager::new(&mysql_db, None, None); + let storage_manager = StorageManager::new(&mysql_db, None, None, None); crate::test_util::test_lookups::<_, HardCodedAkdVRF>(&storage_manager, &vrf, 50, 5, 100) .await; diff --git a/poc/src/main.rs b/poc/src/main.rs index eb933485..248fced4 100644 --- a/poc/src/main.rs +++ b/poc/src/main.rs @@ -165,8 +165,12 @@ async fn main() { if let Some(()) = pre_process_input(&cli, &tx, Some(&mysql_db)).await { return; } - let storage_manager = - StorageManager::new(&mysql_db, Some(Duration::from_secs(10 * 60)), None); + let storage_manager = StorageManager::new( + &mysql_db, + Some(Duration::from_secs(10 * 60)), + None, + Some(Duration::from_secs(15)), + ); let mut directory = Directory::<_, _, Blake3>::new(&storage_manager, &vrf, false) .await .unwrap(); From de6dd2b311f3f68c01a293a6e88397a6c9490a84 Mon Sep 17 00:00:00 2001 From: Sean Lawlor Date: Thu, 24 Nov 2022 09:07:48 -0500 Subject: [PATCH 13/13] logical ordering of transaction v. cache mgmt --- akd/src/storage/cache/high_parallelism.rs | 44 +++++++++++++---------- akd/src/storage/manager.rs | 22 +++++++----- 2 files changed, 38 insertions(+), 28 deletions(-) diff --git a/akd/src/storage/cache/high_parallelism.rs b/akd/src/storage/cache/high_parallelism.rs index 1806b5fe..d8a0052d 100644 --- a/akd/src/storage/cache/high_parallelism.rs +++ b/akd/src/storage/cache/high_parallelism.rs @@ -92,28 +92,31 @@ impl TimedCache { let now = Instant::now(); if let Some(memory_limit_bytes) = self.memory_limit_bytes { let mut retained_size = 0; - let mut retained_elements = 0f64; - let mut removed = 0; + let mut num_retained = 0u32; + let mut num_removed = 0u32; self.map.retain(|k, v| { if v.expiration >= now { retained_size += k.len() + v.size_of(); - retained_elements += 1.0; + num_retained += 1; true } else { - removed += 1; + num_removed += 1; false } }); - info!("Removed {} expired elements from the cache", removed); + + info!("Removed {} expired elements from the cache", num_removed); debug!("Retained cache size is {} bytes", retained_size); + if retained_size > memory_limit_bytes { debug!("BEGIN cache memory pressure clean"); + info!("Retained cache size has exceeded the predefined limit, cleaning old entries"); // calculate the percentage we'd need to trim off to get to 100% utilization and take another 5% let percent_clean = 0.05 + 1.0 - (memory_limit_bytes as f64) / (retained_size as f64); // convert that to the number of items to delete based on the size of the dictionary - let num_clean = (retained_elements * percent_clean).round() as usize; + let num_clean = ((num_retained as f64) * percent_clean).ceil() as usize; // sort the dict based on the oldest entries let mut keys_and_expiration = self .map @@ -121,7 +124,7 @@ impl TimedCache { .map(|kv| (kv.key().clone(), kv.value().expiration)) .collect::>(); keys_and_expiration.sort_by(|(_, a), (_, b)| a.cmp(b)); - // take those old entries, and remove them + // take `num_clean` old entries and remove them for key in keys_and_expiration .into_iter() .take(num_clean) @@ -129,9 +132,11 @@ impl TimedCache { { self.map.remove(&key); } + debug!("END cache memory pressure clean") } } else { + // memory pressure analysis is disabled, simply utilize timed cache cleaning self.map.retain(|_, v| v.expiration >= now); } @@ -142,14 +147,6 @@ impl TimedCache { } } - /// Measure the size of the underlying hashmap and storage utilized - pub fn measure(&self) -> usize { - self.map - .iter() - .map(|kv| kv.key().len() + kv.value().size_of()) - .sum() - } - /// Create a new timed cache instance. You can supply an optional item lifetime parameter /// or take the default (30s) and an optional memory-pressure limit, where the cache will be /// cleaned if too much memory is being utilized @@ -218,6 +215,7 @@ impl TimedCache { return Some(result.data.clone()); } } + debug!("END cache retrieve"); None } @@ -227,6 +225,7 @@ impl TimedCache { self.clean().await; debug!("BEGIN cache put"); + let key = record.get_full_binary_id(); // special case for AZKS @@ -249,6 +248,7 @@ impl TimedCache { self.clean().await; debug!("BEGIN cache put batch"); + for record in records.iter() { if let DbRecord::Azks(azks_ref) = &record { let mut azks_guard = self.azks.write().await; @@ -262,20 +262,24 @@ impl TimedCache { self.map.insert(key, item); } } + debug!("END cache put batch"); } /// Flush the cache pub async fn flush(&self) { debug!("BEGIN cache flush"); + self.map.clear(); - let mut azks_guard = self.azks.write().await; - *azks_guard = None; + *(self.azks.write().await) = None; + debug!("END cache flush"); } /// Retrieve all of the cached items pub async fn get_all(&self) -> Vec { + debug!("BEGIN cache get all"); + self.clean().await; let mut items = vec![]; @@ -285,18 +289,20 @@ impl TimedCache { for kv in self.map.iter() { items.push(kv.value().data.clone()); } + + debug!("END cache get all"); items } /// Disable cache-cleaning (i.e. during a transaction) pub fn disable_clean(&self) { - debug!("Disabling MySQL object cache cleaning"); + debug!("Disabling cache cleaning"); self.can_clean.store(false, Ordering::Relaxed); } /// Re-enable cache cleaning (i.e. when a transaction is over) pub fn enable_clean(&self) { - debug!("Enabling MySQL object cache cleaning"); + debug!("Enabling cache cleaning"); self.can_clean.store(true, Ordering::Relaxed); } } diff --git a/akd/src/storage/manager.rs b/akd/src/storage/manager.rs index 1ccdc9aa..373db0f3 100644 --- a/akd/src/storage/manager.rs +++ b/akd/src/storage/manager.rs @@ -157,26 +157,28 @@ impl StorageManager { /// Start an in-memory transaction of changes pub async fn begin_transaction(&self) -> bool { + let started = self.transaction.begin_transaction().await; + // disable the cache cleaning since we're in a write transaction // and will want to keep cache'd objects for the life of the transaction if let Some(cache) = &self.cache { cache.disable_clean(); } - self.transaction.begin_transaction().await + started } /// Commit a transaction in the database pub async fn commit_transaction(&self) -> Result<(), StorageError> { + // this retrieves all the trans operations, and "de-activates" the transaction flag + let ops = self.transaction.commit_transaction().await?; + // The transaction is now complete (or reverted) and therefore we can re-enable // the cache cleaning status if let Some(cache) = &self.cache { cache.enable_clean(); } - // this retrieves all the trans operations, and "de-activates" the transaction flag - let ops = self.transaction.commit_transaction().await?; - let _epoch = match ops.last() { Some(DbRecord::Azks(azks)) => Ok(azks.latest_epoch), other => Err(StorageError::Transaction(format!( @@ -185,20 +187,22 @@ impl StorageManager { ))), }?; - self.tic_toc(METRIC_WRITE_TIME, self.batch_set(ops)).await?; - self.increment_metric(METRIC_BATCH_SET).await; - Ok(()) + // We're calling the internal batch set operation in order to manage the cache correctly. + // Since we've "committed" the in-memory transaction, the batch_set operation will bypass + // the transaction management and simply interact with the cache then write to the underlying + // DB + self.batch_set(ops).await } /// Rollback a transaction pub async fn rollback_transaction(&self) -> Result<(), StorageError> { + self.transaction.rollback_transaction().await?; // The transaction is being reverted and therefore we can re-enable // the cache cleaning status if let Some(cache) = &self.cache { cache.enable_clean(); } - - self.transaction.rollback_transaction().await + Ok(()) } /// Retrieve a flag determining if there is a transaction active