From 85502c9a340f2ea31deeba80bb63dde95990c72d Mon Sep 17 00:00:00 2001 From: James Kay Date: Tue, 2 Apr 2024 17:49:13 +0100 Subject: [PATCH] `linera-views`: add local variants of store traits (#1843) --- Cargo.lock | 6 +- examples/Cargo.lock | 2 +- linera-sdk/src/views/system_api.rs | 4 - linera-storage-service/Cargo.toml | 1 - linera-storage-service/src/client.rs | 7 -- linera-views/Cargo.toml | 1 + linera-views/src/batch.rs | 4 +- linera-views/src/common.rs | 90 +++++++++----- linera-views/src/dynamo_db.rs | 12 +- linera-views/src/journaling.rs | 4 - linera-views/src/key_value_store_view.rs | 3 +- linera-views/src/lru_caching.rs | 146 +++++++++++------------ linera-views/src/memory.rs | 5 - linera-views/src/metering.rs | 3 - linera-views/src/rocks_db.rs | 8 -- linera-views/src/scylla_db.rs | 12 +- linera-views/src/test_utils/mod.rs | 22 ++-- linera-views/src/value_splitting.rs | 8 -- 18 files changed, 154 insertions(+), 184 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index b786dbe9e00..237cdeef4bd 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -3513,7 +3513,6 @@ version = "0.11.0" dependencies = [ "anyhow", "async-lock", - "async-trait", "bcs", "cfg_aliases", "clap", @@ -3586,6 +3585,7 @@ dependencies = [ "tokio", "tokio-test", "tracing", + "trait-variant", ] [[package]] @@ -7142,9 +7142,9 @@ dependencies = [ [[package]] name = "web-sys" -version = "0.3.68" +version = "0.3.69" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "96565907687f7aceb35bc5fc03770a8a0471d82e479f25832f54a0e3f4b28446" +checksum = "77afa9a11836342370f4817622a2f0f418b134426d91a82dfb48f532d2ec13ef" dependencies = [ "js-sys", "wasm-bindgen", diff --git a/examples/Cargo.lock b/examples/Cargo.lock index 7d73aacb44c..46f2962d628 100644 --- a/examples/Cargo.lock +++ b/examples/Cargo.lock @@ -1998,7 +1998,6 @@ version = "0.11.0" dependencies = [ "anyhow", "async-lock", - "async-trait", "bcs", "cfg_aliases", "clap", @@ -2057,6 +2056,7 @@ dependencies = [ "thiserror", "tokio", "tracing", + "trait-variant", ] [[package]] diff --git a/linera-sdk/src/views/system_api.rs b/linera-sdk/src/views/system_api.rs index d3f3bc1f2b7..5708b037474 100644 --- a/linera-sdk/src/views/system_api.rs +++ b/linera-sdk/src/views/system_api.rs @@ -3,7 +3,6 @@ //! Functions and types to interface with the system API available to application views. -use async_trait::async_trait; use linera_base::ensure; use linera_views::{ batch::{Batch, WriteOperation}, @@ -40,7 +39,6 @@ impl AppStateStore { } } -#[async_trait] impl ReadableKeyValueStore for AppStateStore { // The AppStateStore of the system_api does not have limits // on the size of its values. @@ -101,7 +99,6 @@ impl ReadableKeyValueStore for AppStateStore { } } -#[async_trait] impl WritableKeyValueStore for AppStateStore { const MAX_VALUE_SIZE: usize = usize::MAX; @@ -135,7 +132,6 @@ impl WritableKeyValueStore for AppStateStore { } } -#[async_trait] impl KeyValueStore for AppStateStore { type Error = ViewError; } diff --git a/linera-storage-service/Cargo.toml b/linera-storage-service/Cargo.toml index 3d8a15eb3f9..792cbc6b00c 100644 --- a/linera-storage-service/Cargo.toml +++ b/linera-storage-service/Cargo.toml @@ -24,7 +24,6 @@ path = "src/server.rs" [dependencies] anyhow.workspace = true async-lock.workspace = true -async-trait.workspace = true bcs.workspace = true clap.workspace = true linera-base.workspace = true diff --git a/linera-storage-service/src/client.rs b/linera-storage-service/src/client.rs index f1f9d0608dd..65a6351d4d2 100644 --- a/linera-storage-service/src/client.rs +++ b/linera-storage-service/src/client.rs @@ -4,7 +4,6 @@ use std::{mem, sync::Arc}; use async_lock::{RwLock, RwLockWriteGuard, Semaphore, SemaphoreGuard}; -use async_trait::async_trait; use linera_base::ensure; #[cfg(with_testing)] use linera_views::test_utils::generate_test_namespace; @@ -58,7 +57,6 @@ pub struct ServiceStoreClient { namespace: Vec, } -#[async_trait] impl ReadableKeyValueStore for ServiceStoreClient { const MAX_KEY_SIZE: usize = MAX_KEY_SIZE; type Keys = Vec>; @@ -199,14 +197,10 @@ impl ReadableKeyValueStore for ServiceStoreClient { } } -#[async_trait] impl WritableKeyValueStore for ServiceStoreClient { const MAX_VALUE_SIZE: usize = usize::MAX; async fn write_batch(&self, batch: Batch, _base_key: &[u8]) -> Result<(), ServiceContextError> { - use linera_views::batch::WriteOperation; - - use crate::client::Operation; let mut statements = Vec::new(); let mut chunk_size = 0; for operation in batch.operations { @@ -345,7 +339,6 @@ impl ServiceStoreClient { } } -#[async_trait] impl AdminKeyValueStore for ServiceStoreClient { type Error = ServiceContextError; type Config = ServiceStoreConfig; diff --git a/linera-views/Cargo.toml b/linera-views/Cargo.toml index 76698154f27..18fa0cce37c 100644 --- a/linera-views/Cargo.toml +++ b/linera-views/Cargo.toml @@ -52,6 +52,7 @@ tempfile.workspace = true thiserror.workspace = true tokio = { workspace = true, features = ["rt", "sync"] } tracing.workspace = true +trait-variant.workspace = true [dev-dependencies] linera-views = { path = ".", features = ["test"] } diff --git a/linera-views/src/batch.rs b/linera-views/src/batch.rs index fbee52c993a..eef5a0470eb 100644 --- a/linera-views/src/batch.rs +++ b/linera-views/src/batch.rs @@ -348,8 +348,8 @@ impl Batch { /// Certain databases (e.g. DynamoDB) do not support the deletion by prefix. /// Thus we need to access the databases in order to replace a `DeletePrefix` /// by a vector of the keys to be removed. -#[async_trait] -pub trait DeletePrefixExpander { +#[trait_variant::make(DeletePrefixExpander: Send)] +pub trait LocalDeletePrefixExpander { /// The error type that can happen when expanding the key_prefix. type Error: Debug; diff --git a/linera-views/src/common.rs b/linera-views/src/common.rs index d4e58934837..f4f69c7da36 100644 --- a/linera-views/src/common.rs +++ b/linera-views/src/common.rs @@ -295,8 +295,8 @@ pub trait KeyValueIterable { } /// Low-level, asynchronous read key-value operations. Useful for storage APIs not based on views. -#[async_trait] -pub trait ReadableKeyValueStore { +#[trait_variant::make(ReadableKeyValueStore: Send)] +pub trait LocalReadableKeyValueStore { /// The maximal size of keys that can be stored. const MAX_KEY_SIZE: usize; @@ -324,33 +324,44 @@ pub trait ReadableKeyValueStore { /// Finds the `(key,value)` pairs matching the prefix. The prefix is not included in the returned keys. async fn find_key_values_by_prefix(&self, key_prefix: &[u8]) -> Result; + // We can't use `async fn` here in the below implementations due to + // https://github.com/rust-lang/impl-trait-utils/issues/17, but once that bug is fixed + // we can revert them to `async fn` syntax, which is neater. + /// Reads a single `key` and deserializes the result if present. - async fn read_value(&self, key: &[u8]) -> Result, E> + fn read_value( + &self, + key: &[u8], + ) -> impl Future, E>> where + Self: Sync, E: From, { - from_bytes_opt(&self.read_value_bytes(key).await?) + async { from_bytes_opt(&self.read_value_bytes(key).await?) } } /// Reads multiple `keys` and deserializes the results if present. - async fn read_multi_values( + fn read_multi_values( &self, keys: Vec>, - ) -> Result>, E> + ) -> impl Future>, E>> where + Self: Sync, E: From, { - let mut values = Vec::with_capacity(keys.len()); - for entry in self.read_multi_values_bytes(keys).await? { - values.push(from_bytes_opt(&entry)?); + async { + let mut values = Vec::with_capacity(keys.len()); + for entry in self.read_multi_values_bytes(keys).await? { + values.push(from_bytes_opt(&entry)?); + } + Ok(values) } - Ok(values) } } /// Low-level, asynchronous write key-value operations. Useful for storage APIs not based on views. -#[async_trait] -pub trait WritableKeyValueStore { +#[trait_variant::make(WritableKeyValueStore: Send)] +pub trait LocalWritableKeyValueStore { /// The maximal size of values that can be stored. const MAX_VALUE_SIZE: usize; @@ -363,10 +374,10 @@ pub trait WritableKeyValueStore { } /// Low-level trait for the administration of stores and their namespaces. -#[async_trait] -pub trait AdminKeyValueStore: Sized { +#[trait_variant::make(AdminKeyValueStore: Send)] +pub trait LocalAdminKeyValueStore: Sized { /// The error type returned by the store's methods. - type Error: Send; + type Error; /// The configuration needed to interact with a new store. type Config: Send + Sync; @@ -378,11 +389,14 @@ pub trait AdminKeyValueStore: Sized { async fn list_all(config: &Self::Config) -> Result, Self::Error>; /// Deletes all the existing namespaces. - async fn delete_all(config: &Self::Config) -> Result<(), Self::Error> { - for namespace in Self::list_all(config).await? { - Self::delete(config, &namespace).await?; + fn delete_all(config: &Self::Config) -> impl Future> { + async { + let namespaces = Self::list_all(config).await?; + for namespace in namespaces { + Self::delete(config, &namespace).await?; + } + Ok(()) } - Ok(()) } /// Tests if a given namespace exists. @@ -395,26 +409,30 @@ pub trait AdminKeyValueStore: Sized { async fn delete(config: &Self::Config, namespace: &str) -> Result<(), Self::Error>; /// Initializes a storage if missing and provides it. - async fn maybe_create_and_connect( + fn maybe_create_and_connect( config: &Self::Config, namespace: &str, - ) -> Result { - if !Self::exists(config, namespace).await? { - Self::create(config, namespace).await?; + ) -> impl Future> { + async { + if !Self::exists(config, namespace).await? { + Self::create(config, namespace).await?; + } + Self::connect(config, namespace).await } - Self::connect(config, namespace).await } /// Creates a new storage. Overwrites it if this namespace already exists. - async fn recreate_and_connect( + fn recreate_and_connect( config: &Self::Config, namespace: &str, - ) -> Result { - if Self::exists(config, namespace).await? { - Self::delete(config, namespace).await?; + ) -> impl Future> { + async { + if Self::exists(config, namespace).await? { + Self::delete(config, namespace).await?; + } + Self::create(config, namespace).await?; + Self::connect(config, namespace).await } - Self::create(config, namespace).await?; - Self::connect(config, namespace).await } } @@ -426,6 +444,18 @@ pub trait KeyValueStore: type Error: Debug; } +/// Low-level, asynchronous write and read key-value operations, without a `Send` bound. Useful for storage APIs not based on views. +pub trait LocalKeyValueStore: + LocalReadableKeyValueStore + LocalWritableKeyValueStore +{ + /// The error type. + type Error: Debug; +} + +impl LocalKeyValueStore for S { + type Error = ::Error; +} + #[doc(hidden)] /// Iterates keys by reference in a vector of keys. /// Inspired by https://depth-first.com/articles/2020/06/22/returning-rust-iterators/ diff --git a/linera-views/src/dynamo_db.rs b/linera-views/src/dynamo_db.rs index 6545949d467..a672eb3720a 100644 --- a/linera-views/src/dynamo_db.rs +++ b/linera-views/src/dynamo_db.rs @@ -324,7 +324,6 @@ pub struct DynamoDbStoreConfig { pub common_config: CommonStoreConfig, } -#[async_trait] impl AdminKeyValueStore for DynamoDbStoreInternal { type Error = DynamoDbContextError; type Config = DynamoDbStoreConfig; @@ -785,7 +784,6 @@ impl KeyValueIterable for DynamoDbKeyValues { } } -#[async_trait] impl ReadableKeyValueStore for DynamoDbStoreInternal { const MAX_KEY_SIZE: usize = MAX_KEY_SIZE; type Keys = DynamoDbKeys; @@ -818,8 +816,10 @@ impl ReadableKeyValueStore for DynamoDbStoreInternal { let handle = self.read_value_bytes_general(key_db); handles.push(handle); } - let result = join_all(handles).await; - Ok(result.into_iter().collect::>()?) + join_all(handles) + .await + .into_iter() + .collect::>() } async fn find_keys_by_prefix( @@ -890,7 +890,6 @@ pub struct DynamoDbStore { store: LruCachingStore>>, } -#[async_trait] impl ReadableKeyValueStore for DynamoDbStore { const MAX_KEY_SIZE: usize = MAX_KEY_SIZE - 4; type Keys = Vec>; @@ -930,7 +929,6 @@ impl ReadableKeyValueStore for DynamoDbStore { } } -#[async_trait] impl WritableKeyValueStore for DynamoDbStore { const MAX_VALUE_SIZE: usize = DynamoDbStoreInternal::MAX_VALUE_SIZE; @@ -943,12 +941,10 @@ impl WritableKeyValueStore for DynamoDbStore { } } -#[async_trait] impl KeyValueStore for DynamoDbStore { type Error = DynamoDbContextError; } -#[async_trait] impl AdminKeyValueStore for DynamoDbStore { type Error = DynamoDbContextError; type Config = DynamoDbStoreConfig; diff --git a/linera-views/src/journaling.rs b/linera-views/src/journaling.rs index 4e9956d0614..a370d4655a2 100644 --- a/linera-views/src/journaling.rs +++ b/linera-views/src/journaling.rs @@ -102,7 +102,6 @@ pub struct JournalingKeyValueStore { pub store: K, } -#[async_trait] impl DeletePrefixExpander for &JournalingKeyValueStore where K: DirectKeyValueStore + Send + Sync, @@ -117,7 +116,6 @@ where } } -#[async_trait] impl ReadableKeyValueStore for JournalingKeyValueStore where K: DirectKeyValueStore + Send + Sync, @@ -161,7 +159,6 @@ where } } -#[async_trait] impl AdminKeyValueStore for JournalingKeyValueStore where K: AdminKeyValueStore + Send + Sync, @@ -195,7 +192,6 @@ where } } -#[async_trait] impl WritableKeyValueStore for JournalingKeyValueStore where K: DirectKeyValueStore + Send + Sync, diff --git a/linera-views/src/key_value_store_view.rs b/linera-views/src/key_value_store_view.rs index 664b24a68c3..a9efd6efb43 100644 --- a/linera-views/src/key_value_store_view.rs +++ b/linera-views/src/key_value_store_view.rs @@ -1017,7 +1017,6 @@ pub struct ViewContainer { } #[cfg(with_testing)] -#[async_trait] impl ReadableKeyValueStore for ViewContainer where C: Context + Sync + Send + Clone, @@ -1062,8 +1061,8 @@ where view.find_key_values_by_prefix(key_prefix).await } } + #[cfg(with_testing)] -#[async_trait] impl WritableKeyValueStore for ViewContainer where C: Context + Sync + Send + Clone, diff --git a/linera-views/src/lru_caching.rs b/linera-views/src/lru_caching.rs index 74a475ebf46..42f4e34953d 100644 --- a/linera-views/src/lru_caching.rs +++ b/linera-views/src/lru_caching.rs @@ -10,7 +10,6 @@ use std::{ }; use async_lock::Mutex; -use async_trait::async_trait; use linked_hash_map::LinkedHashMap; #[cfg(with_testing)] use { @@ -111,7 +110,6 @@ pub struct LruCachingStore { lru_read_values: Option>>, } -#[async_trait] impl ReadableKeyValueStore for LruCachingStore where K: KeyValueStore + Send + Sync, @@ -126,27 +124,24 @@ where } async fn read_value_bytes(&self, key: &[u8]) -> Result>, K::Error> { - match &self.lru_read_values { - None => { - return self.store.read_value_bytes(key).await; - } - Some(lru_read_values) => { - // First inquiring in the read_value_bytes LRU - let lru_read_values_container = lru_read_values.lock().await; - if let Some(value) = lru_read_values_container.query(key) { - #[cfg(with_metrics)] - NUM_CACHE_SUCCESS.with_label_values(&[]).inc(); - return Ok(value.clone()); - } - drop(lru_read_values_container); - #[cfg(with_metrics)] - NUM_CACHE_FAULT.with_label_values(&[]).inc(); - let value = self.store.read_value_bytes(key).await?; - let mut lru_read_values = lru_read_values.lock().await; - lru_read_values.insert(key.to_vec(), value.clone()); - Ok(value) - } + let Some(lru_read_values) = &self.lru_read_values else { + return self.store.read_value_bytes(key).await; + }; + + // First inquiring in the read_value_bytes LRU + let lru_read_values_container = lru_read_values.lock().await; + if let Some(value) = lru_read_values_container.query(key) { + #[cfg(with_metrics)] + NUM_CACHE_SUCCESS.with_label_values(&[]).inc(); + return Ok(value.clone()); } + drop(lru_read_values_container); + #[cfg(with_metrics)] + NUM_CACHE_FAULT.with_label_values(&[]).inc(); + let value = self.store.read_value_bytes(key).await?; + let mut lru_read_values = lru_read_values.lock().await; + lru_read_values.insert(key.to_vec(), value.clone()); + Ok(value) } async fn contains_key(&self, key: &[u8]) -> Result { @@ -163,44 +158,41 @@ where &self, keys: Vec>, ) -> Result>>, K::Error> { - match &self.lru_read_values { - None => { - return self.store.read_multi_values_bytes(keys).await; - } - Some(lru_read_values) => { - let mut result = Vec::with_capacity(keys.len()); - let mut cache_miss_indices = Vec::new(); - let mut miss_keys = Vec::new(); - let lru_read_values_container = lru_read_values.lock().await; - for (i, key) in keys.into_iter().enumerate() { - if let Some(value) = lru_read_values_container.query(&key) { - #[cfg(with_metrics)] - NUM_CACHE_SUCCESS.with_label_values(&[]).inc(); - result.push(value.clone()); - } else { - #[cfg(with_metrics)] - NUM_CACHE_FAULT.with_label_values(&[]).inc(); - result.push(None); - cache_miss_indices.push(i); - miss_keys.push(key); - } - } - drop(lru_read_values_container); - let values = self - .store - .read_multi_values_bytes(miss_keys.clone()) - .await?; - let mut lru_read_values = lru_read_values.lock().await; - for (i, (key, value)) in cache_miss_indices - .into_iter() - .zip(miss_keys.into_iter().zip(values)) - { - lru_read_values.insert(key, value.clone()); - result[i] = value; - } - Ok(result) + let Some(lru_read_values) = &self.lru_read_values else { + return self.store.read_multi_values_bytes(keys).await; + }; + + let mut result = Vec::with_capacity(keys.len()); + let mut cache_miss_indices = Vec::new(); + let mut miss_keys = Vec::new(); + let lru_read_values_container = lru_read_values.lock().await; + for (i, key) in keys.into_iter().enumerate() { + if let Some(value) = lru_read_values_container.query(&key) { + #[cfg(with_metrics)] + NUM_CACHE_SUCCESS.with_label_values(&[]).inc(); + result.push(value.clone()); + } else { + #[cfg(with_metrics)] + NUM_CACHE_FAULT.with_label_values(&[]).inc(); + result.push(None); + cache_miss_indices.push(i); + miss_keys.push(key); } } + drop(lru_read_values_container); + let values = self + .store + .read_multi_values_bytes(miss_keys.clone()) + .await?; + let mut lru_read_values = lru_read_values.lock().await; + for (i, (key, value)) in cache_miss_indices + .into_iter() + .zip(miss_keys.into_iter().zip(values)) + { + lru_read_values.insert(key, value.clone()); + result[i] = value; + } + Ok(result) } async fn find_keys_by_prefix(&self, key_prefix: &[u8]) -> Result { @@ -215,7 +207,6 @@ where } } -#[async_trait] impl WritableKeyValueStore for LruCachingStore where K: KeyValueStore + Send + Sync, @@ -224,29 +215,26 @@ where const MAX_VALUE_SIZE: usize = K::MAX_VALUE_SIZE; async fn write_batch(&self, batch: Batch, base_key: &[u8]) -> Result<(), K::Error> { - match &self.lru_read_values { - None => { - return self.store.write_batch(batch, base_key).await; - } - Some(lru_read_values) => { - let mut lru_read_values = lru_read_values.lock().await; - for operation in &batch.operations { - match operation { - WriteOperation::Put { key, value } => { - lru_read_values.insert(key.to_vec(), Some(value.to_vec())); - } - WriteOperation::Delete { key } => { - lru_read_values.insert(key.to_vec(), None); - } - WriteOperation::DeletePrefix { key_prefix } => { - lru_read_values.delete_prefix(key_prefix); - } - } + let Some(lru_read_values) = &self.lru_read_values else { + return self.store.write_batch(batch, base_key).await; + }; + + let mut lru_read_values = lru_read_values.lock().await; + for operation in &batch.operations { + match operation { + WriteOperation::Put { key, value } => { + lru_read_values.insert(key.to_vec(), Some(value.to_vec())); + } + WriteOperation::Delete { key } => { + lru_read_values.insert(key.to_vec(), None); + } + WriteOperation::DeletePrefix { key_prefix } => { + lru_read_values.delete_prefix(key_prefix); } - drop(lru_read_values); - self.store.write_batch(batch, base_key).await } } + drop(lru_read_values); + self.store.write_batch(batch, base_key).await } async fn clear_journal(&self, base_key: &[u8]) -> Result<(), K::Error> { diff --git a/linera-views/src/memory.rs b/linera-views/src/memory.rs index fd64334b9e5..63ebf71dfcf 100644 --- a/linera-views/src/memory.rs +++ b/linera-views/src/memory.rs @@ -4,7 +4,6 @@ use std::{collections::BTreeMap, fmt::Debug, sync::Arc}; use async_lock::{Mutex, MutexGuardArc, RwLock}; -use async_trait::async_trait; use futures::FutureExt; use thiserror::Error; @@ -53,7 +52,6 @@ pub struct MemoryStore { pub max_stream_queries: usize, } -#[async_trait] impl ReadableKeyValueStore for MemoryStore { const MAX_KEY_SIZE: usize = usize::MAX; type Keys = Vec>; @@ -113,7 +111,6 @@ impl ReadableKeyValueStore for MemoryStore { } } -#[async_trait] impl WritableKeyValueStore for MemoryStore { const MAX_VALUE_SIZE: usize = usize::MAX; @@ -146,7 +143,6 @@ impl WritableKeyValueStore for MemoryStore { } } -#[async_trait] impl AdminKeyValueStore for MemoryStore { type Error = MemoryContextError; type Config = MemoryStoreConfig; @@ -263,7 +259,6 @@ impl From for ViewError { } } -#[async_trait] impl DeletePrefixExpander for MemoryContext<()> { type Error = MemoryContextError; diff --git a/linera-views/src/metering.rs b/linera-views/src/metering.rs index 7a1b3ef697b..f878f8b7d17 100644 --- a/linera-views/src/metering.rs +++ b/linera-views/src/metering.rs @@ -1,7 +1,6 @@ // Copyright (c) Zefchain Labs, Inc. // SPDX-License-Identifier: Apache-2.0 -use async_trait::async_trait; use convert_case::{Case, Casing}; use linera_base::{ prometheus_util::{register_histogram_vec, MeasureLatency}, @@ -116,7 +115,6 @@ pub struct MeteredStore { pub store: K, } -#[async_trait] impl ReadableKeyValueStore for MeteredStore where K: ReadableKeyValueStore + Send + Sync, @@ -155,7 +153,6 @@ where } } -#[async_trait] impl WritableKeyValueStore for MeteredStore where K: WritableKeyValueStore + Send + Sync, diff --git a/linera-views/src/rocks_db.rs b/linera-views/src/rocks_db.rs index abf1352ff18..c750f6cab51 100644 --- a/linera-views/src/rocks_db.rs +++ b/linera-views/src/rocks_db.rs @@ -8,7 +8,6 @@ use std::{ sync::Arc, }; -use async_trait::async_trait; use linera_base::ensure; use thiserror::Error; #[cfg(with_testing)] @@ -74,7 +73,6 @@ impl RocksDbStoreInternal { } } -#[async_trait] impl ReadableKeyValueStore for RocksDbStoreInternal { const MAX_KEY_SIZE: usize = MAX_KEY_SIZE; type Keys = Vec>; @@ -180,7 +178,6 @@ impl ReadableKeyValueStore for RocksDbStoreInternal { } } -#[async_trait] impl WritableKeyValueStore for RocksDbStoreInternal { const MAX_VALUE_SIZE: usize = MAX_VALUE_SIZE; @@ -245,7 +242,6 @@ impl WritableKeyValueStore for RocksDbStoreInternal { } } -#[async_trait] impl AdminKeyValueStore for RocksDbStoreInternal { type Error = RocksDbContextError; type Config = RocksDbStoreConfig; @@ -391,7 +387,6 @@ pub async fn create_rocks_db_test_store() -> (RocksDbStore, TempDir) { /// An implementation of [`crate::common::Context`] based on RocksDB pub type RocksDbContext = ContextFromStore; -#[async_trait] impl ReadableKeyValueStore for RocksDbStore { const MAX_KEY_SIZE: usize = MAX_KEY_SIZE; type Keys = Vec>; @@ -431,7 +426,6 @@ impl ReadableKeyValueStore for RocksDbStore { } } -#[async_trait] impl WritableKeyValueStore for RocksDbStore { const MAX_VALUE_SIZE: usize = usize::MAX; @@ -444,7 +438,6 @@ impl WritableKeyValueStore for RocksDbStore { } } -#[async_trait] impl AdminKeyValueStore for RocksDbStore { type Error = RocksDbContextError; type Config = RocksDbStoreConfig; @@ -484,7 +477,6 @@ impl AdminKeyValueStore for RocksDbStore { } } -#[async_trait] impl KeyValueStore for RocksDbStore { type Error = RocksDbContextError; } diff --git a/linera-views/src/scylla_db.rs b/linera-views/src/scylla_db.rs index 3a5ed0560fd..f5a59ff4951 100644 --- a/linera-views/src/scylla_db.rs +++ b/linera-views/src/scylla_db.rs @@ -344,7 +344,6 @@ impl From for crate::views::ViewError { } } -#[async_trait] impl ReadableKeyValueStore for ScyllaDbStoreInternal { const MAX_KEY_SIZE: usize = MAX_KEY_SIZE; type Keys = Vec>; @@ -373,8 +372,10 @@ impl ReadableKeyValueStore for ScyllaDbStoreInternal { let store = self.store.deref(); let _guard = self.acquire().await; let handles = keys.into_iter().map(|key| store.read_value_internal(key)); - let result = join_all(handles).await; - Ok(result.into_iter().collect::>()?) + join_all(handles) + .await + .into_iter() + .collect::>() } async fn find_keys_by_prefix( @@ -426,7 +427,6 @@ impl DirectWritableKeyValueStore for ScyllaDbStoreInternal } } -#[async_trait] impl AdminKeyValueStore for ScyllaDbStoreInternal { type Error = ScyllaDbContextError; type Config = ScyllaDbStoreConfig; @@ -589,7 +589,6 @@ impl DirectKeyValueStore for ScyllaDbStoreInternal { type Error = ScyllaDbContextError; } -#[async_trait] impl DeletePrefixExpander for ScyllaDbClient { type Error = ScyllaDbContextError; @@ -645,7 +644,6 @@ pub struct ScyllaDbStoreConfig { pub common_config: CommonStoreConfig, } -#[async_trait] impl ReadableKeyValueStore for ScyllaDbStore { const MAX_KEY_SIZE: usize = ScyllaDbStoreInternal::MAX_KEY_SIZE; @@ -688,7 +686,6 @@ impl ReadableKeyValueStore for ScyllaDbStore { } } -#[async_trait] impl WritableKeyValueStore for ScyllaDbStore { const MAX_VALUE_SIZE: usize = ScyllaDbStoreInternal::MAX_VALUE_SIZE; @@ -701,7 +698,6 @@ impl WritableKeyValueStore for ScyllaDbStore { } } -#[async_trait] impl AdminKeyValueStore for ScyllaDbStore { type Error = ScyllaDbContextError; type Config = ScyllaDbStoreConfig; diff --git a/linera-views/src/test_utils/mod.rs b/linera-views/src/test_utils/mod.rs index 8f7c2e17d74..282da596a56 100644 --- a/linera-views/src/test_utils/mod.rs +++ b/linera-views/src/test_utils/mod.rs @@ -15,7 +15,7 @@ use crate::{ Batch, WriteOperation, WriteOperation::{Delete, Put}, }, - common::{AdminKeyValueStore, KeyIterable, KeyValueIterable, KeyValueStore}, + common::{KeyIterable, KeyValueIterable, LocalAdminKeyValueStore, LocalKeyValueStore}, }; // The following seed is chosen to have equal numbers of 1s and 0s, as advised by @@ -237,7 +237,7 @@ pub fn span_random_reordering_put_delete( /// * `read_multi_values_bytes` /// * `find_keys_by_prefix` / `find_key_values_by_prefix` /// * The ordering of keys returned by `find_keys_by_prefix` and `find_key_values_by_prefix` -pub async fn run_reads(store: S, key_values: Vec<(Vec, Vec)>) { +pub async fn run_reads(store: S, key_values: Vec<(Vec, Vec)>) { // We need a nontrivial key_prefix because dynamo requires a non-trivial prefix let mut batch = Batch::new(); let mut keys = Vec::new(); @@ -426,7 +426,7 @@ fn realize_batch(batch: &Batch) -> BTreeMap, Vec> { kv_state } -async fn read_key_values_prefix( +async fn read_key_values_prefix( key_value_store: &C, key_prefix: &[u8], ) -> BTreeMap, Vec> { @@ -446,7 +446,7 @@ async fn read_key_values_prefix( } /// Writes and then reads data under a prefix, and verifies the result. -pub async fn run_test_batch_from_blank( +pub async fn run_test_batch_from_blank( key_value_store: &C, key_prefix: Vec, batch: Batch, @@ -459,7 +459,7 @@ pub async fn run_test_batch_from_blank( } /// Run many operations on batches always starting from a blank state. -pub async fn run_writes_from_blank(key_value_store: &C) { +pub async fn run_writes_from_blank(key_value_store: &C) { let mut rng = make_deterministic_rng(); let n_oper = 10; let batch_size = 500; @@ -487,7 +487,7 @@ pub async fn run_writes_from_blank(key_value_store: &C) /// Then we select half of them at random and delete them. By the random /// selection, Scylla is forced to introduce around 100000 tombstones /// which triggers the crash with the default settings. -pub async fn tombstone_triggering_test(key_value_store: C) { +pub async fn tombstone_triggering_test(key_value_store: C) { let mut rng = make_deterministic_rng(); let value_size = 100; let n_entry = 200000; @@ -525,7 +525,7 @@ pub async fn tombstone_triggering_test(key_value_store: /// must handle that. /// /// The size of the value vary as each size has its own issues. -pub async fn run_big_write_read( +pub async fn run_big_write_read( key_value_store: C, target_size: usize, value_sizes: Vec, @@ -547,7 +547,7 @@ pub async fn run_big_write_read( type StateBatch = (Vec<(Vec, Vec)>, Batch); -async fn run_test_batch_from_state( +async fn run_test_batch_from_state( key_value_store: &C, key_prefix: Vec, state_and_batch: StateBatch, @@ -642,7 +642,7 @@ fn generate_specific_state_batch(key_prefix: &[u8], option: usize) -> StateBatch /// Run some deterministic and random batches operation and check their /// correctness -pub async fn run_writes_from_state(key_value_store: &C) { +pub async fn run_writes_from_state(key_value_store: &C) { for option in 0..7 { let key_prefix = if option == 6 { vec![255, 255, 255] @@ -654,7 +654,7 @@ pub async fn run_writes_from_state(key_value_store: &C) } } -async fn namespaces_with_prefix( +async fn namespaces_with_prefix( config: &S::Config, prefix: &str, ) -> BTreeSet @@ -671,7 +671,7 @@ where /// Exercises the functionalities of the `AdminKeyValueStore`. /// This tests everything except the `delete_all` which would /// interact with other namespaces. -pub async fn admin_test(config: &S::Config) +pub async fn admin_test(config: &S::Config) where S::Error: Debug, { diff --git a/linera-views/src/value_splitting.rs b/linera-views/src/value_splitting.rs index 0c36dbd16ab..346975f6966 100644 --- a/linera-views/src/value_splitting.rs +++ b/linera-views/src/value_splitting.rs @@ -3,7 +3,6 @@ use std::fmt::Debug; -use async_trait::async_trait; use futures::FutureExt; use linera_base::ensure; use thiserror::Error; @@ -45,7 +44,6 @@ pub struct ValueSplittingStore { pub store: K, } -#[async_trait] impl ReadableKeyValueStore for ValueSplittingStore where K: KeyValueStore + Send + Sync, @@ -196,7 +194,6 @@ where } } -#[async_trait] impl WritableKeyValueStore for ValueSplittingStore where K: KeyValueStore + Send + Sync, @@ -242,7 +239,6 @@ where } } -#[async_trait] impl AdminKeyValueStore for ValueSplittingStore where K: AdminKeyValueStore + Send + Sync, @@ -343,7 +339,6 @@ impl Default for TestMemoryStoreInternal { } } -#[async_trait] impl ReadableKeyValueStore for TestMemoryStoreInternal { const MAX_KEY_SIZE: usize = usize::MAX; type Keys = Vec>; @@ -383,7 +378,6 @@ impl ReadableKeyValueStore for TestMemoryStoreInternal { } } -#[async_trait] impl WritableKeyValueStore for TestMemoryStoreInternal { // We set up the MAX_VALUE_SIZE to the artificially low value of 100 // purely for testing purposes. @@ -430,7 +424,6 @@ pub struct TestMemoryStore { store: ValueSplittingStore, } -#[async_trait] impl ReadableKeyValueStore for TestMemoryStore { const MAX_KEY_SIZE: usize = usize::MAX; type Keys = Vec>; @@ -470,7 +463,6 @@ impl ReadableKeyValueStore for TestMemoryStore { } } -#[async_trait] impl WritableKeyValueStore for TestMemoryStore { const MAX_VALUE_SIZE: usize = usize::MAX;