From f5111b075f82d8c64a6a8ee63eda8ae90a578d45 Mon Sep 17 00:00:00 2001 From: "Nathan (Blaise) Bruer" Date: Mon, 27 May 2024 18:32:09 -0500 Subject: [PATCH] Refactor Store Api into client side and driver side Refactors the Store api into the driver (backend) implementation and a client Store/StoreLike api. Store & StoreLike have their sizes known at compile-time. This enables us to add templates to the client-side to make it easier to work with, for example, we no longer need to Pin the store and we'll be able to add things like `digest: impl Into` to make the callers life much easier. towards: #934 --- deployment-examples/docker-compose/Dockerfile | 4 +- .../src/cache_lookup_scheduler.rs | 22 +- .../tests/cache_lookup_scheduler_test.rs | 13 +- nativelink-service/src/ac_server.rs | 18 +- nativelink-service/src/bytestream_server.rs | 25 +- nativelink-service/src/cas_server.rs | 27 +- nativelink-service/src/execution_server.rs | 11 +- nativelink-service/tests/ac_server_test.rs | 17 +- .../tests/bytestream_server_test.rs | 33 +- nativelink-service/tests/cas_server_test.rs | 26 +- nativelink-store/src/ac_utils.rs | 10 +- .../src/completeness_checking_store.rs | 63 ++-- nativelink-store/src/compression_store.rs | 30 +- nativelink-store/src/dedup_store.rs | 58 ++-- nativelink-store/src/default_store_factory.rs | 14 +- nativelink-store/src/existence_cache_store.rs | 32 +- nativelink-store/src/fast_slow_store.rs | 152 ++++----- nativelink-store/src/filesystem_store.rs | 35 +- nativelink-store/src/grpc_store.rs | 12 +- nativelink-store/src/memory_store.rs | 14 +- nativelink-store/src/noop_store.rs | 12 +- nativelink-store/src/redis_store.rs | 26 +- nativelink-store/src/ref_store.rs | 44 +-- nativelink-store/src/s3_store.rs | 14 +- nativelink-store/src/shard_store.rs | 47 +-- .../src/size_partitioning_store.rs | 55 ++-- nativelink-store/src/store_manager.rs | 8 +- nativelink-store/src/verify_store.rs | 33 +- nativelink-store/tests/ac_utils_test.rs | 8 +- .../tests/completeness_checking_store_test.rs | 86 ++--- .../tests/compression_store_test.rs | 29 +- nativelink-store/tests/dedup_store_test.rs | 90 ++--- .../tests/existence_store_test.rs | 24 +- .../tests/fast_slow_store_test.rs | 126 +++---- .../tests/filesystem_store_test.rs | 148 +++------ nativelink-store/tests/memory_store_test.rs | 11 +- nativelink-store/tests/redis_store_test.rs | 34 +- nativelink-store/tests/ref_store_test.rs | 66 ++-- nativelink-store/tests/s3_store_test.rs | 41 +-- nativelink-store/tests/shard_store_test.rs | 68 +--- .../tests/size_partitioning_store_test.rs | 42 +-- nativelink-store/tests/verify_store_test.rs | 81 +++-- nativelink-util/src/health_utils.rs | 2 +- nativelink-util/src/store_trait.rs | 308 ++++++++++++++---- nativelink-worker/src/local_worker.rs | 15 +- .../src/running_actions_manager.rs | 63 ++-- nativelink-worker/tests/local_worker_test.rs | 32 +- .../tests/running_actions_manager_test.rs | 297 +++++++++-------- 48 files changed, 1097 insertions(+), 1329 deletions(-) diff --git a/deployment-examples/docker-compose/Dockerfile b/deployment-examples/docker-compose/Dockerfile index 9f4ea0607..aae7d0ff3 100644 --- a/deployment-examples/docker-compose/Dockerfile +++ b/deployment-examples/docker-compose/Dockerfile @@ -29,7 +29,7 @@ RUN apt-get update \ DEBIAN_FRONTEND=noninteractive \ apt-get install --no-install-recommends -y \ npm=8.5.1~ds-1 \ - git=1:2.34.1-1ubuntu1.10 \ + git=1:2.34.1-1ubuntu1.11 \ gcc=4:11.2.0-1ubuntu1 \ g++=4:11.2.0-1ubuntu1 \ python3=3.10.6-1~22.04 \ @@ -38,7 +38,7 @@ RUN apt-get update \ DEBIAN_FRONTEND=noninteractive \ apt-get install --no-install-recommends -y \ npm=6.14.4+ds-1ubuntu2 \ - git=1:2.25.1-1ubuntu3.11 \ + git=1:2.25.1-1ubuntu3.12 \ gcc=4:9.3.0-1ubuntu2 \ g++=4:9.3.0-1ubuntu2 \ python3=3.8.2-0ubuntu2 \ diff --git a/nativelink-scheduler/src/cache_lookup_scheduler.rs b/nativelink-scheduler/src/cache_lookup_scheduler.rs index 50538044b..c5856f55c 100644 --- a/nativelink-scheduler/src/cache_lookup_scheduler.rs +++ b/nativelink-scheduler/src/cache_lookup_scheduler.rs @@ -13,7 +13,6 @@ // limitations under the License. use std::collections::HashMap; -use std::pin::Pin; use std::sync::Arc; use async_trait::async_trait; @@ -30,7 +29,7 @@ use nativelink_util::action_messages::{ use nativelink_util::background_spawn; use nativelink_util::common::DigestInfo; use nativelink_util::digest_hasher::DigestHasherFunc; -use nativelink_util::store_trait::Store; +use nativelink_util::store_trait::{Store, StoreLike}; use parking_lot::{Mutex, MutexGuard}; use scopeguard::guard; use tokio::select; @@ -50,7 +49,7 @@ type CheckActions = HashMap, + ac_store: Store, /// The "real" scheduler to use to perform actions if they were not found /// in the action cache. action_scheduler: Arc, @@ -59,14 +58,13 @@ pub struct CacheLookupScheduler { } async fn get_action_from_store( - ac_store: Pin<&dyn Store>, + ac_store: &Store, action_digest: DigestInfo, instance_name: String, digest_function: DigestHasherFunc, ) -> Option { // If we are a GrpcStore we shortcut here, as this is a special store. - let any_store = ac_store.inner_store(Some(action_digest)).as_any(); - if let Some(grpc_store) = any_store.downcast_ref::() { + if let Some(grpc_store) = ac_store.downcast_ref::(Some(action_digest)) { let action_result_request = GetActionResultRequest { instance_name, action_digest: Some(action_digest.into()), @@ -103,10 +101,7 @@ fn subscribe_to_existing_action( } impl CacheLookupScheduler { - pub fn new( - ac_store: Arc, - action_scheduler: Arc, - ) -> Result { + pub fn new(ac_store: Store, action_scheduler: Arc) -> Result { Ok(Self { ac_store, action_scheduler, @@ -169,17 +164,14 @@ impl ActionScheduler for CacheLookupScheduler { let action_digest = current_state.action_digest(); let instance_name = action_info.instance_name().clone(); if let Some(action_result) = get_action_from_store( - Pin::new(ac_store.as_ref()), + &ac_store, *action_digest, instance_name, current_state.unique_qualifier.digest_function, ) .await { - match Pin::new(ac_store.clone().as_ref()) - .has(*action_digest) - .await - { + match ac_store.has(*action_digest).await { Ok(Some(_)) => { Arc::make_mut(&mut current_state).stage = ActionStage::CompletedFromCache(action_result); diff --git a/nativelink-scheduler/tests/cache_lookup_scheduler_test.rs b/nativelink-scheduler/tests/cache_lookup_scheduler_test.rs index b2bf30c4d..bf026ed81 100644 --- a/nativelink-scheduler/tests/cache_lookup_scheduler_test.rs +++ b/nativelink-scheduler/tests/cache_lookup_scheduler_test.rs @@ -13,7 +13,6 @@ // limitations under the License. use std::collections::HashMap; -use std::pin::Pin; use std::sync::Arc; use std::time::UNIX_EPOCH; @@ -33,7 +32,7 @@ use nativelink_store::memory_store::MemoryStore; use nativelink_util::action_messages::{ActionInfoHashKey, ActionResult, ActionStage, ActionState}; use nativelink_util::common::DigestInfo; use nativelink_util::digest_hasher::DigestHasherFunc; -use nativelink_util::store_trait::Store; +use nativelink_util::store_trait::{Store, StoreLike}; use prost::Message; use tokio::sync::watch; use tokio::{self}; @@ -42,15 +41,15 @@ use utils::scheduler_utils::{make_base_action_info, INSTANCE_NAME}; struct TestContext { mock_scheduler: Arc, - ac_store: Arc, + ac_store: Store, cache_scheduler: CacheLookupScheduler, } fn make_cache_scheduler() -> Result { let mock_scheduler = Arc::new(MockActionScheduler::new()); - let ac_store = Arc::new(MemoryStore::new( + let ac_store = Store::new(Arc::new(MemoryStore::new( &nativelink_config::stores::MemoryStore::default(), - )); + ))); let cache_scheduler = CacheLookupScheduler::new(ac_store.clone(), mock_scheduler.clone())?; Ok(TestContext { mock_scheduler, @@ -91,8 +90,8 @@ mod cache_lookup_scheduler_tests { let context = make_cache_scheduler()?; let action_info = make_base_action_info(UNIX_EPOCH); let action_result = ProtoActionResult::from(ActionResult::default()); - let store_pin = Pin::new(context.ac_store.as_ref()); - store_pin + context + .ac_store .update_oneshot(*action_info.digest(), action_result.encode_to_vec().into()) .await?; let (_forward_watch_channel_tx, forward_watch_channel_rx) = diff --git a/nativelink-service/src/ac_server.rs b/nativelink-service/src/ac_server.rs index 8cd5ba6c1..74897f514 100644 --- a/nativelink-service/src/ac_server.rs +++ b/nativelink-service/src/ac_server.rs @@ -14,8 +14,6 @@ use std::collections::HashMap; use std::fmt::Debug; -use std::pin::Pin; -use std::sync::Arc; use bytes::BytesMut; use nativelink_config::cas_server::{AcStoreConfig, InstanceName}; @@ -31,14 +29,14 @@ use nativelink_store::grpc_store::GrpcStore; use nativelink_store::store_manager::StoreManager; use nativelink_util::common::DigestInfo; use nativelink_util::digest_hasher::make_ctx_for_hash_func; -use nativelink_util::store_trait::Store; +use nativelink_util::store_trait::{Store, StoreLike}; use prost::Message; use tonic::{Request, Response, Status}; use tracing::{error_span, event, instrument, Level}; #[derive(Clone)] pub struct AcStoreInfo { - store: Arc, + store: Store, read_only: bool, } @@ -97,14 +95,12 @@ impl AcServer { .try_into()?; // If we are a GrpcStore we shortcut here, as this is a special store. - let any_store = store_info.store.inner_store(Some(digest)).as_any(); - if let Some(grpc_store) = any_store.downcast_ref::() { + if let Some(grpc_store) = store_info.store.downcast_ref::(Some(digest)) { return grpc_store.get_action_result(Request::new(request)).await; } Ok(Response::new( - get_and_decode_digest::(Pin::new(store_info.store.as_ref()), &digest) - .await?, + get_and_decode_digest::(&store_info.store, &digest).await?, )) } @@ -132,8 +128,7 @@ impl AcServer { .try_into()?; // If we are a GrpcStore we shortcut here, as this is a special store. - let any_store = store_info.store.inner_store(Some(digest)).as_any(); - if let Some(grpc_store) = any_store.downcast_ref::() { + if let Some(grpc_store) = store_info.store.downcast_ref::(Some(digest)) { return grpc_store.update_action_result(Request::new(request)).await; } @@ -146,7 +141,8 @@ impl AcServer { .encode(&mut store_data) .err_tip(|| "Provided ActionResult could not be serialized")?; - Pin::new(store_info.store.as_ref()) + store_info + .store .update_oneshot(digest, store_data.freeze()) .await .err_tip(|| "Failed to update in action cache")?; diff --git a/nativelink-service/src/bytestream_server.rs b/nativelink-service/src/bytestream_server.rs index 6b32de3ae..4de21704d 100644 --- a/nativelink-service/src/bytestream_server.rs +++ b/nativelink-service/src/bytestream_server.rs @@ -44,7 +44,7 @@ use nativelink_util::digest_hasher::{ use nativelink_util::proto_stream_utils::WriteRequestStreamWrapper; use nativelink_util::resource_info::ResourceInfo; use nativelink_util::spawn; -use nativelink_util::store_trait::{Store, UploadSizeInfo}; +use nativelink_util::store_trait::{Store, StoreLike, UploadSizeInfo}; use nativelink_util::task::JoinHandleDropGuard; use parking_lot::Mutex; use tokio::time::sleep; @@ -153,7 +153,7 @@ type BytesWrittenAndIdleStream = (Arc, Option); type SleepFn = Arc BoxFuture<'static, ()> + Send + Sync>; pub struct ByteStreamServer { - stores: HashMap>, + stores: HashMap, // Max number of bytes to send on each grpc stream chunk. max_bytes_per_stream: usize, active_uploads: Arc>>, @@ -206,7 +206,7 @@ impl ByteStreamServer { fn create_or_join_upload_stream( &self, uuid: String, - store: Arc, + store: Store, digest: DigestInfo, ) -> Result, Error> { let (uuid, bytes_received) = match self.active_uploads.lock().entry(uuid) { @@ -236,7 +236,7 @@ impl ByteStreamServer { let store_update_fut = Box::pin(async move { // We need to wrap `Store::update()` in a another future because we need to capture // `store` to ensure it's lifetime follows the future and not the caller. - Pin::new(store.as_ref()) + store // Bytestream always uses digest size as the actual byte size. .update( digest, @@ -260,7 +260,7 @@ impl ByteStreamServer { async fn inner_read( &self, - store: Arc, + store: Store, digest: DigestInfo, read_request: ReadRequest, ) -> Result, Error> { @@ -289,7 +289,7 @@ impl ByteStreamServer { maybe_get_part_result: None, get_part_fut: Box::pin(async move { store - .get_part_arc(digest, tx, read_request.read_offset as usize, read_limit) + .get_part(digest, tx, read_request.read_offset as usize, read_limit) .await }), }); @@ -383,7 +383,7 @@ impl ByteStreamServer { )] async fn inner_write( &self, - store: Arc, + store: Store, digest: DigestInfo, stream: WriteRequestStreamWrapper, Status>, ) -> Result, Error> { @@ -520,8 +520,7 @@ impl ByteStreamServer { let digest = DigestInfo::try_new(resource_info.hash.as_ref(), resource_info.expected_size)?; // If we are a GrpcStore we shortcut here, as this is a special store. - let any_store = store_clone.inner_store(Some(digest)).as_any(); - if let Some(grpc_store) = any_store.downcast_ref::() { + if let Some(grpc_store) = store_clone.downcast_ref::(Some(digest)) { return grpc_store .query_write_status(Request::new(query_request.clone())) .await; @@ -544,7 +543,7 @@ impl ByteStreamServer { } } - let has_fut = Pin::new(store_clone.as_ref()).has(digest); + let has_fut = store_clone.has(digest); let Some(item_size) = has_fut.await.err_tip(|| "Failed to call .has() on store")? else { return Err(make_err!(Code::NotFound, "{}", "not found")); }; @@ -583,8 +582,7 @@ impl ByteStream for ByteStreamServer { let digest = DigestInfo::try_new(resource_info.hash.as_ref(), resource_info.expected_size)?; // If we are a GrpcStore we shortcut here, as this is a special store. - let any_store = store.inner_store(Some(digest)).as_any(); - if let Some(grpc_store) = any_store.downcast_ref::() { + if let Some(grpc_store) = store.downcast_ref::(Some(digest)) { let stream = grpc_store.read(Request::new(read_request)).await?; return Ok(Response::new(Box::pin(stream))); } @@ -640,8 +638,7 @@ impl ByteStream for ByteStreamServer { .err_tip(|| "Invalid digest input in ByteStream::write")?; // If we are a GrpcStore we shortcut here, as this is a special store. - let any_store = store.inner_store(Some(digest)).as_any(); - if let Some(grpc_store) = any_store.downcast_ref::() { + if let Some(grpc_store) = store.downcast_ref::(Some(digest)) { return grpc_store.write(stream).await.map_err(|e| e.into()); } diff --git a/nativelink-service/src/cas_server.rs b/nativelink-service/src/cas_server.rs index 8155f713e..26d2431bb 100644 --- a/nativelink-service/src/cas_server.rs +++ b/nativelink-service/src/cas_server.rs @@ -14,7 +14,6 @@ use std::collections::{HashMap, VecDeque}; use std::pin::Pin; -use std::sync::Arc; use bytes::Bytes; use futures::stream::{FuturesUnordered, Stream}; @@ -35,12 +34,12 @@ use nativelink_store::grpc_store::GrpcStore; use nativelink_store::store_manager::StoreManager; use nativelink_util::common::DigestInfo; use nativelink_util::digest_hasher::make_ctx_for_hash_func; -use nativelink_util::store_trait::Store; +use nativelink_util::store_trait::{Store, StoreLike}; use tonic::{Request, Response, Status}; use tracing::{error_span, event, instrument, Level}; pub struct CasServer { - stores: HashMap>, + stores: HashMap, } type GetTreeStream = Pin> + Send + 'static>>; @@ -79,7 +78,7 @@ impl CasServer { for digest in request.blob_digests.iter() { requested_blobs.push(DigestInfo::try_from(digest.clone())?); } - let sizes = Pin::new(store.as_ref()) + let sizes = store .has_many(&requested_blobs) .await .err_tip(|| "In find_missing_blobs")?; @@ -109,12 +108,11 @@ impl CasServer { // If we are a GrpcStore we shortcut here, as this is a special store. // Note: We don't know the digests here, so we try perform a very shallow // check to see if it's a grpc store. - let any_store = store.inner_store(None).as_any(); - if let Some(grpc_store) = any_store.downcast_ref::() { + if let Some(grpc_store) = store.downcast_ref::(None) { return grpc_store.batch_update_blobs(Request::new(request)).await; } - let store_pin = Pin::new(store.as_ref()); + let store_ref = &store; let update_futures: FuturesUnordered<_> = request .requests .into_iter() @@ -133,7 +131,7 @@ impl CasServer { size_bytes, request_data.len() ); - let result = store_pin + let result = store_ref .update_oneshot(digest_info, request_data) .await .err_tip(|| "Error writing to store"); @@ -165,19 +163,18 @@ impl CasServer { // If we are a GrpcStore we shortcut here, as this is a special store. // Note: We don't know the digests here, so we try perform a very shallow // check to see if it's a grpc store. - let any_store = store.inner_store(None).as_any(); - if let Some(grpc_store) = any_store.downcast_ref::() { + if let Some(grpc_store) = store.downcast_ref::(None) { return grpc_store.batch_read_blobs(Request::new(request)).await; } - let store_pin = Pin::new(store.as_ref()); + let store_ref = &store; let read_futures: FuturesUnordered<_> = request .digests .into_iter() .map(|digest| async move { let digest_copy = DigestInfo::try_from(digest.clone())?; // TODO(allada) There is a security risk here of someone taking all the memory on the instance. - let result = store_pin + let result = store_ref .get_part_unchunked(digest_copy, 0, None) .await .err_tip(|| "Error reading from store"); @@ -223,15 +220,13 @@ impl CasServer { // If we are a GrpcStore we shortcut here, as this is a special store. // Note: We don't know the digests here, so we try perform a very shallow // check to see if it's a grpc store. - let any_store = store.inner_store(None).as_any(); - if let Some(grpc_store) = any_store.downcast_ref::() { + if let Some(grpc_store) = store.downcast_ref::(None) { let stream = grpc_store .get_tree(Request::new(request)) .await? .into_inner(); return Ok(Response::new(Box::pin(stream))); } - let store_pin = Pin::new(store.as_ref()); let root_digest: DigestInfo = request .root_digest .err_tip(|| "Expected root_digest to exist in GetTreeRequest")? @@ -260,7 +255,7 @@ impl CasServer { while !deque.is_empty() { let digest: DigestInfo = deque.pop_front().err_tip(|| "In VecDeque::pop_front")?; - let directory = get_and_decode_digest::(store_pin, &digest) + let directory = get_and_decode_digest::(&store, &digest) .await .err_tip(|| "Converting digest to Directory")?; if digest == page_token_digest { diff --git a/nativelink-service/src/execution_server.rs b/nativelink-service/src/execution_server.rs index 31d1d5182..cc5f23481 100644 --- a/nativelink-service/src/execution_server.rs +++ b/nativelink-service/src/execution_server.rs @@ -45,14 +45,10 @@ use tracing::{error_span, event, instrument, Level}; struct InstanceInfo { scheduler: Arc, - cas_store: Arc, + cas_store: Store, } impl InstanceInfo { - fn cas_pin(&self) -> Pin<&dyn Store> { - Pin::new(self.cas_store.as_ref()) - } - async fn build_action_info( &self, instance_name: String, @@ -98,7 +94,8 @@ impl InstanceInfo { // Goma puts the properties in the Command. if platform_properties.is_empty() { - let command = get_and_decode_digest::(self.cas_pin(), &command_digest).await?; + let command = + get_and_decode_digest::(&self.cas_store, &command_digest).await?; if let Some(platform) = &command.platform { for property in &platform.properties { let platform_property = self @@ -212,7 +209,7 @@ impl ExecutionServer { .execution_policy .map_or(DEFAULT_EXECUTION_PRIORITY, |p| p.priority); - let action = get_and_decode_digest::(instance_info.cas_pin(), &digest).await?; + let action = get_and_decode_digest::(&instance_info.cas_store, &digest).await?; let action_info = instance_info .build_action_info( instance_name, diff --git a/nativelink-service/tests/ac_server_test.rs b/nativelink-service/tests/ac_server_test.rs index 326fe2de0..29c719e1c 100644 --- a/nativelink-service/tests/ac_server_test.rs +++ b/nativelink-service/tests/ac_server_test.rs @@ -27,7 +27,7 @@ use nativelink_service::ac_server::AcServer; use nativelink_store::default_store_factory::store_factory; use nativelink_store::store_manager::StoreManager; use nativelink_util::common::DigestInfo; -use nativelink_util::store_trait::Store; +use nativelink_util::store_trait::StoreLike; use prometheus_client::registry::Registry; use prost::Message; use tonic::{Code, Request, Response, Status}; @@ -37,7 +37,7 @@ const HASH1: &str = "0123456789abcdef000000000000000000000000000000000123456789a const HASH1_SIZE: i64 = 147; async fn insert_into_store( - store: Pin<&dyn Store>, + store: Pin<&impl StoreLike>, hash: &str, action_size: i64, action_result: &T, @@ -138,15 +138,14 @@ mod get_action_result { async fn has_single_item() -> Result<(), Box> { let store_manager = make_store_manager().await?; let ac_server = make_ac_server(&store_manager)?; - let ac_store_owned = store_manager.get_store("main_ac").unwrap(); + let ac_store = store_manager.get_store("main_ac").unwrap(); let action_result = ActionResult { exit_code: 45, ..Default::default() }; - let ac_store = Pin::new(ac_store_owned.as_ref()); - insert_into_store(ac_store, HASH1, HASH1_SIZE, &action_result).await?; + insert_into_store(ac_store.as_pin(), HASH1, HASH1_SIZE, &action_result).await?; let raw_response = get_action_result(&ac_server, HASH1, HASH1_SIZE).await; assert!( @@ -161,15 +160,14 @@ mod get_action_result { async fn single_item_wrong_digest_size() -> Result<(), Box> { let store_manager = make_store_manager().await?; let ac_server = make_ac_server(&store_manager)?; - let ac_store_owned = store_manager.get_store("main_ac").unwrap(); + let ac_store = store_manager.get_store("main_ac").unwrap(); let action_result = ActionResult { exit_code: 45, ..Default::default() }; - let ac_store = Pin::new(ac_store_owned.as_ref()); - insert_into_store(ac_store, HASH1, HASH1_SIZE, &action_result).await?; + insert_into_store(ac_store.as_pin(), HASH1, HASH1_SIZE, &action_result).await?; let raw_response = get_action_result(&ac_server, HASH1, HASH1_SIZE - 1).await; let err = raw_response.unwrap_err(); @@ -215,7 +213,7 @@ mod update_action_result { async fn one_item_update_test() -> Result<(), Box> { let store_manager = make_store_manager().await?; let ac_server = make_ac_server(&store_manager)?; - let ac_store_owned = store_manager.get_store("main_ac").unwrap(); + let ac_store = store_manager.get_store("main_ac").unwrap(); let action_result = ActionResult { exit_code: 45, @@ -241,7 +239,6 @@ mod update_action_result { assert_eq!(raw_response.unwrap().into_inner(), action_result); let digest = DigestInfo::try_new(HASH1, size_bytes)?; - let ac_store = Pin::new(ac_store_owned.as_ref()); let raw_data = ac_store.get_part_unchunked(digest, 0, None).await?; let decoded_action_result = ActionResult::decode(raw_data)?; diff --git a/nativelink-service/tests/bytestream_server_test.rs b/nativelink-service/tests/bytestream_server_test.rs index f2a249181..efae5cdee 100644 --- a/nativelink-service/tests/bytestream_server_test.rs +++ b/nativelink-service/tests/bytestream_server_test.rs @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::pin::Pin; use std::sync::Arc; use futures::poll; @@ -27,6 +26,7 @@ use nativelink_store::default_store_factory::store_factory; use nativelink_store::store_manager::StoreManager; use nativelink_util::common::{encode_stream_proto, DigestInfo}; use nativelink_util::spawn; +use nativelink_util::store_trait::StoreLike; use nativelink_util::task::JoinHandleDropGuard; use prometheus_client::registry::Registry; use tokio::task::yield_now; @@ -86,9 +86,7 @@ pub mod write_tests { pub async fn chunked_stream_receives_all_data() -> Result<(), Box> { let store_manager = make_store_manager().await?; let bs_server = make_bytestream_server(store_manager.as_ref())?; - let store_owned = store_manager.get_store("main_cas").unwrap(); - - let store = Pin::new(store_owned.as_ref()); + let store = store_manager.get_store("main_cas").unwrap(); // Setup stream. let (mut tx, join_handle) = { @@ -181,9 +179,7 @@ pub mod write_tests { pub async fn resume_write_success() -> Result<(), Box> { let store_manager = make_store_manager().await?; let bs_server = make_bytestream_server(store_manager.as_ref())?; - let store_owned = store_manager.get_store("main_cas").unwrap(); - - let store = Pin::new(store_owned.as_ref()); + let store = store_manager.get_store("main_cas").unwrap(); async fn setup_stream( bs_server: ByteStreamServer, @@ -277,9 +273,7 @@ pub mod write_tests { pub async fn restart_write_success() -> Result<(), Box> { let store_manager = make_store_manager().await?; let bs_server = make_bytestream_server(store_manager.as_ref())?; - let store_owned = store_manager.get_store("main_cas").unwrap(); - - let store = Pin::new(store_owned.as_ref()); + let store = store_manager.get_store("main_cas").unwrap(); async fn setup_stream( bs_server: ByteStreamServer, @@ -379,9 +373,7 @@ pub mod write_tests { pub async fn restart_mid_stream_write_success() -> Result<(), Box> { let store_manager = make_store_manager().await?; let bs_server = make_bytestream_server(store_manager.as_ref())?; - let store_owned = store_manager.get_store("main_cas").unwrap(); - - let store = Pin::new(store_owned.as_ref()); + let store = store_manager.get_store("main_cas").unwrap(); async fn setup_stream( bs_server: ByteStreamServer, @@ -485,9 +477,7 @@ pub mod write_tests { ) -> Result<(), Box> { let store_manager = make_store_manager().await?; let bs_server = make_bytestream_server(store_manager.as_ref())?; - let store_owned = store_manager.get_store("main_cas").unwrap(); - - let store = Pin::new(store_owned.as_ref()); + let store = store_manager.get_store("main_cas").unwrap(); // Setup stream. let (mut tx, mut write_fut) = { @@ -658,8 +648,7 @@ pub mod write_tests { pub async fn upload_zero_byte_chunk() -> Result<(), Box> { let store_manager = make_store_manager().await?; let bs_server = make_bytestream_server(store_manager.as_ref())?; - let store_owned = store_manager.get_store("main_cas").unwrap(); - let store = Pin::new(store_owned.as_ref()); + let store = store_manager.get_store("main_cas").unwrap(); async fn setup_stream( bs_server: ByteStreamServer, @@ -854,9 +843,7 @@ pub mod read_tests { { let store_manager = make_store_manager().await?; let bs_server = make_bytestream_server(store_manager.as_ref())?; - let store_owned = store_manager.get_store("main_cas").unwrap(); - - let store = Pin::new(store_owned.as_ref()); + let store = store_manager.get_store("main_cas").unwrap(); const VALUE1: &str = "12456789abcdefghijk"; @@ -890,9 +877,7 @@ pub mod read_tests { pub async fn chunked_stream_reads_10mb_of_data() -> Result<(), Box> { let store_manager = make_store_manager().await?; let bs_server = make_bytestream_server(store_manager.as_ref())?; - let store_owned = store_manager.get_store("main_cas").unwrap(); - - let store = Pin::new(store_owned.as_ref()); + let store = store_manager.get_store("main_cas").unwrap(); const DATA_SIZE: usize = 10_000_000; let mut raw_data = vec![41u8; DATA_SIZE]; diff --git a/nativelink-service/tests/cas_server_test.rs b/nativelink-service/tests/cas_server_test.rs index 937122427..439ed9062 100644 --- a/nativelink-service/tests/cas_server_test.rs +++ b/nativelink-service/tests/cas_server_test.rs @@ -26,6 +26,7 @@ use nativelink_store::ac_utils::serialize_and_upload_message; use nativelink_store::default_store_factory::store_factory; use nativelink_store::store_manager::StoreManager; use nativelink_util::common::DigestInfo; +use nativelink_util::store_trait::StoreLike; use prometheus_client::registry::Registry; use tonic::Request; @@ -95,11 +96,10 @@ mod find_missing_blobs { async fn store_one_item_existence() -> Result<(), Box> { let store_manager = make_store_manager().await?; let cas_server = make_cas_server(&store_manager)?; - let store_owned = store_manager.get_store("main_cas").unwrap(); + let store = store_manager.get_store("main_cas").unwrap(); const VALUE: &str = "1"; - let store = Pin::new(store_owned.as_ref()); store .update_oneshot(DigestInfo::try_new(HASH1, VALUE.len())?, VALUE.into()) .await?; @@ -123,11 +123,10 @@ mod find_missing_blobs { async fn has_three_requests_one_bad_hash() -> Result<(), Box> { let store_manager = make_store_manager().await?; let cas_server = make_cas_server(&store_manager)?; - let store_owned = store_manager.get_store("main_cas").unwrap(); + let store = store_manager.get_store("main_cas").unwrap(); const VALUE: &str = "1"; - let store = Pin::new(store_owned.as_ref()); store .update_oneshot(DigestInfo::try_new(HASH1, VALUE.len())?, VALUE.into()) .await?; @@ -174,7 +173,7 @@ mod batch_update_blobs { async fn update_existing_item() -> Result<(), Box> { let store_manager = make_store_manager().await?; let cas_server = make_cas_server(&store_manager)?; - let store_owned = store_manager.get_store("main_cas").unwrap(); + let store = store_manager.get_store("main_cas").unwrap(); const VALUE1: &str = "1"; const VALUE2: &str = "2"; @@ -184,7 +183,6 @@ mod batch_update_blobs { size_bytes: VALUE2.len() as i64, }; - let store = Pin::new(store_owned.as_ref()); store .update_oneshot(DigestInfo::try_new(HASH1, VALUE1.len())?, VALUE1.into()) .await @@ -243,7 +241,7 @@ mod batch_read_blobs { ) -> Result<(), Box> { let store_manager = make_store_manager().await?; let cas_server = make_cas_server(&store_manager)?; - let store_owned = store_manager.get_store("main_cas").unwrap(); + let store = store_manager.get_store("main_cas").unwrap(); const VALUE1: &str = "1"; const VALUE2: &str = "23"; @@ -258,7 +256,6 @@ mod batch_read_blobs { }; { // Insert dummy data. - let store = Pin::new(store_owned.as_ref()); store .update_oneshot(DigestInfo::try_new(HASH1, VALUE1.len())?, VALUE1.into()) .await @@ -331,7 +328,6 @@ mod get_tree { digest_function, Directory, DirectoryNode, GetTreeRequest, GetTreeResponse, NodeProperties, }; use nativelink_util::digest_hasher::DigestHasherFunc; - use nativelink_util::store_trait::Store; use pretty_assertions::assert_eq; // Must be declared in every module. use prost_types::Timestamp; @@ -344,7 +340,7 @@ mod get_tree { sub_directory_digest_infos: Vec, } async fn setup_directory_structure( - store_pinned: Pin<&dyn Store>, + store_pinned: Pin<&impl StoreLike>, ) -> Result { // Set up 5 sub-directories. const SUB_DIRECTORIES_LENGTH: i32 = 5; @@ -406,8 +402,7 @@ mod get_tree { async fn get_tree_read_directories_without_paging() -> Result<(), Box> { let store_manager = make_store_manager().await?; let cas_server = make_cas_server(&store_manager)?; - let store_owned = store_manager.get_store("main_cas").unwrap(); - let store_pinned = Pin::new(store_owned.as_ref()); + let store = store_manager.get_store("main_cas").unwrap(); // Setup directory structure. let SetupDirectoryResult { @@ -415,7 +410,7 @@ mod get_tree { root_directory_digest_info, sub_directories, sub_directory_digest_infos: _, - } = setup_directory_structure(store_pinned).await?; + } = setup_directory_structure(store.as_pin()).await?; // Must work when paging is disabled ( `page_size` is 0 ). // It reads all directories at once. @@ -460,8 +455,7 @@ mod get_tree { async fn get_tree_read_directories_with_paging() -> Result<(), Box> { let store_manager = make_store_manager().await?; let cas_server = make_cas_server(&store_manager)?; - let store_owned = store_manager.get_store("main_cas").unwrap(); - let store_pinned = Pin::new(store_owned.as_ref()); + let store = store_manager.get_store("main_cas").unwrap(); // Setup directory structure. let SetupDirectoryResult { @@ -469,7 +463,7 @@ mod get_tree { root_directory_digest_info, sub_directories, sub_directory_digest_infos, - } = setup_directory_structure(store_pinned).await?; + } = setup_directory_structure(store.as_pin()).await?; // Must work when paging is enabled ( `page_size` is 2 ). // First, it reads `root_directory` and `sub_directory[0]`. diff --git a/nativelink-store/src/ac_utils.rs b/nativelink-store/src/ac_utils.rs index 2c4b3f4a2..f5760f55d 100644 --- a/nativelink-store/src/ac_utils.rs +++ b/nativelink-store/src/ac_utils.rs @@ -24,7 +24,7 @@ use futures::TryFutureExt; use nativelink_error::{Code, Error, ResultExt}; use nativelink_util::common::DigestInfo; use nativelink_util::digest_hasher::DigestHasher; -use nativelink_util::store_trait::Store; +use nativelink_util::store_trait::StoreLike; use prost::Message; // NOTE(blaise.bruer) From some local testing it looks like action cache items are rarely greater than @@ -38,7 +38,7 @@ const MAX_ACTION_MSG_SIZE: usize = 10 << 20; // 10mb. /// Attempts to fetch the digest contents from a store into the associated proto. pub async fn get_and_decode_digest( - store: Pin<&dyn Store>, + store: &impl StoreLike, digest: &DigestInfo, ) -> Result { get_size_and_decode_digest(store, digest) @@ -47,8 +47,8 @@ pub async fn get_and_decode_digest( } /// Attempts to fetch the digest contents from a store into the associated proto. -pub async fn get_size_and_decode_digest( - store: Pin<&dyn Store>, +pub async fn get_size_and_decode_digest<'a, T: Message + Default>( + store: &impl StoreLike, digest: &DigestInfo, ) -> Result<(T, usize), Error> { let mut store_data_resp = store @@ -91,7 +91,7 @@ pub fn message_to_digest( /// Takes a proto message and will serialize it and upload it to the provided store. pub async fn serialize_and_upload_message<'a, T: Message>( message: &'a T, - cas_store: Pin<&'a dyn Store>, + cas_store: Pin<&'a impl StoreLike>, hasher: &mut impl DigestHasher, ) -> Result { let mut buffer = BytesMut::with_capacity(message.encoded_len()); diff --git a/nativelink-store/src/completeness_checking_store.rs b/nativelink-store/src/completeness_checking_store.rs index d0656ef2c..12de40079 100644 --- a/nativelink-store/src/completeness_checking_store.rs +++ b/nativelink-store/src/completeness_checking_store.rs @@ -29,7 +29,7 @@ use nativelink_util::health_utils::{default_health_status_indicator, HealthStatu use nativelink_util::metrics_utils::{ Collector, CollectorState, CounterWithTime, MetricsComponent, Registry, }; -use nativelink_util::store_trait::{Store, UploadSizeInfo}; +use nativelink_util::store_trait::{Store, StoreDriver, StoreLike, UploadSizeInfo}; use parking_lot::Mutex; use tokio::sync::Notify; use tracing::{event, Level}; @@ -62,7 +62,7 @@ fn get_digests_and_output_dirs( /// that need to be checked and pass them into `handle_digest_infos_fn` /// as they are found. async fn check_output_directories( - cas_store: Pin<&dyn Store>, + cas_store: &Store, output_directories: Vec, handle_digest_infos_fn: &impl Fn(Vec), ) -> Result<(), Error> { @@ -106,15 +106,15 @@ async fn check_output_directories( } pub struct CompletenessCheckingStore { - cas_store: Arc, - ac_store: Arc, + cas_store: Store, + ac_store: Store, incomplete_entries_counter: CounterWithTime, complete_entries_counter: CounterWithTime, } impl CompletenessCheckingStore { - pub fn new(ac_store: Arc, cas_store: Arc) -> Self { + pub fn new(ac_store: Store, cas_store: Store) -> Self { CompletenessCheckingStore { cas_store, ac_store, @@ -132,8 +132,6 @@ impl CompletenessCheckingStore { action_result_digests: &[DigestInfo], results: &mut [Option], ) -> Result<(), Error> { - let ac_store = Pin::new(self.ac_store.as_ref()); - let cas_store = Pin::new(self.cas_store.as_ref()); // Holds shared state between the different futures. // This is how get around lifetime issues. struct State<'a> { @@ -164,7 +162,8 @@ impl CompletenessCheckingStore { async move { // Note: We don't err_tip here because often have NotFound here which is ok. let (action_result, size) = - get_size_and_decode_digest::(ac_store, digest).await?; + get_size_and_decode_digest::(&self.ac_store, digest) + .await?; let (mut digest_infos, output_directories) = get_digests_and_output_dirs(action_result)?; @@ -197,15 +196,19 @@ impl CompletenessCheckingStore { return Ok(()); } - check_output_directories(cas_store, output_directories, &move |digest_infos| { - let mut state = state_mux.lock(); - let rep_len = digest_infos.len(); - state.digests_to_check.extend(digest_infos); - state - .digests_to_check_idxs - .extend(iter::repeat(i).take(rep_len)); - state.notify.notify_one(); - }) + check_output_directories( + &self.cas_store, + output_directories, + &move |digest_infos| { + let mut state = state_mux.lock(); + let rep_len = digest_infos.len(); + state.digests_to_check.extend(digest_infos); + state + .digests_to_check_idxs + .extend(iter::repeat(i).take(rep_len)); + state.notify.notify_one(); + }, + ) .await?; Result::<(), Error>::Ok(()) @@ -262,7 +265,7 @@ impl CompletenessCheckingStore { // Recycle our results vector to avoid needless allocations. has_results.clear(); has_results.resize(digests.len(), None); - cas_store + self.cas_store .has_with_results(&digests, &mut has_results[..]) .await .err_tip(|| { @@ -331,7 +334,7 @@ impl CompletenessCheckingStore { } #[async_trait] -impl Store for CompletenessCheckingStore { +impl StoreDriver for CompletenessCheckingStore { async fn has_with_results( self: Pin<&Self>, action_result_digests: &[DigestInfo], @@ -347,12 +350,10 @@ impl Store for CompletenessCheckingStore { reader: DropCloserReadHalf, size_info: UploadSizeInfo, ) -> Result<(), Error> { - Pin::new(self.ac_store.as_ref()) - .update(digest, reader, size_info) - .await + self.ac_store.update(digest, reader, size_info).await } - async fn get_part_ref( + async fn get_part( self: Pin<&Self>, digest: DigestInfo, writer: &mut DropCloserWriteHalf, @@ -362,23 +363,17 @@ impl Store for CompletenessCheckingStore { let results = &mut [None]; self.inner_has_with_results(&[digest], results) .await - .err_tip(|| "when calling CompletenessCheckingStore::get_part_ref")?; + .err_tip(|| "when calling CompletenessCheckingStore::get_part")?; if results[0].is_none() { return Err(make_err!( Code::NotFound, - "Digest found, but not all parts were found in CompletenessCheckingStore::get_part_ref" + "Digest found, but not all parts were found in CompletenessCheckingStore::get_part" )); } - Pin::new(self.ac_store.as_ref()) - .get_part_ref(digest, writer, offset, length) - .await - } - - fn inner_store(&self, _digest: Option) -> &'_ dyn Store { - self + self.ac_store.get_part(digest, writer, offset, length).await } - fn inner_store_arc(self: Arc, _digest: Option) -> Arc { + fn inner_store(&self, _digest: Option) -> &dyn StoreDriver { self } @@ -392,10 +387,8 @@ impl Store for CompletenessCheckingStore { fn register_metrics(self: Arc, registry: &mut Registry) { self.cas_store - .clone() .register_metrics(registry.sub_registry_with_prefix("cas_store")); self.ac_store - .clone() .register_metrics(registry.sub_registry_with_prefix("ac_store")); registry.register_collector(Box::new(Collector::new(&self))); } diff --git a/nativelink-store/src/compression_store.rs b/nativelink-store/src/compression_store.rs index fcbae2a8f..fc204df4d 100644 --- a/nativelink-store/src/compression_store.rs +++ b/nativelink-store/src/compression_store.rs @@ -31,7 +31,7 @@ use nativelink_util::common::DigestInfo; use nativelink_util::health_utils::{default_health_status_indicator, HealthStatusIndicator}; use nativelink_util::metrics_utils::Registry; use nativelink_util::spawn; -use nativelink_util::store_trait::{Store, UploadSizeInfo}; +use nativelink_util::store_trait::{Store, StoreDriver, StoreLike, UploadSizeInfo}; use serde::{Deserialize, Serialize}; use crate::cas_utils::is_zero_digest; @@ -211,7 +211,7 @@ impl UploadState { /// result in the entire contents being read from the inner store but will /// only send the contents requested. pub struct CompressionStore { - inner_store: Arc, + inner_store: Store, config: nativelink_config::stores::Lz4Config, bincode_options: BincodeOptions, } @@ -219,7 +219,7 @@ pub struct CompressionStore { impl CompressionStore { pub fn new( compression_config: nativelink_config::stores::CompressionStore, - inner_store: Arc, + inner_store: Store, ) -> Result { let lz4_config = match compression_config.compression_algorithm { nativelink_config::stores::CompressionAlgorithm::lz4(mut lz4_config) => { @@ -241,15 +241,13 @@ impl CompressionStore { } #[async_trait] -impl Store for CompressionStore { +impl StoreDriver for CompressionStore { async fn has_with_results( self: Pin<&Self>, digests: &[DigestInfo], results: &mut [Option], ) -> Result<(), Error> { - Pin::new(self.inner_store.as_ref()) - .has_with_results(digests, results) - .await + self.inner_store.has_with_results(digests, results).await } async fn update( @@ -264,7 +262,7 @@ impl Store for CompressionStore { let inner_store = self.inner_store.clone(); let update_fut = spawn!("compression_store_update_spawn", async move { - Pin::new(inner_store.as_ref()) + inner_store .update( digest, rx, @@ -388,7 +386,7 @@ impl Store for CompressionStore { write_result.merge(update_result) } - async fn get_part_ref( + async fn get_part( self: Pin<&Self>, digest: DigestInfo, writer: &mut DropCloserWriteHalf, @@ -398,7 +396,7 @@ impl Store for CompressionStore { if is_zero_digest(&digest) { writer .send_eof() - .err_tip(|| "Failed to send zero EOF in filesystem store get_part_ref")?; + .err_tip(|| "Failed to send zero EOF in filesystem store get_part")?; return Ok(()); } @@ -407,7 +405,7 @@ impl Store for CompressionStore { let inner_store = self.inner_store.clone(); let get_part_fut = spawn!("compression_store_get_part_spawn", async move { - Pin::new(inner_store.as_ref()) + inner_store .get_part(digest, tx, 0, None) .await .err_tip(|| "Inner store get in compression store failed") @@ -613,11 +611,7 @@ impl Store for CompressionStore { Ok(()) } - fn inner_store(&self, _digest: Option) -> &'_ dyn Store { - self - } - - fn inner_store_arc(self: Arc, _digest: Option) -> Arc { + fn inner_store(&self, _digest: Option) -> &dyn StoreDriver { self } @@ -631,9 +625,7 @@ impl Store for CompressionStore { fn register_metrics(self: Arc, registry: &mut Registry) { let inner_store_registry = registry.sub_registry_with_prefix("inner_store"); - self.inner_store - .clone() - .register_metrics(inner_store_registry); + self.inner_store.register_metrics(inner_store_registry); } } diff --git a/nativelink-store/src/dedup_store.rs b/nativelink-store/src/dedup_store.rs index 799f00285..1e996792f 100644 --- a/nativelink-store/src/dedup_store.rs +++ b/nativelink-store/src/dedup_store.rs @@ -25,7 +25,7 @@ use nativelink_util::buf_channel::{DropCloserReadHalf, DropCloserWriteHalf}; use nativelink_util::common::DigestInfo; use nativelink_util::fastcdc::FastCDC; use nativelink_util::health_utils::{default_health_status_indicator, HealthStatusIndicator}; -use nativelink_util::store_trait::{Store, UploadSizeInfo}; +use nativelink_util::store_trait::{Store, StoreDriver, StoreLike, UploadSizeInfo}; use serde::{Deserialize, Serialize}; use tokio_util::codec::FramedRead; use tokio_util::io::StreamReader; @@ -44,8 +44,8 @@ pub struct DedupIndex { } pub struct DedupStore { - index_store: Arc, - content_store: Arc, + index_store: Store, + content_store: Store, fast_cdc_decoder: FastCDC, max_concurrent_fetch_per_get: usize, bincode_options: WithOtherIntEncoding, @@ -54,8 +54,8 @@ pub struct DedupStore { impl DedupStore { pub fn new( config: &nativelink_config::stores::DedupStore, - index_store: Arc, - content_store: Arc, + index_store: Store, + content_store: Store, ) -> Self { let min_size = if config.min_size == 0 { DEFAULT_MIN_SIZE @@ -86,16 +86,12 @@ impl DedupStore { } } - fn pin_index_store(&self) -> Pin<&dyn Store> { - Pin::new(self.index_store.as_ref()) - } - async fn has(self: Pin<&Self>, digest: DigestInfo) -> Result, Error> { // First we need to load the index that contains where the individual parts actually // can be fetched from. let index_entries = { let maybe_data = self - .pin_index_store() + .index_store .get_part_unchunked(digest, 0, None) .await .err_tip(|| "Failed to read index store in dedup store"); @@ -130,10 +126,7 @@ impl DedupStore { .map(|index_entry| DigestInfo::new(index_entry.packed_hash, index_entry.size_bytes)) .collect(); let mut sum = 0; - for size in Pin::new(self.content_store.as_ref()) - .has_many(&digests) - .await? - { + for size in self.content_store.has_many(&digests).await? { let Some(size) = size else { // A part is missing so return None meaning not-found. // This will abort all in-flight queries related to this request. @@ -146,7 +139,7 @@ impl DedupStore { } #[async_trait] -impl Store for DedupStore { +impl StoreDriver for DedupStore { async fn has_with_results( self: Pin<&Self>, digests: &[DigestInfo], @@ -178,13 +171,13 @@ impl Store for DedupStore { ) -> Result<(), Error> { let mut bytes_reader = StreamReader::new(reader); let frame_reader = FramedRead::new(&mut bytes_reader, self.fast_cdc_decoder.clone()); - let content_store_pin = Pin::new(self.content_store.as_ref()); let index_entries = frame_reader .map(|r| r.err_tip(|| "Failed to decode frame from fast_cdc")) .map_ok(|frame| async move { let hash = blake3::hash(&frame[..]).into(); let index_entry = DigestInfo::new(hash, frame.len() as i64); - if content_store_pin + if self + .content_store .has(index_entry) .await .err_tip(|| "Failed to call .has() in DedupStore::update()")? @@ -193,7 +186,7 @@ impl Store for DedupStore { // If our store has this digest, we don't need to upload it. return Result::<_, Error>::Ok(index_entry); } - content_store_pin + self.content_store .update_oneshot(index_entry, frame) .await .err_tip(|| "Failed to update content store in dedup_store")?; @@ -216,7 +209,7 @@ impl Store for DedupStore { ) })?; - self.pin_index_store() + self.index_store .update_oneshot(digest, serialized_index.into()) .await .err_tip(|| "Failed to insert our index entry to index_store in dedup_store")?; @@ -224,7 +217,7 @@ impl Store for DedupStore { Ok(()) } - async fn get_part_ref( + async fn get_part( self: Pin<&Self>, digest: DigestInfo, writer: &mut DropCloserWriteHalf, @@ -242,7 +235,7 @@ impl Store for DedupStore { // can be fetched from. let index_entries = { let data = self - .pin_index_store() + .index_store .get_part_unchunked(digest, 0, None) .await .err_tip(|| "Failed to read index store in dedup store")?; @@ -296,17 +289,14 @@ impl Store for DedupStore { // Note: We will buffer our data here up to: // `config.max_size * config.max_concurrent_fetch_per_get` per `get_part()` request. let mut entries_stream = stream::iter(entries) - .map(move |index_entry| { - let content_store = self.content_store.clone(); - - async move { - let data = Pin::new(content_store.as_ref()) - .get_part_unchunked(index_entry, 0, None) - .await - .err_tip(|| "Failed to get_part in content_store in dedup_store")?; + .map(move |index_entry| async move { + let data = self + .content_store + .get_part_unchunked(index_entry, 0, None) + .await + .err_tip(|| "Failed to get_part in content_store in dedup_store")?; - Result::<_, Error>::Ok(data) - } + Result::<_, Error>::Ok(data) }) .buffered(self.max_concurrent_fetch_per_get); @@ -343,11 +333,7 @@ impl Store for DedupStore { Ok(()) } - fn inner_store(&self, _digest: Option) -> &'_ dyn Store { - self - } - - fn inner_store_arc(self: Arc, _digest: Option) -> Arc { + fn inner_store(&self, _digest: Option) -> &dyn StoreDriver { self } diff --git a/nativelink-store/src/default_store_factory.rs b/nativelink-store/src/default_store_factory.rs index 32a53031b..646de0860 100644 --- a/nativelink-store/src/default_store_factory.rs +++ b/nativelink-store/src/default_store_factory.rs @@ -21,7 +21,7 @@ use nativelink_config::stores::StoreConfig; use nativelink_error::Error; use nativelink_util::health_utils::HealthRegistryBuilder; use nativelink_util::metrics_utils::Registry; -use nativelink_util::store_trait::Store; +use nativelink_util::store_trait::{Store, StoreDriver}; use crate::completeness_checking_store::CompletenessCheckingStore; use crate::compression_store::CompressionStore; @@ -40,7 +40,7 @@ use crate::size_partitioning_store::SizePartitioningStore; use crate::store_manager::StoreManager; use crate::verify_store::VerifyStore; -type FutureMaybeStore<'a> = Box, Error>> + 'a>; +type FutureMaybeStore<'a> = Box> + 'a>; pub fn store_factory<'a>( backend: &'a StoreConfig, @@ -49,7 +49,7 @@ pub fn store_factory<'a>( maybe_health_registry_builder: Option<&'a mut HealthRegistryBuilder>, ) -> Pin> { Box::pin(async move { - let store: Arc = match backend { + let store: Arc = match backend { StoreConfig::memory(config) => Arc::new(MemoryStore::new(config)), StoreConfig::experimental_s3_store(config) => Arc::new(S3Store::new(config).await?), StoreConfig::redis_store(config) => Arc::new(RedisStore::new(config)?), @@ -74,12 +74,12 @@ pub fn store_factory<'a>( store_factory(&config.backend, store_manager, None, None).await?, store_factory(&config.cas_store, store_manager, None, None).await?, )), - StoreConfig::fast_slow(config) => Arc::new(FastSlowStore::new( + StoreConfig::fast_slow(config) => FastSlowStore::new( config, store_factory(&config.fast, store_manager, None, None).await?, store_factory(&config.slow, store_manager, None, None).await?, - )), - StoreConfig::filesystem(config) => Arc::new(::new(config).await?), + ), + StoreConfig::filesystem(config) => ::new(config).await?, StoreConfig::ref_store(config) => { Arc::new(RefStore::new(config, Arc::downgrade(store_manager))) } @@ -111,6 +111,6 @@ pub fn store_factory<'a>( store.clone().register_health(health_registry_builder); } - Ok(store) + Ok(Store::new(store)) }) } diff --git a/nativelink-store/src/existence_cache_store.rs b/nativelink-store/src/existence_cache_store.rs index 4a1c98699..77b87bb66 100644 --- a/nativelink-store/src/existence_cache_store.rs +++ b/nativelink-store/src/existence_cache_store.rs @@ -24,7 +24,7 @@ use nativelink_util::common::DigestInfo; use nativelink_util::evicting_map::{EvictingMap, LenEntry}; use nativelink_util::health_utils::{default_health_status_indicator, HealthStatusIndicator}; use nativelink_util::metrics_utils::{CollectorState, MetricsComponent, Registry}; -use nativelink_util::store_trait::{Store, UploadSizeInfo}; +use nativelink_util::store_trait::{Store, StoreDriver, StoreLike, UploadSizeInfo}; #[derive(Clone, Debug)] struct ExistanceItem(usize); @@ -42,12 +42,12 @@ impl LenEntry for ExistanceItem { } pub struct ExistenceCacheStore { - inner_store: Arc, + inner_store: Store, existence_cache: EvictingMap, } impl ExistenceCacheStore { - pub fn new(config: &ExistenceCacheStoreConfig, inner_store: Arc) -> Self { + pub fn new(config: &ExistenceCacheStoreConfig, inner_store: Store) -> Self { let empty_policy = EvictionPolicy::default(); let eviction_policy = config.eviction_policy.as_ref().unwrap_or(&empty_policy); Self { @@ -84,7 +84,7 @@ impl ExistenceCacheStore { // Now query only the items not found in the cache. let mut inner_results = vec![None; not_cached_digests.len()]; - self.pin_inner() + self.inner_store .has_with_results(¬_cached_digests, &mut inner_results) .await .err_tip(|| "In ExistenceCacheStore::inner_has_with_results")?; @@ -122,14 +122,10 @@ impl ExistenceCacheStore { Ok(()) } - - fn pin_inner(&self) -> Pin<&dyn Store> { - Pin::new(self.inner_store.as_ref()) - } } #[async_trait] -impl Store for ExistenceCacheStore { +impl StoreDriver for ExistenceCacheStore { async fn has_with_results( self: Pin<&Self>, digests: &[DigestInfo], @@ -157,7 +153,7 @@ impl Store for ExistenceCacheStore { .err_tip(|| "In ExistenceCacheStore::update")?; return Ok(()); } - let result = self.pin_inner().update(digest, reader, size_info).await; + let result = self.inner_store.update(digest, reader, size_info).await; if result.is_ok() { if let UploadSizeInfo::ExactSize(size) = size_info { let _ = self @@ -169,7 +165,7 @@ impl Store for ExistenceCacheStore { result } - async fn get_part_ref( + async fn get_part( self: Pin<&Self>, digest: DigestInfo, writer: &mut DropCloserWriteHalf, @@ -177,8 +173,8 @@ impl Store for ExistenceCacheStore { length: Option, ) -> Result<(), Error> { let result = self - .pin_inner() - .get_part_ref(digest, writer, offset, length) + .inner_store + .get_part(digest, writer, offset, length) .await; if result.is_ok() { let size = usize::try_from(digest.size_bytes) @@ -191,11 +187,7 @@ impl Store for ExistenceCacheStore { result } - fn inner_store(&self, _digest: Option) -> &'_ dyn Store { - self - } - - fn inner_store_arc(self: Arc, _digest: Option) -> Arc { + fn inner_store(&self, _digest: Option) -> &dyn StoreDriver { self } @@ -209,9 +201,7 @@ impl Store for ExistenceCacheStore { fn register_metrics(self: Arc, registry: &mut Registry) { let inner_store_registry = registry.sub_registry_with_prefix("inner_store"); - self.inner_store - .clone() - .register_metrics(inner_store_registry); + self.inner_store.register_metrics(inner_store_registry); } } diff --git a/nativelink-store/src/fast_slow_store.rs b/nativelink-store/src/fast_slow_store.rs index fa37b0f33..a57bbd45e 100644 --- a/nativelink-store/src/fast_slow_store.rs +++ b/nativelink-store/src/fast_slow_store.rs @@ -12,11 +12,12 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::borrow::BorrowMut; use std::cmp::{max, min}; use std::ops::Range; use std::pin::Pin; use std::sync::atomic::{AtomicU64, Ordering}; -use std::sync::Arc; +use std::sync::{Arc, Weak}; use async_trait::async_trait; use futures::{join, FutureExt}; @@ -29,7 +30,7 @@ use nativelink_util::fs; use nativelink_util::health_utils::{default_health_status_indicator, HealthStatusIndicator}; use nativelink_util::metrics_utils::{CollectorState, MetricsComponent, Registry}; use nativelink_util::store_trait::{ - slow_update_store_with_file, Store, StoreOptimizations, UploadSizeInfo, + slow_update_store_with_file, Store, StoreDriver, StoreLike, StoreOptimizations, UploadSizeInfo, }; // TODO(blaise.bruer) This store needs to be evaluated for more efficient memory usage, @@ -40,40 +41,45 @@ use nativelink_util::store_trait::{ // "BufferedStore" that could be placed on the "slow" store that would hang up early // if data is in the buffer. pub struct FastSlowStore { - fast_store: Arc, - slow_store: Arc, - + fast_store: Store, + slow_store: Store, + weak_self: Weak, metrics: FastSlowStoreMetrics, } impl FastSlowStore { pub fn new( _config: &nativelink_config::stores::FastSlowStore, - fast_store: Arc, - slow_store: Arc, - ) -> Self { - Self { + fast_store: Store, + slow_store: Store, + ) -> Arc { + Arc::new_cyclic(|weak_self| Self { fast_store, slow_store, + weak_self: weak_self.clone(), metrics: FastSlowStoreMetrics::default(), - } + }) } - pub fn fast_store(&self) -> &Arc { + pub fn fast_store(&self) -> &Store { &self.fast_store } - pub fn slow_store(&self) -> &Arc { + pub fn slow_store(&self) -> &Store { &self.slow_store } + pub fn get_arc(&self) -> Option> { + self.weak_self.upgrade() + } + /// Ensure our fast store is populated. This should be kept as a low /// cost function. Since the data itself is shared and not copied it should be fairly /// low cost to just discard the data, but does cost a few mutex locks while /// streaming. - pub async fn populate_fast_store(self: Pin<&Self>, digest: DigestInfo) -> Result<(), Error> { + pub async fn populate_fast_store(&self, digest: DigestInfo) -> Result<(), Error> { let maybe_size_info = self - .pin_fast_store() + .fast_store .has(digest) .await .err_tip(|| "While querying in populate_fast_store")?; @@ -88,18 +94,10 @@ impl FastSlowStore { while !rx.recv().await?.is_empty() {} Ok(()) }; - let (drain_res, get_res) = join!(drain_fut, self.get(digest, tx)); + let (drain_res, get_res) = join!(drain_fut, StoreDriver::get(Pin::new(self), digest, tx)); get_res.err_tip(|| "Failed to populate()").merge(drain_res) } - fn pin_fast_store(&self) -> Pin<&dyn Store> { - Pin::new(self.fast_store.as_ref()) - } - - fn pin_slow_store(&self) -> Pin<&dyn Store> { - Pin::new(self.slow_store.as_ref()) - } - /// Returns the range of bytes that should be sent given a slice bounds /// offset so the output range maps the received_range.start to 0. // TODO(allada) This should be put into utils, as this logic is used @@ -125,7 +123,7 @@ impl FastSlowStore { } #[async_trait] -impl Store for FastSlowStore { +impl StoreDriver for FastSlowStore { async fn has_with_results( self: Pin<&Self>, digests: &[DigestInfo], @@ -135,18 +133,13 @@ impl Store for FastSlowStore { // so only check the fast store in such case. let slow_store = self.slow_store.inner_store(None); if slow_store.optimized_for(StoreOptimizations::NoopDownloads) { - return self - .pin_fast_store() - .has_with_results(digests, results) - .await; + return self.fast_store.has_with_results(digests, results).await; } // Only check the slow store because if it's not there, then something // down stream might be unable to get it. This should not affect // workers as they only use get() and a CAS can use an // ExistenceCacheStore to avoid the bottleneck. - self.pin_slow_store() - .has_with_results(digests, results) - .await + self.slow_store.has_with_results(digests, results).await } async fn update( @@ -159,17 +152,11 @@ impl Store for FastSlowStore { // and just use the store that is not a noop store. let slow_store = self.slow_store.inner_store(Some(digest)); if slow_store.optimized_for(StoreOptimizations::NoopUpdates) { - return self - .pin_fast_store() - .update(digest, reader, size_info) - .await; + return self.fast_store.update(digest, reader, size_info).await; } let fast_store = self.fast_store.inner_store(Some(digest)); if fast_store.optimized_for(StoreOptimizations::NoopUpdates) { - return self - .pin_slow_store() - .update(digest, reader, size_info) - .await; + return self.slow_store.update(digest, reader, size_info).await; } let (mut fast_tx, fast_rx) = make_buf_channel_pair(); @@ -212,8 +199,8 @@ impl Store for FastSlowStore { } }; - let fast_store_fut = self.pin_fast_store().update(digest, fast_rx, size_info); - let slow_store_fut = self.pin_slow_store().update(digest, slow_rx, size_info); + let fast_store_fut = self.fast_store.update(digest, fast_rx, size_info); + let slow_store_fut = self.slow_store.update(digest, slow_rx, size_info); let (data_stream_res, fast_res, slow_res) = join!(data_stream_fut, fast_store_fut, slow_store_fut); @@ -235,26 +222,48 @@ impl Store for FastSlowStore { mut file: fs::ResumeableFileSlot, upload_size: UploadSizeInfo, ) -> Result, Error> { - let fast_store = self.fast_store.inner_store(Some(digest)); - let slow_store = self.slow_store.inner_store(Some(digest)); - if fast_store.optimized_for(StoreOptimizations::FileUpdates) { - if !slow_store.optimized_for(StoreOptimizations::NoopUpdates) { - slow_update_store_with_file(Pin::new(slow_store), digest, &mut file, upload_size) - .await - .err_tip(|| "In FastSlowStore::update_with_whole_file slow_store")?; + if self + .fast_store + .optimized_for(StoreOptimizations::FileUpdates) + { + if !self + .slow_store + .optimized_for(StoreOptimizations::NoopUpdates) + { + slow_update_store_with_file( + self.slow_store.as_store_driver_pin(), + digest, + &mut file, + upload_size, + ) + .await + .err_tip(|| "In FastSlowStore::update_with_whole_file slow_store")?; } - return Pin::new(fast_store) + return self + .fast_store .update_with_whole_file(digest, file, upload_size) .await; } - if slow_store.optimized_for(StoreOptimizations::FileUpdates) { - if !fast_store.optimized_for(StoreOptimizations::NoopUpdates) { - slow_update_store_with_file(Pin::new(fast_store), digest, &mut file, upload_size) - .await - .err_tip(|| "In FastSlowStore::update_with_whole_file fast_store")?; + if self + .slow_store + .optimized_for(StoreOptimizations::FileUpdates) + { + if !self + .fast_store + .optimized_for(StoreOptimizations::NoopUpdates) + { + slow_update_store_with_file( + self.fast_store.as_store_driver_pin(), + digest, + &mut file, + upload_size, + ) + .await + .err_tip(|| "In FastSlowStore::update_with_whole_file fast_store")?; } - return Pin::new(slow_store) + return self + .slow_store .update_with_whole_file(digest, file, upload_size) .await; } @@ -265,7 +274,7 @@ impl Store for FastSlowStore { Ok(Some(file)) } - async fn get_part_ref( + async fn get_part( self: Pin<&Self>, digest: DigestInfo, writer: &mut DropCloserWriteHalf, @@ -274,14 +283,12 @@ impl Store for FastSlowStore { ) -> Result<(), Error> { // TODO(blaise.bruer) Investigate if we should maybe ignore errors here instead of // forwarding the up. - let fast_store = self.pin_fast_store(); - let slow_store = self.pin_slow_store(); - if fast_store.has(digest).await?.is_some() { + if self.fast_store.has(digest).await?.is_some() { self.metrics .fast_store_hit_count .fetch_add(1, Ordering::Acquire); - fast_store - .get_part_ref(digest, writer, offset, length) + self.fast_store + .get_part(digest, writer.borrow_mut(), offset, length) .await?; self.metrics .fast_store_downloaded_bytes @@ -289,7 +296,8 @@ impl Store for FastSlowStore { return Ok(()); } - let sz = slow_store + let sz = self + .slow_store .has(digest) .await .err_tip(|| "Failed to run has() on slow store")? @@ -343,8 +351,10 @@ impl Store for FastSlowStore { } }; - let slow_store_fut = slow_store.get(digest, slow_tx); - let fast_store_fut = fast_store.update(digest, fast_rx, UploadSizeInfo::ExactSize(sz)); + let slow_store_fut = self.slow_store.get(digest, slow_tx); + let fast_store_fut = self + .fast_store + .update(digest, fast_rx, UploadSizeInfo::ExactSize(sz)); let (data_stream_res, slow_res, fast_res) = join!(data_stream_fut, slow_store_fut, fast_store_fut); @@ -362,11 +372,7 @@ impl Store for FastSlowStore { } } - fn inner_store(&self, _digest: Option) -> &'_ dyn Store { - self - } - - fn inner_store_arc(self: Arc, _digest: Option) -> Arc { + fn inner_store(&self, _digest: Option) -> &dyn StoreDriver { self } @@ -380,13 +386,9 @@ impl Store for FastSlowStore { fn register_metrics(self: Arc, registry: &mut Registry) { let fast_store_registry = registry.sub_registry_with_prefix("fast"); - self.fast_store - .clone() - .register_metrics(fast_store_registry); + self.fast_store.register_metrics(fast_store_registry); let slow_store_registry = registry.sub_registry_with_prefix("slow"); - self.slow_store - .clone() - .register_metrics(slow_store_registry); + self.slow_store.register_metrics(slow_store_registry); } } diff --git a/nativelink-store/src/filesystem_store.rs b/nativelink-store/src/filesystem_store.rs index ee3d236cb..17d62e9cf 100644 --- a/nativelink-store/src/filesystem_store.rs +++ b/nativelink-store/src/filesystem_store.rs @@ -17,7 +17,7 @@ use std::ffi::{OsStr, OsString}; use std::fmt::{Debug, Formatter}; use std::pin::Pin; use std::sync::atomic::{AtomicU64, Ordering}; -use std::sync::Arc; +use std::sync::{Arc, Weak}; use std::time::{Duration, SystemTime}; use async_lock::RwLock; @@ -34,7 +34,7 @@ use nativelink_util::common::{fs, DigestInfo}; use nativelink_util::evicting_map::{EvictingMap, LenEntry}; use nativelink_util::health_utils::{HealthRegistryBuilder, HealthStatus, HealthStatusIndicator}; use nativelink_util::metrics_utils::{Collector, CollectorState, MetricsComponent, Registry}; -use nativelink_util::store_trait::{Store, StoreOptimizations, UploadSizeInfo}; +use nativelink_util::store_trait::{StoreDriver, StoreOptimizations, UploadSizeInfo}; use nativelink_util::{background_spawn, spawn_blocking}; use tokio::io::{AsyncReadExt, AsyncSeekExt, AsyncWriteExt, SeekFrom}; use tokio::time::{sleep, timeout, Sleep}; @@ -522,12 +522,15 @@ pub struct FilesystemStore { evicting_map: Arc, SystemTime>>, block_size: u64, read_buffer_size: usize, + weak_self: Weak, sleep_fn: fn(Duration) -> Sleep, rename_fn: fn(&OsStr, &OsStr) -> Result<(), std::io::Error>, } impl FilesystemStore { - pub async fn new(config: &nativelink_config::stores::FilesystemStore) -> Result { + pub async fn new( + config: &nativelink_config::stores::FilesystemStore, + ) -> Result, Error> { Self::new_with_timeout_and_rename_fn(config, sleep, |from, to| std::fs::rename(from, to)) .await } @@ -536,7 +539,7 @@ impl FilesystemStore { config: &nativelink_config::stores::FilesystemStore, sleep_fn: fn(Duration) -> Sleep, rename_fn: fn(&OsStr, &OsStr) -> Result<(), std::io::Error>, - ) -> Result { + ) -> Result, Error> { let now = SystemTime::now(); let empty_policy = nativelink_config::stores::EvictionPolicy::default(); @@ -569,15 +572,19 @@ impl FilesystemStore { } else { config.read_buffer_size as usize }; - let store = Self { + Ok(Arc::new_cyclic(|weak_self| Self { shared_context, evicting_map, block_size, read_buffer_size, + weak_self: weak_self.clone(), sleep_fn, rename_fn, - }; - Ok(store) + })) + } + + pub fn get_arc(&self) -> Option> { + self.weak_self.upgrade() } pub async fn get_file_entry_for_digest(&self, digest: &DigestInfo) -> Result, Error> { @@ -714,7 +721,7 @@ impl FilesystemStore { } #[async_trait] -impl Store for FilesystemStore { +impl StoreDriver for FilesystemStore { async fn has_with_results( self: Pin<&Self>, digests: &[DigestInfo], @@ -811,7 +818,7 @@ impl Store for FilesystemStore { return Ok(None); } - async fn get_part_ref( + async fn get_part( self: Pin<&Self>, digest: DigestInfo, writer: &mut DropCloserWriteHalf, @@ -824,7 +831,7 @@ impl Store for FilesystemStore { .err_tip(|| "Failed to check if zero digest exists in filesystem store")?; writer .send_eof() - .err_tip(|| "Failed to send zero EOF in filesystem store get_part_ref")?; + .err_tip(|| "Failed to send zero EOF in filesystem store get_part")?; return Ok(()); } @@ -886,11 +893,7 @@ impl Store for FilesystemStore { Ok(()) } - fn inner_store(&self, _digest: Option) -> &'_ dyn Store { - self - } - - fn inner_store_arc(self: Arc, _digest: Option) -> Arc { + fn inner_store(&self, _digest: Option) -> &dyn StoreDriver { self } @@ -944,6 +947,6 @@ impl HealthStatusIndicator for FilesystemStore { } async fn check_health(&self, namespace: Cow<'static, str>) -> HealthStatus { - Store::check_health(Pin::new(self), namespace).await + StoreDriver::check_health(Pin::new(self), namespace).await } } diff --git a/nativelink-store/src/grpc_store.rs b/nativelink-store/src/grpc_store.rs index 371205ee5..570996ab5 100644 --- a/nativelink-store/src/grpc_store.rs +++ b/nativelink-store/src/grpc_store.rs @@ -45,7 +45,7 @@ use nativelink_util::proto_stream_utils::{ }; use nativelink_util::resource_info::ResourceInfo; use nativelink_util::retry::{Retrier, RetryResult}; -use nativelink_util::store_trait::{Store, UploadSizeInfo}; +use nativelink_util::store_trait::{StoreDriver, UploadSizeInfo}; use nativelink_util::{default_health_status_indicator, tls_utils}; use parking_lot::Mutex; use prost::Message; @@ -501,7 +501,7 @@ impl GrpcStore { } #[async_trait] -impl Store for GrpcStore { +impl StoreDriver for GrpcStore { // NOTE: This function can only be safely used on CAS stores. AC stores may return a size that // is incorrect. async fn has_with_results( @@ -640,7 +640,7 @@ impl Store for GrpcStore { Ok(()) } - async fn get_part_ref( + async fn get_part( self: Pin<&Self>, digest: DigestInfo, writer: &mut DropCloserWriteHalf, @@ -735,11 +735,7 @@ impl Store for GrpcStore { .await } - fn inner_store(&self, _digest: Option) -> &'_ dyn Store { - self - } - - fn inner_store_arc(self: Arc, _digest: Option) -> Arc { + fn inner_store(&self, _digest: Option) -> &dyn StoreDriver { self } diff --git a/nativelink-store/src/memory_store.rs b/nativelink-store/src/memory_store.rs index ba88a39f1..5af4845cf 100644 --- a/nativelink-store/src/memory_store.rs +++ b/nativelink-store/src/memory_store.rs @@ -25,7 +25,7 @@ use nativelink_util::common::DigestInfo; use nativelink_util::evicting_map::{EvictingMap, LenEntry}; use nativelink_util::health_utils::{default_health_status_indicator, HealthStatusIndicator}; use nativelink_util::metrics_utils::{Collector, CollectorState, MetricsComponent, Registry}; -use nativelink_util::store_trait::{Store, UploadSizeInfo}; +use nativelink_util::store_trait::{StoreDriver, UploadSizeInfo}; use crate::cas_utils::is_zero_digest; @@ -75,7 +75,7 @@ impl MemoryStore { } #[async_trait] -impl Store for MemoryStore { +impl StoreDriver for MemoryStore { async fn has_with_results( self: Pin<&Self>, digests: &[DigestInfo], @@ -118,7 +118,7 @@ impl Store for MemoryStore { Ok(()) } - async fn get_part_ref( + async fn get_part( self: Pin<&Self>, digest: DigestInfo, writer: &mut DropCloserWriteHalf, @@ -128,7 +128,7 @@ impl Store for MemoryStore { if is_zero_digest(&digest) { writer .send_eof() - .err_tip(|| "Failed to send zero EOF in filesystem store get_part_ref")?; + .err_tip(|| "Failed to send zero EOF in filesystem store get_part")?; return Ok(()); } @@ -156,11 +156,7 @@ impl Store for MemoryStore { Ok(()) } - fn inner_store(&self, _digest: Option) -> &'_ dyn Store { - self - } - - fn inner_store_arc(self: Arc, _digest: Option) -> Arc { + fn inner_store(&self, _digest: Option) -> &dyn StoreDriver { self } diff --git a/nativelink-store/src/noop_store.rs b/nativelink-store/src/noop_store.rs index d9aaf7700..6902116a4 100644 --- a/nativelink-store/src/noop_store.rs +++ b/nativelink-store/src/noop_store.rs @@ -20,7 +20,7 @@ use nativelink_error::{make_err, Code, Error, ResultExt}; use nativelink_util::buf_channel::{DropCloserReadHalf, DropCloserWriteHalf}; use nativelink_util::common::DigestInfo; use nativelink_util::health_utils::{default_health_status_indicator, HealthStatusIndicator}; -use nativelink_util::store_trait::{Store, StoreOptimizations, UploadSizeInfo}; +use nativelink_util::store_trait::{StoreDriver, StoreOptimizations, UploadSizeInfo}; #[derive(Default)] pub struct NoopStore; @@ -32,7 +32,7 @@ impl NoopStore { } #[async_trait] -impl Store for NoopStore { +impl StoreDriver for NoopStore { async fn has_with_results( self: Pin<&Self>, _digests: &[DigestInfo], @@ -59,7 +59,7 @@ impl Store for NoopStore { || optimization == StoreOptimizations::NoopDownloads } - async fn get_part_ref( + async fn get_part( self: Pin<&Self>, _digest: DigestInfo, _writer: &mut DropCloserWriteHalf, @@ -69,11 +69,7 @@ impl Store for NoopStore { Err(make_err!(Code::NotFound, "Not found in noop store")) } - fn inner_store(&self, _digest: Option) -> &'_ dyn Store { - self - } - - fn inner_store_arc(self: Arc, _digest: Option) -> Arc { + fn inner_store(&self, _digest: Option) -> &dyn StoreDriver { self } diff --git a/nativelink-store/src/redis_store.rs b/nativelink-store/src/redis_store.rs index 75370d7f6..382f42ed8 100644 --- a/nativelink-store/src/redis_store.rs +++ b/nativelink-store/src/redis_store.rs @@ -27,7 +27,7 @@ use nativelink_util::buf_channel::{DropCloserReadHalf, DropCloserWriteHalf}; use nativelink_util::common::DigestInfo; use nativelink_util::health_utils::{HealthRegistryBuilder, HealthStatus, HealthStatusIndicator}; use nativelink_util::metrics_utils::{Collector, CollectorState, MetricsComponent, Registry}; -use nativelink_util::store_trait::{Store, UploadSizeInfo}; +use nativelink_util::store_trait::{StoreDriver, UploadSizeInfo}; use redis::aio::{ConnectionLike, ConnectionManager}; use redis::AsyncCommands; @@ -112,7 +112,7 @@ impl RedisStore { } #[async_trait] -impl Store for RedisStore { +impl StoreDriver for RedisStore { async fn has_with_results( self: Pin<&Self>, digests: &[DigestInfo], @@ -223,7 +223,7 @@ impl Store for RedisS Ok(()) } - async fn get_part_ref( + async fn get_part( self: Pin<&Self>, digest: DigestInfo, writer: &mut DropCloserWriteHalf, @@ -235,7 +235,7 @@ impl Store for RedisS if is_zero_digest(&digest) { writer .send_eof() - .err_tip(|| "Failed to send zero EOF in redis store get_part_ref")?; + .err_tip(|| "Failed to send zero EOF in redis store get_part")?; return Ok(()); } @@ -245,7 +245,7 @@ impl Store for RedisS .exists::<_, bool>(digest_to_key(&digest)) .await .map_err(from_redis_err) - .err_tip(|| "In RedisStore::get_part_ref::zero_exists")?; + .err_tip(|| "In RedisStore::get_part::zero_exists")?; if !exists { return Err(make_err!( Code::NotFound, @@ -255,14 +255,14 @@ impl Store for RedisS } writer .send_eof() - .err_tip(|| "Failed to write EOF in redis store get_part_ref")?; + .err_tip(|| "Failed to write EOF in redis store get_part")?; return Ok(()); } let mut current_start = isize::try_from(offset) - .err_tip(|| "Cannot convert offset to isize in RedisStore::get_part_ref()")?; + .err_tip(|| "Cannot convert offset to isize in RedisStore::get_part()")?; let max_length = isize::try_from(length.unwrap_or(isize::MAX as usize)) - .err_tip(|| "Cannot convert length to isize in RedisStore::get_part_ref()")?; + .err_tip(|| "Cannot convert length to isize in RedisStore::get_part()")?; let end_position = current_start.saturating_add(max_length); loop { @@ -273,7 +273,7 @@ impl Store for RedisS .getrange::<_, Bytes>(digest_to_key(&digest), current_start, current_end) .await .map_err(from_redis_err) - .err_tip(|| "In RedisStore::get_part_ref::getrange")?; + .err_tip(|| "In RedisStore::get_part::getrange")?; if chunk.is_empty() { writer @@ -308,11 +308,7 @@ impl Store for RedisS Ok(()) } - fn inner_store(&self, _digest: Option) -> &'_ dyn Store { - self - } - - fn inner_store_arc(self: Arc, _digest: Option) -> Arc { + fn inner_store(&self, _digest: Option) -> &dyn StoreDriver { self } @@ -346,7 +342,7 @@ impl } async fn check_health(&self, namespace: Cow<'static, str>) -> HealthStatus { - Store::check_health(Pin::new(self), namespace).await + StoreDriver::check_health(Pin::new(self), namespace).await } } diff --git a/nativelink-store/src/ref_store.rs b/nativelink-store/src/ref_store.rs index 9206bb331..d8bcb8125 100644 --- a/nativelink-store/src/ref_store.rs +++ b/nativelink-store/src/ref_store.rs @@ -21,13 +21,13 @@ use nativelink_error::{make_err, make_input_err, Code, Error, ResultExt}; use nativelink_util::buf_channel::{DropCloserReadHalf, DropCloserWriteHalf}; use nativelink_util::common::DigestInfo; use nativelink_util::health_utils::{default_health_status_indicator, HealthStatusIndicator}; -use nativelink_util::store_trait::{Store, UploadSizeInfo}; +use nativelink_util::store_trait::{Store, StoreDriver, StoreLike, UploadSizeInfo}; use tracing::{event, Level}; use crate::store_manager::StoreManager; #[repr(C, align(8))] -struct AlignedStoreCell(UnsafeCell>>); +struct AlignedStoreCell(UnsafeCell>); struct StoreReference { cell: AlignedStoreCell, @@ -65,7 +65,7 @@ impl RefStore { // 2. It should only happen on platforms that are < 64 bit address space // 3. It is likely that the internals of how Option work protect us anyway. #[inline] - fn get_store(&self) -> Result<&Arc, Error> { + fn get_store(&self) -> Result<&Store, Error> { let ref_store = self.ref_store.cell.0.get(); unsafe { if let Some(ref store) = *ref_store { @@ -87,7 +87,7 @@ impl RefStore { .err_tip(|| "Store manager is gone")?; if let Some(store) = store_manager.get_store(&self.ref_store_name) { unsafe { - *ref_store = Some(store.clone()); + *ref_store = Some(store); return Ok((*ref_store).as_ref().unwrap()); } } @@ -99,16 +99,13 @@ impl RefStore { } #[async_trait] -impl Store for RefStore { +impl StoreDriver for RefStore { async fn has_with_results( self: Pin<&Self>, digests: &[DigestInfo], results: &mut [Option], ) -> Result<(), Error> { - let store = self.get_store()?; - Pin::new(store.as_ref()) - .has_with_results(digests, results) - .await + self.get_store()?.has_with_results(digests, results).await } async fn update( @@ -117,26 +114,22 @@ impl Store for RefStore { reader: DropCloserReadHalf, size_info: UploadSizeInfo, ) -> Result<(), Error> { - let store = self.get_store()?; - Pin::new(store.as_ref()) - .update(digest, reader, size_info) - .await + self.get_store()?.update(digest, reader, size_info).await } - async fn get_part_ref( + async fn get_part( self: Pin<&Self>, digest: DigestInfo, writer: &mut DropCloserWriteHalf, offset: usize, length: Option, ) -> Result<(), Error> { - let store = self.get_store()?; - Pin::new(store.as_ref()) - .get_part_ref(digest, writer, offset, length) + self.get_store()? + .get_part(digest, writer, offset, length) .await } - fn inner_store(&self, digest: Option) -> &'_ dyn Store { + fn inner_store(&self, digest: Option) -> &'_ dyn StoreDriver { match self.get_store() { Ok(store) => store.inner_store(digest), Err(err) => { @@ -151,21 +144,6 @@ impl Store for RefStore { } } - fn inner_store_arc(self: Arc, digest: Option) -> Arc { - match self.get_store() { - Ok(store) => store.clone().inner_store_arc(digest), - Err(err) => { - event!( - Level::ERROR, - ?digest, - ?err, - "Failed to get store for digest", - ); - self - } - } - } - fn as_any<'a>(&'a self) -> &'a (dyn std::any::Any + Sync + Send + 'static) { self } diff --git a/nativelink-store/src/s3_store.rs b/nativelink-store/src/s3_store.rs index 9a3a1e664..766abe6e1 100644 --- a/nativelink-store/src/s3_store.rs +++ b/nativelink-store/src/s3_store.rs @@ -48,7 +48,7 @@ use nativelink_util::common::DigestInfo; use nativelink_util::fs; use nativelink_util::health_utils::{default_health_status_indicator, HealthStatusIndicator}; use nativelink_util::retry::{Retrier, RetryResult}; -use nativelink_util::store_trait::{Store, UploadSizeInfo}; +use nativelink_util::store_trait::{StoreDriver, UploadSizeInfo}; use rand::rngs::OsRng; use rand::Rng; use tokio::io::{AsyncRead, AsyncWrite, ReadBuf}; @@ -354,7 +354,7 @@ impl S3Store { } #[async_trait] -impl Store for S3Store { +impl StoreDriver for S3Store { async fn has_with_results( self: Pin<&Self>, digests: &[DigestInfo], @@ -639,7 +639,7 @@ impl Store for S3Store { .await } - async fn get_part_ref( + async fn get_part( self: Pin<&Self>, digest: DigestInfo, writer: &mut DropCloserWriteHalf, @@ -649,7 +649,7 @@ impl Store for S3Store { if is_zero_digest(&digest) { writer .send_eof() - .err_tip(|| "Failed to send zero EOF in filesystem store get_part_ref")?; + .err_tip(|| "Failed to send zero EOF in filesystem store get_part")?; return Ok(()); } @@ -738,11 +738,7 @@ impl Store for S3Store { .await } - fn inner_store(&self, _digest: Option) -> &'_ dyn Store { - self - } - - fn inner_store_arc(self: Arc, _digest: Option) -> Arc { + fn inner_store(&self, _digest: Option) -> &'_ dyn StoreDriver { self } diff --git a/nativelink-store/src/shard_store.rs b/nativelink-store/src/shard_store.rs index 6c6e779b6..bc03e5a93 100644 --- a/nativelink-store/src/shard_store.rs +++ b/nativelink-store/src/shard_store.rs @@ -23,18 +23,18 @@ use nativelink_util::buf_channel::{DropCloserReadHalf, DropCloserWriteHalf}; use nativelink_util::common::DigestInfo; use nativelink_util::health_utils::{default_health_status_indicator, HealthStatusIndicator}; use nativelink_util::metrics_utils::Registry; -use nativelink_util::store_trait::{Store, UploadSizeInfo}; +use nativelink_util::store_trait::{Store, StoreDriver, StoreLike, UploadSizeInfo}; pub struct ShardStore { // The weights will always be in ascending order a specific store is choosen based on the // the hash of the digest hash that is nearest-binary searched using the u32 as the index. - weights_and_stores: Vec<(u32, Arc)>, + weights_and_stores: Vec<(u32, Store)>, } impl ShardStore { pub fn new( config: &nativelink_config::stores::ShardStore, - stores: Vec>, + stores: Vec, ) -> Result { error_if!( config.stores.len() != stores.len(), @@ -107,14 +107,14 @@ impl ShardStore { .unwrap_or_else(|index| index) } - fn get_store(&self, digest: &DigestInfo) -> Pin<&dyn Store> { + fn get_store(&self, digest: &DigestInfo) -> &Store { let index = self.get_store_index(digest); - Pin::new(self.weights_and_stores[index].1.as_ref()) + &self.weights_and_stores[index].1 } } #[async_trait] -impl Store for ShardStore { +impl StoreDriver for ShardStore { async fn has_with_results( self: Pin<&Self>, digests: &[DigestInfo], @@ -123,7 +123,7 @@ impl Store for ShardStore { if digests.len() == 1 { // Hot path: It is very common to lookup only one digest. let store_idx = self.get_store_index(&digests[0]); - let store = Pin::new(self.weights_and_stores[store_idx].1.as_ref()); + let store = &self.weights_and_stores[store_idx].1; return store .has_with_results(digests, results) .await @@ -151,7 +151,7 @@ impl Store for ShardStore { .into_iter() .enumerate() .map(|(store_idx, (digest_idxs, digests))| async move { - let store = Pin::new(self.weights_and_stores[store_idx].1.as_ref()); + let store = &self.weights_and_stores[store_idx].1; let mut inner_results = vec![None; digests.len()]; store .has_with_results(&digests, &mut inner_results) @@ -183,7 +183,7 @@ impl Store for ShardStore { .err_tip(|| "In ShardStore::update()") } - async fn get_part_ref( + async fn get_part( self: Pin<&Self>, digest: DigestInfo, writer: &mut DropCloserWriteHalf, @@ -192,20 +192,12 @@ impl Store for ShardStore { ) -> Result<(), Error> { let store = self.get_store(&digest); store - .get_part_ref(digest, writer, offset, length) + .get_part(digest, writer, offset, length) .await - .err_tip(|| "In ShardStore::get_part_ref()") + .err_tip(|| "In ShardStore::get_part()") } - fn as_any<'a>(&'a self) -> &'a (dyn std::any::Any + Sync + Send + 'static) { - self - } - - fn as_any_arc(self: Arc) -> Arc { - self - } - - fn inner_store(&self, digest: Option) -> &'_ dyn Store { + fn inner_store(&self, digest: Option) -> &'_ dyn StoreDriver { let Some(digest) = digest else { return self; }; @@ -213,15 +205,12 @@ impl Store for ShardStore { self.weights_and_stores[index].1.inner_store(Some(digest)) } - fn inner_store_arc(self: Arc, digest: Option) -> Arc { - let Some(digest) = digest else { - return self; - }; - let index = self.get_store_index(&digest); - self.weights_and_stores[index] - .1 - .clone() - .inner_store_arc(Some(digest)) + fn as_any<'a>(&'a self) -> &'a (dyn std::any::Any + Sync + Send + 'static) { + self + } + + fn as_any_arc(self: Arc) -> Arc { + self } fn register_metrics(self: Arc, registry: &mut Registry) { diff --git a/nativelink-store/src/size_partitioning_store.rs b/nativelink-store/src/size_partitioning_store.rs index e66bcbdbb..1057d3e2c 100644 --- a/nativelink-store/src/size_partitioning_store.rs +++ b/nativelink-store/src/size_partitioning_store.rs @@ -21,20 +21,20 @@ use nativelink_util::buf_channel::{DropCloserReadHalf, DropCloserWriteHalf}; use nativelink_util::common::DigestInfo; use nativelink_util::health_utils::{default_health_status_indicator, HealthStatusIndicator}; use nativelink_util::metrics_utils::{Collector, CollectorState, MetricsComponent, Registry}; -use nativelink_util::store_trait::{Store, UploadSizeInfo}; +use nativelink_util::store_trait::{Store, StoreDriver, StoreLike, UploadSizeInfo}; use tokio::join; pub struct SizePartitioningStore { partition_size: i64, - lower_store: Arc, - upper_store: Arc, + lower_store: Store, + upper_store: Store, } impl SizePartitioningStore { pub fn new( config: &nativelink_config::stores::SizePartitioningStore, - lower_store: Arc, - upper_store: Arc, + lower_store: Store, + upper_store: Store, ) -> Self { SizePartitioningStore { partition_size: config.size as i64, @@ -45,7 +45,7 @@ impl SizePartitioningStore { } #[async_trait] -impl Store for SizePartitioningStore { +impl StoreDriver for SizePartitioningStore { async fn has_with_results( self: Pin<&Self>, digests: &[DigestInfo], @@ -56,8 +56,8 @@ impl Store for SizePartitioningStore { .cloned() .partition(|digest| digest.size_bytes < self.partition_size); let (lower_results, upper_results) = join!( - Pin::new(self.lower_store.as_ref()).has_many(&lower_digests), - Pin::new(self.upper_store.as_ref()).has_many(&upper_digests), + self.lower_store.has_many(&lower_digests), + self.upper_store.has_many(&upper_digests), ); let mut lower_results = match lower_results { Ok(lower_results) => lower_results.into_iter(), @@ -90,16 +90,12 @@ impl Store for SizePartitioningStore { size_info: UploadSizeInfo, ) -> Result<(), Error> { if digest.size_bytes < self.partition_size { - return Pin::new(self.lower_store.as_ref()) - .update(digest, reader, size_info) - .await; + return self.lower_store.update(digest, reader, size_info).await; } - Pin::new(self.upper_store.as_ref()) - .update(digest, reader, size_info) - .await + self.upper_store.update(digest, reader, size_info).await } - async fn get_part_ref( + async fn get_part( self: Pin<&Self>, digest: DigestInfo, writer: &mut DropCloserWriteHalf, @@ -107,16 +103,17 @@ impl Store for SizePartitioningStore { length: Option, ) -> Result<(), Error> { if digest.size_bytes < self.partition_size { - return Pin::new(self.lower_store.as_ref()) - .get_part_ref(digest, writer, offset, length) + return self + .lower_store + .get_part(digest, writer, offset, length) .await; } - Pin::new(self.upper_store.as_ref()) - .get_part_ref(digest, writer, offset, length) + self.upper_store + .get_part(digest, writer, offset, length) .await } - fn inner_store(&self, digest: Option) -> &'_ dyn Store { + fn inner_store(&self, digest: Option) -> &'_ dyn StoreDriver { let Some(digest) = digest else { return self; }; @@ -126,16 +123,6 @@ impl Store for SizePartitioningStore { self.upper_store.inner_store(Some(digest)) } - fn inner_store_arc(self: Arc, digest: Option) -> Arc { - let Some(digest) = digest else { - return self; - }; - if digest.size_bytes < self.partition_size { - return self.lower_store.clone().inner_store_arc(Some(digest)); - } - self.upper_store.clone().inner_store_arc(Some(digest)) - } - fn as_any<'a>(&'a self) -> &'a (dyn std::any::Any + Sync + Send + 'static) { self } @@ -146,13 +133,9 @@ impl Store for SizePartitioningStore { fn register_metrics(self: Arc, registry: &mut Registry) { let lower_store_registry = registry.sub_registry_with_prefix("lower_store"); - self.lower_store - .clone() - .register_metrics(lower_store_registry); + self.lower_store.register_metrics(lower_store_registry); let upper_store_registry = registry.sub_registry_with_prefix("upper_store"); - self.upper_store - .clone() - .register_metrics(upper_store_registry); + self.upper_store.register_metrics(upper_store_registry); registry.register_collector(Box::new(Collector::new(&self))); } } diff --git a/nativelink-store/src/store_manager.rs b/nativelink-store/src/store_manager.rs index 034eec599..e48506a18 100644 --- a/nativelink-store/src/store_manager.rs +++ b/nativelink-store/src/store_manager.rs @@ -13,12 +13,12 @@ // limitations under the License. use std::collections::HashMap; -use std::sync::{Arc, RwLock}; +use std::sync::RwLock; use nativelink_util::store_trait::Store; pub struct StoreManager { - stores: RwLock>>, + stores: RwLock>, } impl StoreManager { @@ -28,7 +28,7 @@ impl StoreManager { } } - pub fn add_store(&self, name: &str, store: Arc) { + pub fn add_store(&self, name: &str, store: Store) { let mut stores = self .stores .write() @@ -36,7 +36,7 @@ impl StoreManager { stores.insert(name.to_string(), store); } - pub fn get_store(&self, name: &str) -> Option> { + pub fn get_store(&self, name: &str) -> Option { let stores = self .stores .read() diff --git a/nativelink-store/src/verify_store.rs b/nativelink-store/src/verify_store.rs index 1fa28b0b4..8a36b949f 100644 --- a/nativelink-store/src/verify_store.rs +++ b/nativelink-store/src/verify_store.rs @@ -29,10 +29,10 @@ use nativelink_util::metrics_utils::{ Collector, CollectorState, CounterWithTime, MetricsComponent, Registry, }; use nativelink_util::origin_context::ActiveOriginContext; -use nativelink_util::store_trait::{Store, UploadSizeInfo}; +use nativelink_util::store_trait::{Store, StoreDriver, StoreLike, UploadSizeInfo}; pub struct VerifyStore { - inner_store: Arc, + inner_store: Store, verify_size: bool, verify_hash: bool, @@ -42,10 +42,7 @@ pub struct VerifyStore { } impl VerifyStore { - pub fn new( - config: &nativelink_config::stores::VerifyStore, - inner_store: Arc, - ) -> Self { + pub fn new(config: &nativelink_config::stores::VerifyStore, inner_store: Store) -> Self { VerifyStore { inner_store, verify_size: config.verify_size, @@ -55,10 +52,6 @@ impl VerifyStore { } } - fn pin_inner(&self) -> Pin<&dyn Store> { - Pin::new(self.inner_store.as_ref()) - } - async fn inner_check_update( &self, mut tx: DropCloserWriteHalf, @@ -118,13 +111,13 @@ impl VerifyStore { } #[async_trait] -impl Store for VerifyStore { +impl StoreDriver for VerifyStore { async fn has_with_results( self: Pin<&Self>, digests: &[DigestInfo], results: &mut [Option], ) -> Result<(), Error> { - self.pin_inner().has_with_results(digests, results).await + self.inner_store.has_with_results(digests, results).await } async fn update( @@ -159,7 +152,7 @@ impl Store for VerifyStore { let (tx, rx) = make_buf_channel_pair(); - let update_fut = self.pin_inner().update(digest, rx, size_info); + let update_fut = self.inner_store.update(digest, rx, size_info); let check_fut = self.inner_check_update(tx, reader, size_info, digest.packed_hash, hasher.as_mut()); @@ -168,23 +161,19 @@ impl Store for VerifyStore { update_res.merge(check_res) } - async fn get_part_ref( + async fn get_part( self: Pin<&Self>, digest: DigestInfo, writer: &mut DropCloserWriteHalf, offset: usize, length: Option, ) -> Result<(), Error> { - self.pin_inner() - .get_part_ref(digest, writer, offset, length) + self.inner_store + .get_part(digest, writer, offset, length) .await } - fn inner_store(&self, _digest: Option) -> &'_ dyn Store { - self - } - - fn inner_store_arc(self: Arc, _digest: Option) -> Arc { + fn inner_store(&self, _digest: Option) -> &'_ dyn StoreDriver { self } @@ -198,7 +187,7 @@ impl Store for VerifyStore { fn register_metrics(self: Arc, registry: &mut Registry) { let backend_store = registry.sub_registry_with_prefix("backend"); - self.inner_store.clone().register_metrics(backend_store); + self.inner_store.register_metrics(backend_store); registry.register_collector(Box::new(Collector::new(&self))); } } diff --git a/nativelink-store/tests/ac_utils_test.rs b/nativelink-store/tests/ac_utils_test.rs index 64b932575..c41933d9f 100644 --- a/nativelink-store/tests/ac_utils_test.rs +++ b/nativelink-store/tests/ac_utils_test.rs @@ -14,14 +14,13 @@ use std::env; use std::ffi::OsString; -use std::pin::Pin; use std::sync::Arc; use nativelink_error::{Error, ResultExt}; use nativelink_macro::nativelink_test; use nativelink_store::memory_store::MemoryStore; use nativelink_util::common::{fs, DigestInfo}; -use nativelink_util::store_trait::{Store, UploadSizeInfo}; +use nativelink_util::store_trait::{StoreLike, UploadSizeInfo}; use rand::{thread_rng, Rng}; use tokio::io::AsyncWriteExt; @@ -56,7 +55,6 @@ mod ac_utils_tests { let store = Arc::new(MemoryStore::new( &nativelink_config::stores::MemoryStore::default(), )); - let store_pin = Pin::new(store.as_ref()); let digest = DigestInfo::try_new(HASH1, HASH1_SIZE)?; // Dummy hash data. { // Write 1MB of 0x88s to the file. @@ -72,7 +70,7 @@ mod ac_utils_tests { { // Upload our file. let resumeable_file = fs::open_file(filepath, u64::MAX).await?; - store_pin + store .update_with_whole_file( digest, resumeable_file, @@ -82,7 +80,7 @@ mod ac_utils_tests { } { // Check to make sure the file was saved correctly to the store. - let store_data = store_pin.get_part_unchunked(digest, 0, None).await?; + let store_data = store.get_part_unchunked(digest, 0, None).await?; assert_eq!(store_data.len(), expected_data.len()); assert_eq!(store_data, expected_data); } diff --git a/nativelink-store/tests/completeness_checking_store_test.rs b/nativelink-store/tests/completeness_checking_store_test.rs index 06b875caa..d059bc430 100644 --- a/nativelink-store/tests/completeness_checking_store_test.rs +++ b/nativelink-store/tests/completeness_checking_store_test.rs @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::pin::Pin; use std::sync::Arc; use nativelink_config::stores::MemoryStore as MemoryStoreConfig; @@ -27,7 +26,7 @@ use nativelink_store::completeness_checking_store::CompletenessCheckingStore; use nativelink_store::memory_store::MemoryStore; use nativelink_util::common::DigestInfo; use nativelink_util::digest_hasher::DigestHasherFunc; -use nativelink_util::store_trait::Store; +use nativelink_util::store_trait::{Store, StoreLike}; #[cfg(test)] mod completeness_checking_store_tests { @@ -42,21 +41,19 @@ mod completeness_checking_store_tests { async fn setup() -> Result<(Arc, Arc, DigestInfo), Error> { - let backend_store = Arc::new(MemoryStore::new(&MemoryStoreConfig::default())); + let backend_store = Store::new(Arc::new(MemoryStore::new(&MemoryStoreConfig::default()))); let cas_store = Arc::new(MemoryStore::new(&MemoryStoreConfig::default())); - let ac_owned = Arc::new(CompletenessCheckingStore::new( + let ac_store = Arc::new(CompletenessCheckingStore::new( backend_store.clone(), - cas_store.clone(), + Store::new(cas_store.clone()), )); - let pinned_ac: Pin<&dyn Store> = Pin::new(backend_store.as_ref()); - let pinned_cas: Pin<&dyn Store> = Pin::new(cas_store.as_ref()); - pinned_cas.update_oneshot(ROOT_FILE, "".into()).await?; + cas_store.update_oneshot(ROOT_FILE, "".into()).await?; // Note: Explicitly not uploading `ROOT_DIRECTORY`. See: TraceMachina/nativelink#747. - pinned_cas.update_oneshot(CHILD_FILE, "".into()).await?; - pinned_cas.update_oneshot(OUTPUT_FILE, "".into()).await?; - pinned_cas.update_oneshot(STDOUT, "".into()).await?; - pinned_cas.update_oneshot(STDERR, "".into()).await?; + cas_store.update_oneshot(CHILD_FILE, "".into()).await?; + cas_store.update_oneshot(OUTPUT_FILE, "".into()).await?; + cas_store.update_oneshot(STDOUT, "".into()).await?; + cas_store.update_oneshot(STDERR, "".into()).await?; let tree = Tree { root: Some(Directory { @@ -79,9 +76,12 @@ mod completeness_checking_store_tests { }], }; - let tree_digest = - serialize_and_upload_message(&tree, pinned_cas, &mut DigestHasherFunc::Blake3.hasher()) - .await?; + let tree_digest = serialize_and_upload_message( + &tree, + cas_store.as_pin(), + &mut DigestHasherFunc::Blake3.hasher(), + ) + .await?; let output_directory = OutputDirectory { tree_digest: Some(tree_digest.into()), @@ -90,7 +90,7 @@ mod completeness_checking_store_tests { serialize_and_upload_message( &output_directory, - pinned_cas, + cas_store.as_pin(), &mut DigestHasherFunc::Blake3.hasher(), ) .await?; @@ -109,12 +109,12 @@ mod completeness_checking_store_tests { // The structure of the action result is not following the spec, but is simplified for testing purposes. let action_result_digest = serialize_and_upload_message( &action_result, - pinned_ac, + ac_store.as_pin(), &mut DigestHasherFunc::Blake3.hasher(), ) .await?; - Ok((ac_owned, cas_store, action_result_digest)) + Ok((ac_store, cas_store, action_result_digest)) } #[nativelink_test] @@ -124,12 +124,7 @@ mod completeness_checking_store_tests { let (ac_store, _cas_store, action_result_digest) = setup().await?; - let pinned_store: Pin<&dyn Store> = Pin::new(ac_store.as_ref()); - - let res = pinned_store - .has_many(&[action_result_digest]) - .await - .unwrap(); + let res = ac_store.has_many(&[action_result_digest]).await.unwrap(); assert!( res[0].is_some(), "Results should be some with all items in CAS." @@ -141,14 +136,9 @@ mod completeness_checking_store_tests { let (ac_store, cas_store, action_result_digest) = setup().await?; - let pinned_store: Pin<&dyn Store> = Pin::new(ac_store.as_ref()); - cas_store.remove_entry(&ROOT_FILE).await; - let res = pinned_store - .has_many(&[action_result_digest]) - .await - .unwrap(); + let res = ac_store.has_many(&[action_result_digest]).await.unwrap(); assert!( res[0].is_none(), "Results should be none with missing root file." @@ -160,13 +150,8 @@ mod completeness_checking_store_tests { let (ac_store, cas_store, action_result_digest) = setup().await?; - let pinned_store: Pin<&dyn Store> = Pin::new(ac_store.as_ref()); - cas_store.remove_entry(&CHILD_FILE).await; - let res = pinned_store - .has_many(&[action_result_digest]) - .await - .unwrap(); + let res = ac_store.has_many(&[action_result_digest]).await.unwrap(); assert!( res[0].is_none(), "Results should be none with missing root file." @@ -178,13 +163,8 @@ mod completeness_checking_store_tests { let (ac_store, cas_store, action_result_digest) = setup().await?; - let pinned_store: Pin<&dyn Store> = Pin::new(ac_store.as_ref()); - cas_store.remove_entry(&OUTPUT_FILE).await; - let res = pinned_store - .has_many(&[action_result_digest]) - .await - .unwrap(); + let res = ac_store.has_many(&[action_result_digest]).await.unwrap(); assert!( res[0].is_none(), "Results should be none with missing root file." @@ -196,13 +176,8 @@ mod completeness_checking_store_tests { let (ac_store, cas_store, action_result_digest) = setup().await?; - let pinned_store: Pin<&dyn Store> = Pin::new(ac_store.as_ref()); - cas_store.remove_entry(&STDOUT).await; - let res = pinned_store - .has_many(&[action_result_digest]) - .await - .unwrap(); + let res = ac_store.has_many(&[action_result_digest]).await.unwrap(); assert!( res[0].is_none(), "Results should be none with missing root file." @@ -214,13 +189,8 @@ mod completeness_checking_store_tests { let (ac_store, cas_store, action_result_digest) = setup().await?; - let pinned_store: Pin<&dyn Store> = Pin::new(ac_store.as_ref()); - cas_store.remove_entry(&STDERR).await; - let res = pinned_store - .has_many(&[action_result_digest]) - .await - .unwrap(); + let res = ac_store.has_many(&[action_result_digest]).await.unwrap(); assert!( res[0].is_none(), "Results should be none with missing root file." @@ -237,10 +207,8 @@ mod completeness_checking_store_tests { let (ac_store, _cas_store, action_result_digest) = setup().await?; - let pinned_store: Pin<&dyn Store> = Pin::new(ac_store.as_ref()); - assert!( - pinned_store + ac_store .get_part_unchunked(action_result_digest, 0, None) .await .is_ok(), @@ -253,12 +221,10 @@ mod completeness_checking_store_tests { let (ac_store, cas_store, action_result_digest) = setup().await?; - let pinned_store: Pin<&dyn Store> = Pin::new(ac_store.as_ref()); - cas_store.remove_entry(&OUTPUT_FILE).await; assert!( - pinned_store + ac_store .get_part_unchunked(action_result_digest, 0, None) .await .is_err(), diff --git a/nativelink-store/tests/compression_store_test.rs b/nativelink-store/tests/compression_store_test.rs index e3f127541..5f450c446 100644 --- a/nativelink-store/tests/compression_store_test.rs +++ b/nativelink-store/tests/compression_store_test.rs @@ -30,7 +30,7 @@ use nativelink_store::memory_store::MemoryStore; use nativelink_util::buf_channel::make_buf_channel_pair; use nativelink_util::common::DigestInfo; use nativelink_util::spawn; -use nativelink_util::store_trait::{Store, UploadSizeInfo}; +use nativelink_util::store_trait::{Store, StoreLike, UploadSizeInfo}; use rand::rngs::SmallRng; use rand::{Rng, SeedableRng}; use sha2::{Digest, Sha256}; @@ -75,7 +75,7 @@ mod compression_store_tests { #[nativelink_test] async fn simple_smoke_test() -> Result<(), Error> { - let store_owned = CompressionStore::new( + let store = CompressionStore::new( nativelink_config::stores::CompressionStore { backend: nativelink_config::stores::StoreConfig::memory( nativelink_config::stores::MemoryStore::default(), @@ -86,12 +86,11 @@ mod compression_store_tests { }, ), }, - Arc::new(MemoryStore::new( + Store::new(Arc::new(MemoryStore::new( &nativelink_config::stores::MemoryStore::default(), - )), + ))), ) .err_tip(|| "Failed to create compression store")?; - let store = Pin::new(&store_owned); const RAW_INPUT: &str = "123"; let digest = DigestInfo::try_new(VALID_HASH, DUMMY_DATA_SIZE).unwrap(); @@ -124,9 +123,9 @@ mod compression_store_tests { }, ), }, - Arc::new(MemoryStore::new( + Store::new(Arc::new(MemoryStore::new( &nativelink_config::stores::MemoryStore::default(), - )), + ))), ) .err_tip(|| "Failed to create compression store")?; let store = Pin::new(&store_owned); @@ -182,9 +181,9 @@ mod compression_store_tests { }, ), }, - Arc::new(MemoryStore::new( + Store::new(Arc::new(MemoryStore::new( &nativelink_config::stores::MemoryStore::default(), - )), + ))), ) .err_tip(|| "Failed to create compression store")?; let store = Pin::new(&store_owned); @@ -221,7 +220,7 @@ mod compression_store_tests { }, ), }, - inner_store.clone(), + Store::new(inner_store.clone()), ) .err_tip(|| "Failed to create compression store")?; let store = Pin::new(&store_owned); @@ -279,7 +278,7 @@ mod compression_store_tests { }, ), }, - inner_store.clone(), + Store::new(inner_store.clone()), ) .err_tip(|| "Failed to create compression store")?; let store = Pin::new(&store_owned); @@ -368,7 +367,7 @@ mod compression_store_tests { }, ), }, - inner_store.clone(), + Store::new(inner_store.clone()), ) .err_tip(|| "Failed to create compression store")?; let store = Pin::new(&store_owned); @@ -520,7 +519,7 @@ mod compression_store_tests { }, ), }, - inner_store.clone(), + Store::new(inner_store.clone()), ) .err_tip(|| "Failed to create compression store")?; let store = Pin::new(Arc::new(store_owned)); @@ -530,9 +529,9 @@ mod compression_store_tests { let _drop_guard = spawn!("get_part_is_zero_digest", async move { let _ = store .as_ref() - .get_part_ref(digest, &mut writer, 0, None) + .get_part(digest, &mut writer, 0, None) .await - .err_tip(|| "Failed to get_part_ref"); + .err_tip(|| "Failed to get_part"); }); let file_data = reader diff --git a/nativelink-store/tests/dedup_store_test.rs b/nativelink-store/tests/dedup_store_test.rs index 1d5a93d28..e1c9bb683 100644 --- a/nativelink-store/tests/dedup_store_test.rs +++ b/nativelink-store/tests/dedup_store_test.rs @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::pin::Pin; use std::sync::Arc; use nativelink_error::{Code, Error, ResultExt}; @@ -20,7 +19,7 @@ use nativelink_macro::nativelink_test; use nativelink_store::dedup_store::DedupStore; use nativelink_store::memory_store::MemoryStore; use nativelink_util::common::DigestInfo; -use nativelink_util::store_trait::Store; +use nativelink_util::store_trait::{Store, StoreLike}; use rand::rngs::SmallRng; use rand::{Rng, SeedableRng}; @@ -58,16 +57,15 @@ mod dedup_store_tests { #[nativelink_test] async fn simple_round_trip_test() -> Result<(), Error> { - let store_owned = DedupStore::new( + let store = DedupStore::new( &make_default_config(), - Arc::new(MemoryStore::new( + Store::new(Arc::new(MemoryStore::new( &nativelink_config::stores::MemoryStore::default(), - )), // Index store. - Arc::new(MemoryStore::new( + ))), // Index store. + Store::new(Arc::new(MemoryStore::new( &nativelink_config::stores::MemoryStore::default(), - )), // Content store. + ))), // Content store. ); - let store = Pin::new(&store_owned); let original_data = make_random_data(MEGABYTE_SZ); let digest = DigestInfo::try_new(VALID_HASH1, MEGABYTE_SZ).unwrap(); @@ -91,14 +89,13 @@ mod dedup_store_tests { let content_store = Arc::new(MemoryStore::new( &nativelink_config::stores::MemoryStore::default(), )); - let store_owned = DedupStore::new( + let store = DedupStore::new( &make_default_config(), - Arc::new(MemoryStore::new( + Store::new(Arc::new(MemoryStore::new( &nativelink_config::stores::MemoryStore::default(), - )), // Index store. - content_store.clone(), + ))), // Index store. + Store::new(content_store.clone()), ); - let store = Pin::new(&store_owned); let original_data = make_random_data(MEGABYTE_SZ); let digest = DigestInfo::try_new(VALID_HASH1, MEGABYTE_SZ).unwrap(); @@ -134,16 +131,15 @@ mod dedup_store_tests { /// requested data; this test covers that use case. #[nativelink_test] async fn fetch_part_test() -> Result<(), Error> { - let store_owned = DedupStore::new( + let store = DedupStore::new( &make_default_config(), - Arc::new(MemoryStore::new( + Store::new(Arc::new(MemoryStore::new( &nativelink_config::stores::MemoryStore::default(), - )), // Index store. - Arc::new(MemoryStore::new( + ))), // Index store. + Store::new(Arc::new(MemoryStore::new( &nativelink_config::stores::MemoryStore::default(), - )), // Content store. + ))), // Content store. ); - let store = Pin::new(&store_owned); const DATA_SIZE: usize = MEGABYTE_SZ / 4; let original_data = make_random_data(DATA_SIZE); @@ -176,7 +172,7 @@ mod dedup_store_tests { #[nativelink_test] async fn check_length_not_set_with_chunk_read_beyond_first_chunk_regression_test( ) -> Result<(), Error> { - let store_owned = DedupStore::new( + let store = DedupStore::new( &nativelink_config::stores::DedupStore { index_store: nativelink_config::stores::StoreConfig::memory( nativelink_config::stores::MemoryStore::default(), @@ -189,14 +185,13 @@ mod dedup_store_tests { max_size: 7, max_concurrent_fetch_per_get: 10, }, - Arc::new(MemoryStore::new( + Store::new(Arc::new(MemoryStore::new( &nativelink_config::stores::MemoryStore::default(), - )), // Index store. - Arc::new(MemoryStore::new( + ))), // Index store. + Store::new(Arc::new(MemoryStore::new( &nativelink_config::stores::MemoryStore::default(), - )), // Content store. + ))), // Content store. ); - let store = Pin::new(&store_owned); const DATA_SIZE: usize = 30; let original_data = make_random_data(DATA_SIZE); @@ -229,7 +224,7 @@ mod dedup_store_tests { #[nativelink_test] async fn check_chunk_boundary_reads_test() -> Result<(), Error> { - let store_owned = DedupStore::new( + let store = DedupStore::new( &nativelink_config::stores::DedupStore { index_store: nativelink_config::stores::StoreConfig::memory( nativelink_config::stores::MemoryStore::default(), @@ -242,14 +237,13 @@ mod dedup_store_tests { max_size: 7, max_concurrent_fetch_per_get: 10, }, - Arc::new(MemoryStore::new( + Store::new(Arc::new(MemoryStore::new( &nativelink_config::stores::MemoryStore::default(), - )), // Index store. - Arc::new(MemoryStore::new( + ))), // Index store. + Store::new(Arc::new(MemoryStore::new( &nativelink_config::stores::MemoryStore::default(), - )), // Content store. + ))), // Content store. ); - let store = Pin::new(&store_owned); const DATA_SIZE: usize = 30; let original_data = make_random_data(DATA_SIZE); @@ -320,26 +314,22 @@ mod dedup_store_tests { let store = DedupStore::new( &make_default_config(), - index_store.clone(), - content_store.clone(), + Store::new(index_store.clone()), + Store::new(content_store.clone()), ); - let store_pin = Pin::new(&store); const DATA_SIZE: usize = MEGABYTE_SZ / 4; let original_data = make_random_data(DATA_SIZE); let digest1 = DigestInfo::try_new(VALID_HASH1, DATA_SIZE).unwrap(); - store_pin + store .update_oneshot(digest1, original_data.clone().into()) .await .err_tip(|| "Failed to write data to dedup store")?; { // Check to ensure we our baseline `.has()` succeeds. - let size_info = store_pin - .has(digest1) - .await - .err_tip(|| "Failed to run .has")?; + let size_info = store.has(digest1).await.err_tip(|| "Failed to run .has")?; assert_eq!(size_info, Some(DATA_SIZE), "Expected sizes to match"); } { @@ -348,25 +338,19 @@ mod dedup_store_tests { // By doing this, we now check that it returns false when we call `.has()`. const DATA2: &str = "1234"; let digest2 = DigestInfo::try_new(VALID_HASH2, DATA2.len()).unwrap(); - store_pin + store .update_oneshot(digest2, DATA2.into()) .await .err_tip(|| "Failed to write data to dedup store")?; { // Check our recently added entry is still valid. - let size_info = store_pin - .has(digest2) - .await - .err_tip(|| "Failed to run .has")?; + let size_info = store.has(digest2).await.err_tip(|| "Failed to run .has")?; assert_eq!(size_info, Some(DATA2.len()), "Expected sizes to match"); } { // Check our first added entry is now invalid (because part of it was evicted). - let size_info = store_pin - .has(digest1) - .await - .err_tip(|| "Failed to run .has")?; + let size_info = store.has(digest1).await.err_tip(|| "Failed to run .has")?; assert_eq!( size_info, None, "Expected .has() to return None (not found)" @@ -393,19 +377,15 @@ mod dedup_store_tests { let store = DedupStore::new( &make_default_config(), - index_store.clone(), - content_store.clone(), + Store::new(index_store.clone()), + Store::new(content_store.clone()), ); - let store_pin = Pin::new(&store); const DATA_SIZE: usize = 10; let digest = DigestInfo::try_new(VALID_HASH1, DATA_SIZE).unwrap(); { - let size_info = store_pin - .has(digest) - .await - .err_tip(|| "Failed to run .has")?; + let size_info = store.has(digest).await.err_tip(|| "Failed to run .has")?; assert_eq!( size_info, None, "Expected None to be returned, got {:?}", diff --git a/nativelink-store/tests/existence_store_test.rs b/nativelink-store/tests/existence_store_test.rs index be7c50ea7..d1eea1304 100644 --- a/nativelink-store/tests/existence_store_test.rs +++ b/nativelink-store/tests/existence_store_test.rs @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::pin::Pin; use std::sync::Arc; use nativelink_config::stores::{ExistenceCacheStore as ExistenceCacheStoreConfig, StoreConfig}; @@ -21,7 +20,7 @@ use nativelink_macro::nativelink_test; use nativelink_store::existence_cache_store::ExistenceCacheStore; use nativelink_store::memory_store::MemoryStore; use nativelink_util::common::DigestInfo; -use nativelink_util::store_trait::Store; +use nativelink_util::store_trait::{Store, StoreLike}; #[cfg(test)] mod verify_store_tests { @@ -36,11 +35,10 @@ mod verify_store_tests { backend: StoreConfig::noop, // Note: Not used. eviction_policy: Default::default(), }; - let inner_store = Arc::new(MemoryStore::new( + let inner_store = Store::new(Arc::new(MemoryStore::new( &nativelink_config::stores::MemoryStore::default(), - )); - let store_owned = ExistenceCacheStore::new(&config, inner_store.clone()); - let store = Pin::new(&store_owned); + ))); + let store = ExistenceCacheStore::new(&config, inner_store.clone()); let digest = DigestInfo::try_new(VALID_HASH1, 3).unwrap(); store @@ -77,13 +75,13 @@ mod verify_store_tests { backend: StoreConfig::noop, eviction_policy: Default::default(), }; - let inner_store = Arc::new(MemoryStore::new( + let inner_store = Store::new(Arc::new(MemoryStore::new( &nativelink_config::stores::MemoryStore::default(), - )); + ))); let store = ExistenceCacheStore::new(&config, inner_store.clone()); let digest = DigestInfo::try_new(VALID_HASH1, 3).unwrap(); - Pin::new(&store) + store .update_oneshot(digest, VALUE.into()) .await .err_tip(|| "Failed to update store")?; @@ -102,17 +100,17 @@ mod verify_store_tests { backend: StoreConfig::noop, eviction_policy: Default::default(), }; - let inner_store = Arc::new(MemoryStore::new( + let inner_store = Store::new(Arc::new(MemoryStore::new( &nativelink_config::stores::MemoryStore::default(), - )); + ))); let digest = DigestInfo::try_new(VALID_HASH1, 3).unwrap(); - Pin::new(inner_store.as_ref()) + inner_store .update_oneshot(digest, VALUE.into()) .await .err_tip(|| "Failed to update store")?; let store = ExistenceCacheStore::new(&config, inner_store.clone()); - let _ = Pin::new(&store) + let _ = store .get_part_unchunked(digest, 0, None) .await .err_tip(|| "Expected get_part to succeed")?; diff --git a/nativelink-store/tests/fast_slow_store_test.rs b/nativelink-store/tests/fast_slow_store_test.rs index 04988daec..efb51563d 100644 --- a/nativelink-store/tests/fast_slow_store_test.rs +++ b/nativelink-store/tests/fast_slow_store_test.rs @@ -13,28 +13,33 @@ // limitations under the License. use std::pin::Pin; -use std::sync::Arc; +use std::sync::atomic::{AtomicBool, Ordering}; +use std::sync::{Arc, Mutex}; -use nativelink_error::Error; +use async_trait::async_trait; +use bytes::Bytes; +use nativelink_error::{make_err, Code, Error, ResultExt}; use nativelink_macro::nativelink_test; use nativelink_store::fast_slow_store::FastSlowStore; use nativelink_store::memory_store::MemoryStore; use nativelink_store::noop_store::NoopStore; +use nativelink_util::buf_channel::make_buf_channel_pair; use nativelink_util::common::DigestInfo; -use nativelink_util::store_trait::Store; +use nativelink_util::health_utils::{default_health_status_indicator, HealthStatusIndicator}; +use nativelink_util::store_trait::{Store, StoreDriver, StoreLike}; use rand::rngs::SmallRng; use rand::{Rng, SeedableRng}; const MEGABYTE_SZ: usize = 1024 * 1024; -fn make_stores() -> (Arc, Arc, Arc) { - let fast_store = Arc::new(MemoryStore::new( +fn make_stores() -> (Store, Store, Store) { + let fast_store = Store::new(Arc::new(MemoryStore::new( &nativelink_config::stores::MemoryStore::default(), - )); - let slow_store = Arc::new(MemoryStore::new( + ))); + let slow_store = Store::new(Arc::new(MemoryStore::new( &nativelink_config::stores::MemoryStore::default(), - )); - let fast_slow_store = Arc::new(FastSlowStore::new( + ))); + let fast_slow_store = Store::new(FastSlowStore::new( &nativelink_config::stores::FastSlowStore { fast: nativelink_config::stores::StoreConfig::memory( nativelink_config::stores::MemoryStore::default(), @@ -56,8 +61,8 @@ fn make_random_data(sz: usize) -> Vec { value } -async fn check_data( - check_store: Pin<&S>, +async fn check_data( + check_store: &Store, digest: DigestInfo, original_data: &Vec, debug_name: &str, @@ -77,14 +82,7 @@ async fn check_data( #[cfg(test)] mod fast_slow_store_tests { - use std::sync::atomic::{AtomicBool, Ordering}; - use std::sync::Mutex; - - use async_trait::async_trait; - use bytes::Bytes; - use nativelink_error::{make_err, Code, ResultExt}; - use nativelink_util::buf_channel::make_buf_channel_pair; - use nativelink_util::health_utils::{default_health_status_indicator, HealthStatusIndicator}; + use pretty_assertions::assert_eq; use super::*; // Must be declared in every module. @@ -94,7 +92,6 @@ mod fast_slow_store_tests { #[nativelink_test] async fn write_large_amount_to_both_stores_test() -> Result<(), Error> { let (store, fast_store, slow_store) = make_stores(); - let store = Pin::new(store.as_ref()); let original_data = make_random_data(20 * MEGABYTE_SZ); let digest = DigestInfo::try_new(VALID_HASH, 100).unwrap(); @@ -102,21 +99,9 @@ mod fast_slow_store_tests { .update_oneshot(digest, original_data.clone().into()) .await?; - check_data(store, digest, &original_data, "fast_slow").await?; - check_data( - Pin::new(fast_store.as_ref()), - digest, - &original_data, - "fast", - ) - .await?; - check_data( - Pin::new(slow_store.as_ref()), - digest, - &original_data, - "slow", - ) - .await?; + check_data(&store, digest, &original_data, "fast_slow").await?; + check_data(&fast_store, digest, &original_data, "fast").await?; + check_data(&slow_store, digest, &original_data, "slow").await?; Ok(()) } @@ -124,9 +109,6 @@ mod fast_slow_store_tests { #[nativelink_test] async fn fetch_slow_store_puts_in_fast_store_test() -> Result<(), Error> { let (fast_slow_store, fast_store, slow_store) = make_stores(); - let fast_slow_store = Pin::new(fast_slow_store.as_ref()); - let fast_store = Pin::new(fast_store.as_ref()); - let slow_store = Pin::new(slow_store.as_ref()); let original_data = make_random_data(MEGABYTE_SZ); let digest = DigestInfo::try_new(VALID_HASH, 100).unwrap(); @@ -145,8 +127,8 @@ mod fast_slow_store_tests { fast_slow_store.get_part_unchunked(digest, 0, None).await?; // Now the data should exist in all the stores. - check_data(fast_store, digest, &original_data, "fast_store").await?; - check_data(slow_store, digest, &original_data, "slow_store").await?; + check_data(&fast_store, digest, &original_data, "fast_store").await?; + check_data(&slow_store, digest, &original_data, "slow_store").await?; Ok(()) } @@ -154,9 +136,6 @@ mod fast_slow_store_tests { #[nativelink_test] async fn partial_reads_copy_full_to_fast_store_test() -> Result<(), Error> { let (fast_slow_store, fast_store, slow_store) = make_stores(); - let fast_slow_store = Pin::new(fast_slow_store.as_ref()); - let fast_store = Pin::new(fast_store.as_ref()); - let slow_store = Pin::new(slow_store.as_ref()); let original_data = make_random_data(MEGABYTE_SZ); let digest = DigestInfo::try_new(VALID_HASH, 100).unwrap(); @@ -174,8 +153,8 @@ mod fast_slow_store_tests { // Full data should exist in the fast store even though only partially // read. - check_data(slow_store, digest, &original_data, "slow_store").await?; - check_data(fast_store, digest, &original_data, "fast_store").await?; + check_data(&slow_store, digest, &original_data, "slow_store").await?; + check_data(&fast_store, digest, &original_data, "fast_store").await?; Ok(()) } @@ -266,7 +245,7 @@ mod fast_slow_store_tests { } #[async_trait] - impl Store for DropCheckStore { + impl StoreDriver for DropCheckStore { async fn has_with_results( self: Pin<&Self>, digests: &[DigestInfo], @@ -303,7 +282,7 @@ mod fast_slow_store_tests { Ok(()) } - async fn get_part_ref( + async fn get_part( self: Pin<&Self>, digest: DigestInfo, writer: &mut nativelink_util::buf_channel::DropCloserWriteHalf, @@ -318,11 +297,7 @@ mod fast_slow_store_tests { writer.send_eof() } - fn inner_store(&self, _digest: Option) -> &'_ dyn Store { - self - } - - fn inner_store_arc(self: Arc, _digest: Option) -> Arc { + fn inner_store(&self, _digest: Option) -> &'_ dyn StoreDriver { self } @@ -353,21 +328,21 @@ mod fast_slow_store_tests { let (fast_store_read_tx, fast_store_read_rx) = tokio::sync::oneshot::channel(); let (fast_store_eof_tx, fast_store_eof_rx) = tokio::sync::oneshot::channel(); let fast_store_dropped = Arc::new(AtomicBool::new(false)); - let fast_store: Arc = Arc::new(DropCheckStore { + let fast_store = Store::new(Arc::new(DropCheckStore { drop_flag: fast_store_dropped.clone(), eof_tx: Mutex::new(Some(fast_store_eof_tx)), read_rx: Mutex::new(Some(fast_store_read_rx)), digest: None, - }); + })); let slow_store_dropped = Arc::new(AtomicBool::new(false)); - let slow_store: Arc = Arc::new(DropCheckStore { + let slow_store = Store::new(Arc::new(DropCheckStore { drop_flag: slow_store_dropped, eof_tx: Mutex::new(None), read_rx: Mutex::new(None), digest: Some(digest), - }); + })); - let fast_slow_store = Arc::new(FastSlowStore::new( + let fast_slow_store = FastSlowStore::new( &nativelink_config::stores::FastSlowStore { fast: nativelink_config::stores::StoreConfig::memory( nativelink_config::stores::MemoryStore::default(), @@ -378,15 +353,15 @@ mod fast_slow_store_tests { }, fast_store, slow_store, - )); + ); let (tx, mut rx) = make_buf_channel_pair(); let (get_res, read_res) = tokio::join!( async move { - // Drop get_part_arc as soon as rx.drain() completes + // Drop get_part as soon as rx.drain() completes tokio::select!( res = rx.drain() => res, - res = fast_slow_store.get_part_arc(digest, tx, 0, Some(digest.size_bytes as usize)) => res, + res = fast_slow_store.get_part(digest, tx, 0, Some(digest.size_bytes as usize)) => res, ) }, async move { @@ -410,12 +385,12 @@ mod fast_slow_store_tests { #[nativelink_test] async fn ignore_value_in_fast_store() -> Result<(), Error> { - let fast_store = Arc::new(MemoryStore::new( + let fast_store = Store::new(Arc::new(MemoryStore::new( &nativelink_config::stores::MemoryStore::default(), - )); - let slow_store = Arc::new(MemoryStore::new( + ))); + let slow_store = Store::new(Arc::new(MemoryStore::new( &nativelink_config::stores::MemoryStore::default(), - )); + ))); let fast_slow_store = Arc::new(FastSlowStore::new( &nativelink_config::stores::FastSlowStore { fast: nativelink_config::stores::StoreConfig::memory( @@ -429,14 +404,11 @@ mod fast_slow_store_tests { slow_store, )); let digest = DigestInfo::try_new(VALID_HASH, 100).unwrap(); - Pin::new(fast_store.as_ref()) + fast_store .update_oneshot(digest, make_random_data(100).into()) .await?; assert!( - Pin::new(fast_slow_store.as_ref()) - .has(digest) - .await? - .is_none(), + fast_slow_store.has(digest).await?.is_none(), "Expected data to not exist in store" ); Ok(()) @@ -445,10 +417,10 @@ mod fast_slow_store_tests { // Regression test for https://github.com/TraceMachina/nativelink/issues/665 #[nativelink_test] async fn has_checks_fast_store_when_noop() -> Result<(), Error> { - let fast_store = Arc::new(MemoryStore::new( + let fast_store = Store::new(Arc::new(MemoryStore::new( &nativelink_config::stores::MemoryStore::default(), - )); - let slow_store = Arc::new(NoopStore::new()); + ))); + let slow_store = Store::new(Arc::new(NoopStore::new())); let fast_slow_store_config = nativelink_config::stores::FastSlowStore { fast: nativelink_config::stores::StoreConfig::memory( nativelink_config::stores::MemoryStore::default(), @@ -465,26 +437,24 @@ mod fast_slow_store_tests { let digest = DigestInfo::try_new(VALID_HASH, data.len()).unwrap(); assert_eq!( - Pin::new(fast_slow_store.as_ref()).has(digest).await, + fast_slow_store.has(digest).await, Ok(None), "Expected data to not exist in store" ); // Upload some dummy data. - Pin::new(fast_store.as_ref()) + fast_store .update_oneshot(digest, data.clone().into()) .await?; assert_eq!( - Pin::new(fast_slow_store.as_ref()).has(digest).await, + fast_slow_store.has(digest).await, Ok(Some(data.len())), "Expected data to exist in store" ); assert_eq!( - Pin::new(fast_slow_store.as_ref()) - .get_part_unchunked(digest, 0, None) - .await, + fast_slow_store.get_part_unchunked(digest, 0, None).await, Ok(data.into()), "Data read from store is not correct" ); diff --git a/nativelink-store/tests/filesystem_store_test.rs b/nativelink-store/tests/filesystem_store_test.rs index 8d467f544..3fac7d5f6 100644 --- a/nativelink-store/tests/filesystem_store_test.rs +++ b/nativelink-store/tests/filesystem_store_test.rs @@ -18,7 +18,6 @@ use std::fmt::{Debug, Formatter}; use std::marker::PhantomData; use std::ops::DerefMut; use std::path::Path; -use std::pin::Pin; use std::sync::atomic::{AtomicBool, AtomicU32, Ordering}; use std::sync::Arc; use std::time::{Duration, SystemTime}; @@ -40,7 +39,7 @@ use nativelink_util::buf_channel::make_buf_channel_pair; use nativelink_util::common::{fs, DigestInfo}; use nativelink_util::evicting_map::LenEntry; use nativelink_util::origin_context::ContextAwareFuture; -use nativelink_util::store_trait::{Store, UploadSizeInfo}; +use nativelink_util::store_trait::{Store, StoreLike, UploadSizeInfo}; use nativelink_util::{background_spawn, spawn}; use once_cell::sync::Lazy; use parking_lot::Mutex; @@ -266,7 +265,7 @@ mod filesystem_store_tests { let content_path = make_temp_path("content_path"); let temp_path = make_temp_path("temp_path"); { - let store = Box::pin( + let store = Store::new( FilesystemStore::::new( &nativelink_config::stores::FilesystemStore { content_path: content_path.clone(), @@ -280,10 +279,10 @@ mod filesystem_store_tests { ); // Insert dummy value into store. - store.as_ref().update_oneshot(digest, VALUE1.into()).await?; + store.update_oneshot(digest, VALUE1.into()).await?; assert_eq!( - store.as_ref().has(digest).await, + store.has(digest).await, Ok(Some(VALUE1.len())), "Expected filesystem store to have hash: {}", HASH1 @@ -303,7 +302,7 @@ mod filesystem_store_tests { .await?, ); - let content = store.as_ref().get_part_unchunked(digest, 0, None).await?; + let content = store.get_part_unchunked(digest, 0, None).await?; assert_eq!(content, VALUE1.as_bytes()); } @@ -340,10 +339,7 @@ mod filesystem_store_tests { .await?, ); - store - .as_ref() - .update_oneshot(digest1, VALUE1.into()) - .await?; + store.update_oneshot(digest1, VALUE1.into()).await?; let expected_file_name = OsString::from(format!( "{}/{}-{}", @@ -362,10 +358,7 @@ mod filesystem_store_tests { } // Replace content. - store - .as_ref() - .update_oneshot(digest1, VALUE2.into()) - .await?; + store.update_oneshot(digest1, VALUE2.into()).await?; { // Check to ensure our file now has new content. @@ -432,23 +425,15 @@ mod filesystem_store_tests { .await?, ); - let store_pin = Pin::new(store.as_ref()); // Insert data into store. - store_pin - .as_ref() - .update_oneshot(digest1, VALUE1.into()) - .await?; + store.update_oneshot(digest1, VALUE1.into()).await?; let (writer, mut reader) = make_buf_channel_pair(); let store_clone = store.clone(); let digest1_clone = digest1; background_spawn!( "file_continues_to_stream_on_content_replace_test_store_get", - async move { - Pin::new(store_clone.as_ref()) - .get(digest1_clone, writer) - .await - }, + async move { store_clone.get(digest1_clone, writer).await }, ); { @@ -465,10 +450,7 @@ mod filesystem_store_tests { } // Replace content. - store_pin - .as_ref() - .update_oneshot(digest1, VALUE2.into()) - .await?; + store.update_oneshot(digest1, VALUE2.into()).await?; // Ensure we let any background tasks finish. tokio::task::yield_now().await; @@ -566,19 +548,15 @@ mod filesystem_store_tests { .await?, ); - let store_pin = Pin::new(store.as_ref()); // Insert data into store. - store_pin - .as_ref() - .update_oneshot(digest1, VALUE1.into()) - .await?; + store.update_oneshot(digest1, VALUE1.into()).await?; let mut reader = { let (writer, reader) = make_buf_channel_pair(); let store_clone = store.clone(); background_spawn!( "file_gets_cleans_up_on_cache_eviction_store_get", - async move { Pin::new(store_clone.as_ref()).get(digest1, writer).await }, + async move { store_clone.get(digest1, writer).await }, ); reader }; @@ -587,10 +565,7 @@ mod filesystem_store_tests { assert!(reader.peek().await.is_ok(), "Could not peek into reader"); // Insert new content. This will evict the old item. - store_pin - .as_ref() - .update_oneshot(digest2, VALUE2.into()) - .await?; + store.update_oneshot(digest2, VALUE2.into()).await?; // Ensure we let any background tasks finish. tokio::task::yield_now().await; @@ -664,10 +639,7 @@ mod filesystem_store_tests { .await?, ); // Insert data into store. - store - .as_ref() - .update_oneshot(digest1, VALUE1.into()) - .await?; + store.update_oneshot(digest1, VALUE1.into()).await?; let file_entry = store.get_file_entry_for_digest(&digest1).await?; file_entry @@ -685,7 +657,7 @@ mod filesystem_store_tests { .await?; // Now touch digest1. - let data = store.as_ref().get_part_unchunked(digest1, 0, None).await?; + let data = store.get_part_unchunked(digest1, 0, None).await?; assert_eq!(data, VALUE1.as_bytes()); file_entry @@ -769,10 +741,7 @@ mod filesystem_store_tests { .await?, ); // Insert data into store. - store - .as_ref() - .update_oneshot(digest1, VALUE1.into()) - .await?; + store.update_oneshot(digest1, VALUE1.into()).await?; let file_entry = store.get_file_entry_for_digest(&digest1).await?; file_entry @@ -790,7 +759,7 @@ mod filesystem_store_tests { .await?; // Now touch digest1. - let data = store.as_ref().get_part_unchunked(digest1, 0, None).await?; + let data = store.get_part_unchunked(digest1, 0, None).await?; assert_eq!(data, VALUE1.as_bytes()); file_entry @@ -822,7 +791,7 @@ mod filesystem_store_tests { .await?, ); // Insert data into store. - store.as_ref().update_oneshot(digest, VALUE1.into()).await?; + store.update_oneshot(digest, VALUE1.into()).await?; let file_entry = store.get_file_entry_for_digest(&digest).await?; { // The file contents should equal our initial data. @@ -837,7 +806,7 @@ mod filesystem_store_tests { } // Now replace the data. - store.as_ref().update_oneshot(digest, VALUE2.into()).await?; + store.update_oneshot(digest, VALUE2.into()).await?; { // The file contents still equal our old data. @@ -894,13 +863,9 @@ mod filesystem_store_tests { ); // Insert data into store. store - .as_ref() .update_oneshot(small_digest, SMALL_VALUE.into()) .await?; - store - .as_ref() - .update_oneshot(big_digest, BIG_VALUE.into()) - .await?; + store.update_oneshot(big_digest, BIG_VALUE.into()).await?; { // Our first digest should have been unrefed exactly once. @@ -948,7 +913,7 @@ mod filesystem_store_tests { ); let (mut tx, rx) = make_buf_channel_pair(); - let update_fut = Arc::new(async_lock::Mutex::new(store.as_ref().update( + let update_fut = Arc::new(async_lock::Mutex::new(store.update( digest, rx, UploadSizeInfo::MaxSize(100), @@ -1050,7 +1015,7 @@ mod filesystem_store_tests { // Finally ensure that our entry is not in the store. assert_eq!( - store.as_ref().has(digest).await?, + store.has(digest).await?, None, "Entry should not be in store" ); @@ -1079,10 +1044,7 @@ mod filesystem_store_tests { .await?, ); - let store_pin = Pin::new(store.as_ref()); - - store_pin - .as_ref() + store .update_oneshot(digest, large_value.clone().into()) .await?; @@ -1091,9 +1053,7 @@ mod filesystem_store_tests { let digest_clone = digest; let _drop_guard = spawn!("get_part_timeout_test_get", async move { - Pin::new(store_clone.as_ref()) - .get(digest_clone, writer) - .await + store_clone.get(digest_clone, writer).await }); let file_data = reader @@ -1137,11 +1097,11 @@ mod filesystem_store_tests { let store_clone = store.clone(); let (mut writer, mut reader) = make_buf_channel_pair(); - let _drop_guard = spawn!("get_part_is_zero_digest_get_part_ref", async move { - let _ = Pin::new(store_clone.as_ref()) - .get_part_ref(digest, &mut writer, 0, None) + let _drop_guard = spawn!("get_part_is_zero_digest_get_part", async move { + let _ = store_clone + .get_part(digest, &mut writer, 0, None) .await - .err_tip(|| "Failed to get_part_ref"); + .err_tip(|| "Failed to get_part"); }); let file_data = reader @@ -1181,10 +1141,10 @@ mod filesystem_store_tests { let digests = vec![digest]; let mut results = vec![None]; - let _ = Pin::new(store.as_ref()) + let _ = store .has_with_results(&digests, &mut results) .await - .err_tip(|| "Failed to get_part_ref"); + .err_tip(|| "Failed to get_part"); assert_eq!(results, vec!(Some(0))); async fn wait_for_empty_content_file< @@ -1265,7 +1225,7 @@ mod filesystem_store_tests { // Populate our first store entry. let first_file_entry = { - store.as_ref().update_oneshot(digest, VALUE1.into()).await?; + store.update_oneshot(digest, VALUE1.into()).await?; store.get_file_entry_for_digest(&digest).await? }; @@ -1276,7 +1236,7 @@ mod filesystem_store_tests { // 4. Then drop the lock. { let rename_pause_request_lock = RENAME_REQUEST_PAUSE_MUX.lock().await; - let mut update_fut = store.as_ref().update_oneshot(digest, VALUE2.into()).boxed(); + let mut update_fut = store.update_oneshot(digest, VALUE2.into()).boxed(); loop { // Try to advance our update future. @@ -1341,7 +1301,7 @@ mod filesystem_store_tests { .await?, ); - store.as_ref().update_oneshot(digest, VALUE1.into()).await?; + store.update_oneshot(digest, VALUE1.into()).await?; let stored_file_path = OsString::from(format!( "{}/{}-{}", @@ -1352,7 +1312,6 @@ mod filesystem_store_tests { std::fs::remove_file(stored_file_path)?; let digest_result = store - .as_ref() .has(digest) .await .err_tip(|| "Failed to execute has")?; @@ -1392,24 +1351,12 @@ mod filesystem_store_tests { .await?, ); - store - .as_ref() - .update_oneshot(digest_1kb, value_1kb.into()) - .await?; - let short_entry = store - .as_ref() - .get_file_entry_for_digest(&digest_1kb) - .await?; + store.update_oneshot(digest_1kb, value_1kb.into()).await?; + let short_entry = store.get_file_entry_for_digest(&digest_1kb).await?; assert_eq!(short_entry.size_on_disk(), 4 * 1024); - store - .as_ref() - .update_oneshot(digest_5kb, value_5kb.into()) - .await?; - let long_entry = store - .as_ref() - .get_file_entry_for_digest(&digest_5kb) - .await?; + store.update_oneshot(digest_5kb, value_5kb.into()).await?; + let long_entry = store.get_file_entry_for_digest(&digest_5kb).await?; assert_eq!(long_entry.size_on_disk(), 8 * 1024); Ok(()) } @@ -1446,10 +1393,7 @@ mod filesystem_store_tests { }) .await?, ); - store - .as_ref() - .update_oneshot(digest, value.clone().into()) - .await?; + store.update_oneshot(digest, value.clone().into()).await?; let mut file = fs::create_file(OsString::from(format!("{temp_path}/dummy_file"))).await?; { @@ -1460,7 +1404,6 @@ mod filesystem_store_tests { } store - .as_ref() .update_with_whole_file(digest, file, UploadSizeInfo::ExactSize(value.len())) .await?; Ok(()) @@ -1487,7 +1430,7 @@ mod filesystem_store_tests { let digest = DigestInfo::try_new(HASH1, value.len())?; - let store = Box::pin(FastSlowStore::new( + let store = FastSlowStore::new( // Note: The config is not needed for this test, so use dummy data. &nativelink_config::stores::FastSlowStore { fast: nativelink_config::stores::StoreConfig::memory( @@ -1497,7 +1440,7 @@ mod filesystem_store_tests { nativelink_config::stores::MemoryStore::default(), ), }, - Arc::new( + Store::new( FilesystemStore::::new( &nativelink_config::stores::FilesystemStore { content_path: make_temp_path("content_path"), @@ -1508,7 +1451,7 @@ mod filesystem_store_tests { ) .await?, ), - Arc::new( + Store::new( FilesystemStore::::new( &nativelink_config::stores::FilesystemStore { content_path: make_temp_path("content_path1"), @@ -1519,11 +1462,8 @@ mod filesystem_store_tests { ) .await?, ), - )); - store - .as_ref() - .update_oneshot(digest, value.clone().into()) - .await?; + ); + store.update_oneshot(digest, value.clone().into()).await?; let temp_path = make_temp_path("temp_path2"); fs::create_dir_all(&temp_path).await?; @@ -1536,7 +1476,6 @@ mod filesystem_store_tests { } store - .as_ref() .update_with_whole_file(digest, file, UploadSizeInfo::ExactSize(value.len())) .await?; Ok(()) @@ -1581,7 +1520,6 @@ mod filesystem_store_tests { .ino(); let result = store - .as_ref() .update_with_whole_file(digest, file, UploadSizeInfo::ExactSize(value.len())) .await?; assert!( diff --git a/nativelink-store/tests/memory_store_test.rs b/nativelink-store/tests/memory_store_test.rs index 4dcfe74c6..74cd44955 100644 --- a/nativelink-store/tests/memory_store_test.rs +++ b/nativelink-store/tests/memory_store_test.rs @@ -23,7 +23,7 @@ use nativelink_store::memory_store::MemoryStore; use nativelink_util::buf_channel::make_buf_channel_pair; use nativelink_util::common::DigestInfo; use nativelink_util::spawn; -use nativelink_util::store_trait::Store; +use nativelink_util::store_trait::StoreLike; use sha2::{Digest, Sha256}; const VALID_HASH1: &str = "0123456789abcdef000000000000000000010000000000000123456789abcdef"; @@ -44,8 +44,7 @@ mod memory_store_tests { async fn insert_one_item_then_update() -> Result<(), Error> { const VALUE1: &str = "13"; const VALUE2: &str = "23"; - let store_owned = MemoryStore::new(&nativelink_config::stores::MemoryStore::default()); - let store = Pin::new(&store_owned); + let store = MemoryStore::new(&nativelink_config::stores::MemoryStore::default()); // Insert dummy value into store. store @@ -258,9 +257,9 @@ mod memory_store_tests { let _drop_guard = spawn!("get_part_is_zero_digest", async move { let _ = Pin::new(store_clone.as_ref()) - .get_part_ref(digest, &mut writer, 0, None) + .get_part(digest, &mut writer, 0, None) .await - .err_tip(|| "Failed to get_part_ref"); + .err_tip(|| "Failed to get_part"); }); let file_data = reader @@ -290,7 +289,7 @@ mod memory_store_tests { .as_ref() .has_with_results(&digests, &mut results) .await - .err_tip(|| "Failed to get_part_ref"); + .err_tip(|| "Failed to get_part"); assert_eq!(results, vec!(Some(0))); Ok(()) diff --git a/nativelink-store/tests/redis_store_test.rs b/nativelink-store/tests/redis_store_test.rs index dc052e3cb..cce5303a7 100644 --- a/nativelink-store/tests/redis_store_test.rs +++ b/nativelink-store/tests/redis_store_test.rs @@ -12,15 +12,14 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::pin::Pin; - use bytes::Bytes; use nativelink_error::Error; use nativelink_macro::nativelink_test; use nativelink_store::cas_utils::ZERO_BYTE_DIGESTS; use nativelink_store::redis_store::{LazyConnection, RedisStore}; +use nativelink_util::buf_channel::make_buf_channel_pair; use nativelink_util::common::DigestInfo; -use nativelink_util::store_trait::Store; +use nativelink_util::store_trait::{StoreLike, UploadSizeInfo}; use redis::{Pipeline, RedisError}; use redis_test::{MockCmd, MockRedisConnection}; @@ -39,9 +38,6 @@ fn mock_uuid_generator() -> String { #[cfg(test)] mod redis_store_tests { - use nativelink_util::buf_channel::make_buf_channel_pair; - use nativelink_util::store_trait::UploadSizeInfo; - use super::*; struct MockRedisConnectionBuilder { @@ -114,17 +110,16 @@ mod redis_store_tests { LazyConnection::Connection(Ok(redis_connection)), mock_uuid_generator, ); - let pinned_store: Pin<&RedisStore> = Pin::new(&store); - pinned_store.update_oneshot(digest, data.clone()).await?; + store.update_oneshot(digest, data.clone()).await?; - let result = pinned_store.has(digest).await?; + let result = store.has(digest).await?; assert!( result.is_some(), "Expected redis store to have hash: {VALID_HASH1}", ); - let result = pinned_store + let result = store .get_part_unchunked(digest, 0, Some(data.clone().len())) .await?; @@ -145,11 +140,10 @@ mod redis_store_tests { LazyConnection::Connection(Ok(redis_connection)), mock_uuid_generator, ); - let pinned_store: Pin<&RedisStore> = Pin::new(&store); - pinned_store.update_oneshot(digest, data).await?; + store.update_oneshot(digest, data).await?; - let result = pinned_store.has(digest).await?; + let result = store.has(digest).await?; assert!( result.is_some(), "Expected redis store to have hash: {VALID_HASH1}", @@ -205,17 +199,16 @@ mod redis_store_tests { LazyConnection::Connection(Ok(redis_connection)), mock_uuid_generator, ); - let pinned_store: Pin<&RedisStore> = Pin::new(&store); - pinned_store.update_oneshot(digest, data.clone()).await?; + store.update_oneshot(digest, data.clone()).await?; - let result = pinned_store.has(digest).await?; + let result = store.has(digest).await?; assert!( result.is_some(), "Expected redis store to have hash: {VALID_HASH1}", ); - let get_result: Bytes = pinned_store + let get_result: Bytes = store .get_part_unchunked(digest, 0, Some(data.clone().len())) .await?; @@ -272,24 +265,23 @@ mod redis_store_tests { LazyConnection::Connection(Ok(redis_connection)), mock_uuid_generator, ); - let pinned_store: Pin<&RedisStore> = Pin::new(&store); let (mut tx, rx) = make_buf_channel_pair(); tx.send(data_p1).await?; tokio::task::yield_now().await; tx.send(data_p2).await?; tx.send_eof()?; - pinned_store + store .update(digest, rx, UploadSizeInfo::ExactSize(data.len())) .await?; - let result = pinned_store.has(digest).await?; + let result = store.has(digest).await?; assert!( result.is_some(), "Expected redis store to have hash: {VALID_HASH1}", ); - let result = pinned_store + let result = store .get_part_unchunked(digest, 0, Some(data.clone().len())) .await?; diff --git a/nativelink-store/tests/ref_store_test.rs b/nativelink-store/tests/ref_store_test.rs index 173f17633..7cdc8a40f 100644 --- a/nativelink-store/tests/ref_store_test.rs +++ b/nativelink-store/tests/ref_store_test.rs @@ -12,50 +12,50 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::pin::Pin; use std::sync::Arc; +use nativelink_error::Error; +use nativelink_macro::nativelink_test; +use nativelink_store::memory_store::MemoryStore; +use nativelink_store::ref_store::RefStore; +use nativelink_store::store_manager::StoreManager; +use nativelink_util::common::DigestInfo; +use nativelink_util::store_trait::{Store, StoreDriver, StoreLike}; + #[cfg(test)] mod ref_store_tests { - use nativelink_error::Error; - use nativelink_macro::nativelink_test; - use nativelink_store::memory_store::MemoryStore; - use nativelink_store::ref_store::RefStore; - use nativelink_store::store_manager::StoreManager; - use nativelink_util::common::DigestInfo; - use nativelink_util::store_trait::Store; use pretty_assertions::assert_eq; // Must be declared in every module. use super::*; const VALID_HASH1: &str = "0123456789abcdef000000000000000000010000000000000123456789abcdef"; - fn setup_stores() -> (Arc, Arc, Arc) { + fn setup_stores() -> (Arc, Store, Store) { let store_manager = Arc::new(StoreManager::new()); - let memory_store_owned = Arc::new(MemoryStore::new( + let memory_store = Store::new(Arc::new(MemoryStore::new( &nativelink_config::stores::MemoryStore::default(), - )); - store_manager.add_store("foo", memory_store_owned.clone()); + ))); + store_manager.add_store("foo", memory_store.clone()); - let ref_store_owned = Arc::new(RefStore::new( + let ref_store = Store::new(Arc::new(RefStore::new( &nativelink_config::stores::RefStore { name: "foo".to_string(), }, Arc::downgrade(&store_manager), - )); - store_manager.add_store("bar", ref_store_owned.clone()); - (store_manager, memory_store_owned, ref_store_owned) + ))); + store_manager.add_store("bar", ref_store.clone()); + (store_manager, memory_store, ref_store) } #[nativelink_test] async fn has_test() -> Result<(), Error> { - let (_store_manager, memory_store_owned, ref_store_owned) = setup_stores(); + let (_store_manager, memory_store, ref_store) = setup_stores(); const VALUE1: &str = "13"; { // Insert data into memory store. - Pin::new(memory_store_owned.as_ref()) + memory_store .update_oneshot( DigestInfo::try_new(VALID_HASH1, VALUE1.len())?, VALUE1.into(), @@ -64,7 +64,7 @@ mod ref_store_tests { } { // Now check if we check of ref_store has the data. - let has_result = Pin::new(ref_store_owned.as_ref()) + let has_result = ref_store .has(DigestInfo::try_new(VALID_HASH1, VALUE1.len())?) .await; assert_eq!( @@ -79,12 +79,12 @@ mod ref_store_tests { #[nativelink_test] async fn get_test() -> Result<(), Error> { - let (_store_manager, memory_store_owned, ref_store_owned) = setup_stores(); + let (_store_manager, memory_store, ref_store) = setup_stores(); const VALUE1: &str = "13"; { // Insert data into memory store. - Pin::new(memory_store_owned.as_ref()) + memory_store .update_oneshot( DigestInfo::try_new(VALID_HASH1, VALUE1.len())?, VALUE1.into(), @@ -93,7 +93,7 @@ mod ref_store_tests { } { // Now check if we read it from ref_store it has same data. - let data = Pin::new(ref_store_owned.as_ref()) + let data = ref_store .get_part_unchunked(DigestInfo::try_new(VALID_HASH1, VALUE1.len())?, 0, None) .await .expect("Get should have succeeded"); @@ -109,12 +109,12 @@ mod ref_store_tests { #[nativelink_test] async fn update_test() -> Result<(), Error> { - let (_store_manager, memory_store_owned, ref_store_owned) = setup_stores(); + let (_store_manager, memory_store, ref_store) = setup_stores(); const VALUE1: &str = "13"; { // Insert data into ref_store. - Pin::new(ref_store_owned.as_ref()) + ref_store .update_oneshot( DigestInfo::try_new(VALID_HASH1, VALUE1.len())?, VALUE1.into(), @@ -123,7 +123,7 @@ mod ref_store_tests { } { // Now check if we read it from memory_store it has same data. - let data = Pin::new(memory_store_owned.as_ref()) + let data = memory_store .get_part_unchunked(DigestInfo::try_new(VALID_HASH1, VALUE1.len())?, 0, None) .await .expect("Get should have succeeded"); @@ -141,31 +141,31 @@ mod ref_store_tests { async fn inner_store_test() -> Result<(), Error> { let store_manager = Arc::new(StoreManager::new()); - let memory_store = Arc::new(MemoryStore::new( + let memory_store = Store::new(Arc::new(MemoryStore::new( &nativelink_config::stores::MemoryStore::default(), - )); + ))); store_manager.add_store("mem_store", memory_store.clone()); - let ref_store_inner = Arc::new(RefStore::new( + let ref_store_inner = Store::new(Arc::new(RefStore::new( &nativelink_config::stores::RefStore { name: "mem_store".to_string(), }, Arc::downgrade(&store_manager), - )); + ))); store_manager.add_store("ref_store_inner", ref_store_inner.clone()); - let ref_store_outer = Arc::new(RefStore::new( + let ref_store_outer = Store::new(Arc::new(RefStore::new( &nativelink_config::stores::RefStore { name: "ref_store_inner".to_string(), }, Arc::downgrade(&store_manager), - )); + ))); store_manager.add_store("ref_store_outer", ref_store_outer.clone()); // Ensure the result of inner_store() points to exact same memory store. assert_eq!( - Arc::as_ptr(&ref_store_outer.inner_store_arc(None)) as *const (), - Arc::as_ptr(&memory_store) as *const (), + ref_store_outer.inner_store(None) as *const dyn StoreDriver as *const (), + memory_store.into_inner().as_ref() as *const dyn StoreDriver as *const (), "Expected inner store to be memory store" ); Ok(()) diff --git a/nativelink-store/tests/s3_store_test.rs b/nativelink-store/tests/s3_store_test.rs index c21048a21..3eaed9889 100644 --- a/nativelink-store/tests/s3_store_test.rs +++ b/nativelink-store/tests/s3_store_test.rs @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::pin::Pin; use std::sync::Arc; use std::time::Duration; @@ -32,7 +31,7 @@ use nativelink_store::s3_store::S3Store; use nativelink_util::buf_channel::make_buf_channel_pair; use nativelink_util::common::DigestInfo; use nativelink_util::spawn; -use nativelink_util::store_trait::{Store, UploadSizeInfo}; +use nativelink_util::store_trait::{StoreLike, UploadSizeInfo}; use sha2::{Digest, Sha256}; // TODO(aaronmondal): Figure out how to test the connector retry mechanism. @@ -70,10 +69,9 @@ mod s3_store_tests { s3_client, Arc::new(move |_delay| Duration::from_secs(0)), )?; - let store_pin = Pin::new(&store); let digest = DigestInfo::try_new(VALID_HASH1, 100).unwrap(); - let result = store_pin.has(digest).await; + let result = store.has(digest).await; assert_eq!( result, Ok(Some(512)), @@ -105,9 +103,8 @@ mod s3_store_tests { s3_client, Arc::new(move |_delay| Duration::from_secs(0)), )?; - let store_pin = Pin::new(&store); let digest = DigestInfo::try_new(VALID_HASH1, 100).unwrap(); - let result = store_pin.has(digest).await; + let result = store.has(digest).await; assert_eq!( result, Ok(None), @@ -170,10 +167,9 @@ mod s3_store_tests { s3_client, Arc::new(move |_delay| Duration::from_secs(0)), )?; - let store_pin = Pin::new(&store); let digest = DigestInfo::try_new(VALID_HASH1, 100).unwrap(); - let result = store_pin.has(digest).await; + let result = store.has(digest).await; assert_eq!( result, Ok(Some(111)), @@ -220,7 +216,7 @@ mod s3_store_tests { // Make future responsible for processing the datastream // and forwarding it to the s3 backend/server. let mut update_fut = Box::pin(async move { - Pin::new(&store) + store .update( DigestInfo::try_new(VALID_HASH1, AC_ENTRY_SIZE)?, rx, @@ -303,9 +299,8 @@ mod s3_store_tests { s3_client, Arc::new(move |_delay| Duration::from_secs(0)), )?; - let store_pin = Pin::new(&store); - let store_data = store_pin + let store_data = store .get_part_unchunked(DigestInfo::try_new(VALID_HASH1, AC_ENTRY_SIZE)?, 0, None) .await?; assert_eq!( @@ -348,9 +343,8 @@ mod s3_store_tests { s3_client, Arc::new(move |_delay| Duration::from_secs(0)), )?; - let store_pin = Pin::new(&store); - store_pin + store .get_part_unchunked( DigestInfo::try_new(VALID_HASH1, AC_ENTRY_SIZE)?, OFFSET, @@ -417,7 +411,7 @@ mod s3_store_tests { )?; let digest = DigestInfo::try_new(VALID_HASH1, 100).unwrap(); - let result = Pin::new(&store).get_part_unchunked(digest, 0, None).await; + let result = store.get_part_unchunked(digest, 0, None).await; assert!(result.is_ok(), "Expected to find item, got: {result:?}"); Ok(()) } @@ -536,9 +530,7 @@ mod s3_store_tests { s3_client, Arc::new(move |_delay| Duration::from_secs(0)), )?; - let _ = Pin::new(&store) - .update_oneshot(digest, send_data.clone().into()) - .await; + let _ = store.update_oneshot(digest, send_data.clone().into()).await; mock_client.assert_requests_match(&[]); Ok(()) } @@ -574,7 +566,6 @@ mod s3_store_tests { s3_client, Arc::new(move |_delay| Duration::from_secs(0)), )?; - let store_pin = Pin::new(&store); let (_, get_part_result) = join!( async move { @@ -583,7 +574,7 @@ mod s3_store_tests { tx.send_data(Bytes::from_static(b"world")).await?; Result::<(), hyper::Error>::Ok(()) }, - store_pin.get_part_unchunked( + store.get_part_unchunked( DigestInfo::try_new(VALID_HASH1, CAS_ENTRY_SIZE)?, 0, Some(CAS_ENTRY_SIZE), @@ -624,10 +615,10 @@ mod s3_store_tests { let (mut writer, mut reader) = make_buf_channel_pair(); let _drop_guard = spawn!("get_part_is_zero_digest", async move { - let _ = Pin::new(store_clone.as_ref()) - .get_part_ref(digest, &mut writer, 0, None) + let _ = store_clone + .get_part(digest, &mut writer, 0, None) .await - .err_tip(|| "Failed to get_part_ref"); + .err_tip(|| "Failed to get_part"); }); let file_data = reader @@ -656,7 +647,7 @@ mod s3_store_tests { .http_client(mock_client) .build(); let s3_client = aws_sdk_s3::Client::from_conf(test_config); - let store_owned = S3Store::new_with_client_and_jitter( + let store = S3Store::new_with_client_and_jitter( &nativelink_config::stores::S3Store { bucket: BUCKET_NAME.to_string(), ..Default::default() @@ -664,13 +655,11 @@ mod s3_store_tests { s3_client, Arc::new(move |_delay| Duration::from_secs(0)), )?; - let store = Pin::new(&store_owned); let _ = store - .as_ref() .has_with_results(&digests, &mut results) .await - .err_tip(|| "Failed to get_part_ref"); + .err_tip(|| "Failed to get_part"); assert_eq!(results, vec!(Some(0))); Ok(()) diff --git a/nativelink-store/tests/shard_store_test.rs b/nativelink-store/tests/shard_store_test.rs index 443cf0789..52f3a37b6 100644 --- a/nativelink-store/tests/shard_store_test.rs +++ b/nativelink-store/tests/shard_store_test.rs @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::pin::Pin; use std::sync::Arc; use nativelink_error::Error; @@ -21,7 +20,7 @@ use nativelink_store::memory_store::MemoryStore; use nativelink_store::shard_store::ShardStore; use nativelink_util::common::DigestInfo; use nativelink_util::digest_hasher::{DigestHasher, DigestHasherFunc}; -use nativelink_util::store_trait::Store; +use nativelink_util::store_trait::{Store, StoreLike}; use rand::rngs::SmallRng; use rand::{Rng, SeedableRng}; @@ -34,12 +33,6 @@ fn make_stores(weights: &[u32]) -> (Arc, Vec>) { .iter() .map(|_| Arc::new(MemoryStore::new(&memory_store_config))) .collect(); - let stores_dyn: Vec<_> = stores - .clone() - .iter() - .cloned() - .map(|x| -> Arc { x }) - .collect(); let shard_store = Arc::new( ShardStore::new( @@ -52,7 +45,10 @@ fn make_stores(weights: &[u32]) -> (Arc, Vec>) { }) .collect(), }, - stores_dyn.clone(), + stores + .iter() + .map(|store| Store::new(store.clone())) + .collect(), ) .unwrap(), ); @@ -73,11 +69,6 @@ async fn verify_weights( print_results: bool, ) -> Result<(), Error> { let (shard_store, stores) = make_stores(weights); - let shard_store = Pin::new(shard_store.as_ref()); - let stores: Vec<_> = stores - .iter() - .map(|store| Pin::new(store.as_ref())) - .collect(); let data = make_random_data(MEGABYTE_SZ); for counter in 0..rounds { @@ -89,16 +80,13 @@ async fn verify_weights( .await?; } - let stores_and_hits: Vec<(&Pin<&MemoryStore>, &usize)> = - stores.iter().zip(expected_hits.iter()).collect(); - - for (index, (&store, &expected_hit)) in stores_and_hits.iter().enumerate() { + for (index, (store, expected_hit)) in stores.iter().zip(expected_hits.iter()).enumerate() { let total_hits = store.len_for_test().await; if print_results { println!("expected_hit: {expected_hit} - total_hits: {total_hits}"); } else { assert_eq!( - expected_hit, total_hits, + *expected_hit, total_hits, "Index {index} failed with expected_hit: {expected_hit} != total_hits: {total_hits}" ) } @@ -119,11 +107,6 @@ mod shard_store_tests { #[nativelink_test] async fn has_with_one_digest() -> Result<(), Error> { let (shard_store, stores) = make_stores(&[1, 1]); - let shard_store = Pin::new(shard_store.as_ref()); - let stores: Vec<_> = stores - .iter() - .map(|store| Pin::new(store.as_ref())) - .collect(); let original_data = make_random_data(MEGABYTE_SZ); let digest1 = DigestInfo::try_new(STORE0_HASH, 100).unwrap(); @@ -138,7 +121,6 @@ mod shard_store_tests { #[nativelink_test] async fn has_with_many_digests_both_missing() -> Result<(), Error> { let (shard_store, _stores) = make_stores(&[1, 1]); - let shard_store = Pin::new(shard_store.as_ref()); let missing_digest1 = DigestInfo::try_new(STORE0_HASH, 100).unwrap(); let missing_digest2 = DigestInfo::try_new(STORE1_HASH, 100).unwrap(); @@ -155,11 +137,6 @@ mod shard_store_tests { #[nativelink_test] async fn has_with_many_digests_one_missing() -> Result<(), Error> { let (shard_store, stores) = make_stores(&[1, 1]); - let shard_store = Pin::new(shard_store.as_ref()); - let stores: Vec<_> = stores - .iter() - .map(|store| Pin::new(store.as_ref())) - .collect(); let original_data = make_random_data(MEGABYTE_SZ); let digest1 = DigestInfo::try_new(STORE0_HASH, 100).unwrap(); @@ -178,11 +155,6 @@ mod shard_store_tests { #[nativelink_test] async fn has_with_many_digests_both_exist() -> Result<(), Error> { let (shard_store, stores) = make_stores(&[1, 1]); - let shard_store = Pin::new(shard_store.as_ref()); - let stores: Vec<_> = stores - .iter() - .map(|store| Pin::new(store.as_ref())) - .collect(); let original_data1 = make_random_data(MEGABYTE_SZ); let original_data2 = make_random_data(2 * MEGABYTE_SZ); @@ -205,11 +177,6 @@ mod shard_store_tests { #[nativelink_test] async fn get_part_reads_store0() -> Result<(), Error> { let (shard_store, stores) = make_stores(&[1, 1]); - let shard_store = Pin::new(shard_store.as_ref()); - let stores: Vec<_> = stores - .iter() - .map(|store| Pin::new(store.as_ref())) - .collect(); let original_data1 = make_random_data(MEGABYTE_SZ); let digest1 = DigestInfo::try_new(STORE0_HASH, 100).unwrap(); @@ -227,11 +194,6 @@ mod shard_store_tests { #[nativelink_test] async fn get_part_reads_store1() -> Result<(), Error> { let (shard_store, stores) = make_stores(&[1, 1]); - let shard_store = Pin::new(shard_store.as_ref()); - let stores: Vec<_> = stores - .iter() - .map(|store| Pin::new(store.as_ref())) - .collect(); let original_data1 = make_random_data(MEGABYTE_SZ); let digest1 = DigestInfo::try_new(STORE1_HASH, 100).unwrap(); @@ -249,11 +211,6 @@ mod shard_store_tests { #[nativelink_test] async fn upload_store0() -> Result<(), Error> { let (shard_store, stores) = make_stores(&[1, 1]); - let shard_store = Pin::new(shard_store.as_ref()); - let stores: Vec<_> = stores - .iter() - .map(|store| Pin::new(store.as_ref())) - .collect(); let original_data1 = make_random_data(MEGABYTE_SZ); let digest1 = DigestInfo::try_new(STORE0_HASH, 100).unwrap(); @@ -271,11 +228,6 @@ mod shard_store_tests { #[nativelink_test] async fn upload_store1() -> Result<(), Error> { let (shard_store, stores) = make_stores(&[1, 1]); - let shard_store = Pin::new(shard_store.as_ref()); - let stores: Vec<_> = stores - .iter() - .map(|store| Pin::new(store.as_ref())) - .collect(); let original_data1 = make_random_data(MEGABYTE_SZ); let digest1 = DigestInfo::try_new(STORE1_HASH, 100).unwrap(); @@ -293,7 +245,6 @@ mod shard_store_tests { #[nativelink_test] async fn upload_download_has_check() -> Result<(), Error> { let (shard_store, _stores) = make_stores(&[1, 1]); - let shard_store = Pin::new(shard_store.as_ref()); let original_data1 = make_random_data(MEGABYTE_SZ); let digest1 = DigestInfo::try_new(STORE1_HASH, 100).unwrap(); @@ -314,11 +265,6 @@ mod shard_store_tests { async fn weights_send_to_proper_store() -> Result<(), Error> { // Very low chance anything will ever go to second store due to weights being so much diff. let (shard_store, stores) = make_stores(&[100000, 1]); - let shard_store = Pin::new(shard_store.as_ref()); - let stores: Vec<_> = stores - .iter() - .map(|store| Pin::new(store.as_ref())) - .collect(); let original_data1 = make_random_data(MEGABYTE_SZ); let digest1 = DigestInfo::try_new(STORE1_HASH, 100).unwrap(); diff --git a/nativelink-store/tests/size_partitioning_store_test.rs b/nativelink-store/tests/size_partitioning_store_test.rs index 6d4815193..25ff0bf40 100644 --- a/nativelink-store/tests/size_partitioning_store_test.rs +++ b/nativelink-store/tests/size_partitioning_store_test.rs @@ -12,17 +12,17 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::pin::Pin; use std::sync::Arc; +use nativelink_error::Error; +use nativelink_macro::nativelink_test; +use nativelink_store::memory_store::MemoryStore; +use nativelink_store::size_partitioning_store::SizePartitioningStore; +use nativelink_util::common::DigestInfo; +use nativelink_util::store_trait::{Store, StoreLike}; + #[cfg(test)] mod ref_store_tests { - use nativelink_error::Error; - use nativelink_macro::nativelink_test; - use nativelink_store::memory_store::MemoryStore; - use nativelink_store::size_partitioning_store::SizePartitioningStore; - use nativelink_util::common::DigestInfo; - use nativelink_util::store_trait::Store; use pretty_assertions::assert_eq; // Must be declared in every module. use super::*; @@ -53,8 +53,8 @@ mod ref_store_tests { nativelink_config::stores::MemoryStore::default(), ), }, - lower_memory_store.clone(), - upper_memory_store.clone(), + Store::new(lower_memory_store.clone()), + Store::new(upper_memory_store.clone()), ); (size_part_store, lower_memory_store, upper_memory_store) } @@ -66,7 +66,7 @@ mod ref_store_tests { { // Insert data into lower store. - Pin::new(lower_memory_store.as_ref()) + lower_memory_store .update_oneshot( DigestInfo::try_new(SMALL_HASH, SMALL_VALUE.len())?, SMALL_VALUE.into(), @@ -74,7 +74,7 @@ mod ref_store_tests { .await?; // Insert data into upper store. - Pin::new(upper_memory_store.as_ref()) + upper_memory_store .update_oneshot( DigestInfo::try_new(BIG_HASH, BIG_VALUE.len())?, BIG_VALUE.into(), @@ -83,7 +83,7 @@ mod ref_store_tests { } { // Check if our partition store has small data. - let small_has_result = Pin::new(&size_part_store) + let small_has_result = size_part_store .has(DigestInfo::try_new(SMALL_HASH, SMALL_VALUE.len())?) .await; assert_eq!( @@ -95,7 +95,7 @@ mod ref_store_tests { } { // Check if our partition store has big data. - let small_has_result = Pin::new(&size_part_store) + let small_has_result = size_part_store .has(DigestInfo::try_new(BIG_HASH, BIG_VALUE.len())?) .await; assert_eq!( @@ -115,7 +115,7 @@ mod ref_store_tests { { // Insert data into lower store. - Pin::new(lower_memory_store.as_ref()) + lower_memory_store .update_oneshot( DigestInfo::try_new(SMALL_HASH, SMALL_VALUE.len())?, SMALL_VALUE.into(), @@ -123,7 +123,7 @@ mod ref_store_tests { .await?; // Insert data into upper store. - Pin::new(upper_memory_store.as_ref()) + upper_memory_store .update_oneshot( DigestInfo::try_new(BIG_HASH, BIG_VALUE.len())?, BIG_VALUE.into(), @@ -132,7 +132,7 @@ mod ref_store_tests { } { // Read the partition store small data. - let data = Pin::new(&size_part_store) + let data = size_part_store .get_part_unchunked(DigestInfo::try_new(SMALL_HASH, SMALL_VALUE.len())?, 0, None) .await .expect("Get should have succeeded"); @@ -145,7 +145,7 @@ mod ref_store_tests { } { // Read the partition store big data. - let data = Pin::new(&size_part_store) + let data = size_part_store .get_part_unchunked(DigestInfo::try_new(BIG_HASH, BIG_VALUE.len())?, 0, None) .await .expect("Get should have succeeded"); @@ -166,7 +166,7 @@ mod ref_store_tests { { // Insert small data into ref_store. - Pin::new(&size_part_store) + size_part_store .update_oneshot( DigestInfo::try_new(SMALL_HASH, SMALL_VALUE.len())?, SMALL_VALUE.into(), @@ -174,7 +174,7 @@ mod ref_store_tests { .await?; // Insert small data into ref_store. - Pin::new(&size_part_store) + size_part_store .update_oneshot( DigestInfo::try_new(BIG_HASH, BIG_VALUE.len())?, BIG_VALUE.into(), @@ -183,7 +183,7 @@ mod ref_store_tests { } { // Check if we read small data from size_partition_store it has same data. - let data = Pin::new(lower_memory_store.as_ref()) + let data = lower_memory_store .get_part_unchunked(DigestInfo::try_new(SMALL_HASH, SMALL_VALUE.len())?, 0, None) .await .expect("Get should have succeeded"); @@ -196,7 +196,7 @@ mod ref_store_tests { } { // Check if we read big data from size_partition_store it has same data. - let data = Pin::new(upper_memory_store.as_ref()) + let data = upper_memory_store .get_part_unchunked(DigestInfo::try_new(BIG_HASH, BIG_VALUE.len())?, 0, None) .await .expect("Get should have succeeded"); diff --git a/nativelink-store/tests/verify_store_test.rs b/nativelink-store/tests/verify_store_test.rs index 989a87062..f983c9d3a 100644 --- a/nativelink-store/tests/verify_store_test.rs +++ b/nativelink-store/tests/verify_store_test.rs @@ -16,22 +16,22 @@ use std::pin::Pin; use std::sync::Arc; use futures::try_join; +use nativelink_error::{Error, ResultExt}; +use nativelink_macro::nativelink_test; +use nativelink_store::memory_store::MemoryStore; +use nativelink_store::verify_store::VerifyStore; +use nativelink_util::buf_channel::make_buf_channel_pair; +use nativelink_util::common::DigestInfo; +use nativelink_util::digest_hasher::{make_ctx_for_hash_func, DigestHasherFunc}; +use nativelink_util::spawn; +use nativelink_util::store_trait::{Store, StoreLike, UploadSizeInfo}; +use tracing::info_span; #[cfg(test)] mod verify_store_tests { - use nativelink_error::{Error, ResultExt}; - use nativelink_macro::nativelink_test; - use nativelink_store::memory_store::MemoryStore; - use nativelink_store::verify_store::VerifyStore; - use nativelink_util::buf_channel::make_buf_channel_pair; - use nativelink_util::common::DigestInfo; - use nativelink_util::digest_hasher::{make_ctx_for_hash_func, DigestHasherFunc}; - use nativelink_util::spawn; - use nativelink_util::store_trait::{Store, UploadSizeInfo}; - use pretty_assertions::assert_eq; // Must be declared in every module. - use tracing::info_span; + use pretty_assertions::assert_eq; - use super::*; + use super::*; // Must be declared in every module. const VALID_HASH1: &str = "0123456789abcdef000000000000000000010000000000000123456789abcdef"; @@ -40,7 +40,7 @@ mod verify_store_tests { let inner_store = Arc::new(MemoryStore::new( &nativelink_config::stores::MemoryStore::default(), )); - let store_owned = VerifyStore::new( + let store = VerifyStore::new( &nativelink_config::stores::VerifyStore { backend: nativelink_config::stores::StoreConfig::memory( nativelink_config::stores::MemoryStore::default(), @@ -48,9 +48,8 @@ mod verify_store_tests { verify_size: false, verify_hash: false, }, - inner_store.clone(), + Store::new(inner_store.clone()), ); - let store = Pin::new(&store_owned); const VALUE1: &str = "123"; let digest = DigestInfo::try_new(VALID_HASH1, 100).unwrap(); @@ -62,7 +61,7 @@ mod verify_store_tests { result ); assert_eq!( - Pin::new(inner_store.as_ref()).has(digest).await, + inner_store.has(digest).await, Ok(Some(VALUE1.len())), "Expected data to exist in store after update" ); @@ -74,7 +73,7 @@ mod verify_store_tests { let inner_store = Arc::new(MemoryStore::new( &nativelink_config::stores::MemoryStore::default(), )); - let store_owned = VerifyStore::new( + let store = VerifyStore::new( &nativelink_config::stores::VerifyStore { backend: nativelink_config::stores::StoreConfig::memory( nativelink_config::stores::MemoryStore::default(), @@ -82,9 +81,8 @@ mod verify_store_tests { verify_size: true, verify_hash: false, }, - inner_store.clone(), + Store::new(inner_store.clone()), ); - let store = Pin::new(&store_owned); const VALUE1: &str = "123"; let digest = DigestInfo::try_new(VALID_HASH1, 100).unwrap(); @@ -105,7 +103,7 @@ mod verify_store_tests { "Error should contain '{EXPECTED_ERR}', got: {err:?}" ); assert_eq!( - Pin::new(inner_store.as_ref()).has(digest).await, + inner_store.has(digest).await, Ok(None), "Expected data to not exist in store after update" ); @@ -117,7 +115,7 @@ mod verify_store_tests { let inner_store = Arc::new(MemoryStore::new( &nativelink_config::stores::MemoryStore::default(), )); - let store_owned = VerifyStore::new( + let store = VerifyStore::new( &nativelink_config::stores::VerifyStore { backend: nativelink_config::stores::StoreConfig::memory( nativelink_config::stores::MemoryStore::default(), @@ -125,16 +123,15 @@ mod verify_store_tests { verify_size: true, verify_hash: false, }, - inner_store.clone(), + Store::new(inner_store.clone()), ); - let store = Pin::new(&store_owned); const VALUE1: &str = "123"; let digest = DigestInfo::try_new(VALID_HASH1, 3).unwrap(); let result = store.update_oneshot(digest, VALUE1.into()).await; assert_eq!(result, Ok(()), "Expected success, got: {:?}", result); assert_eq!( - Pin::new(inner_store.as_ref()).has(digest).await, + inner_store.has(digest).await, Ok(Some(VALUE1.len())), "Expected data to exist in store after update" ); @@ -146,7 +143,7 @@ mod verify_store_tests { let inner_store = Arc::new(MemoryStore::new( &nativelink_config::stores::MemoryStore::default(), )); - let store_owned = VerifyStore::new( + let store = VerifyStore::new( &nativelink_config::stores::VerifyStore { backend: nativelink_config::stores::StoreConfig::memory( nativelink_config::stores::MemoryStore::default(), @@ -154,7 +151,7 @@ mod verify_store_tests { verify_size: true, verify_hash: false, }, - inner_store.clone(), + Store::new(inner_store.clone()), ); let (mut tx, rx) = make_buf_channel_pair(); @@ -164,7 +161,7 @@ mod verify_store_tests { let future = spawn!( "verify_size_true_suceeds_on_multi_chunk_stream_update", async move { - Pin::new(&store_owned) + Pin::new(&store) .update(digest_clone, rx, UploadSizeInfo::ExactSize(6)) .await }, @@ -175,7 +172,7 @@ mod verify_store_tests { let result = future.await.err_tip(|| "Failed to join spawn future")?; assert_eq!(result, Ok(()), "Expected success, got: {:?}", result); assert_eq!( - Pin::new(inner_store.as_ref()).has(digest).await, + inner_store.has(digest).await, Ok(Some(6)), "Expected data to exist in store after update" ); @@ -187,7 +184,7 @@ mod verify_store_tests { let inner_store = Arc::new(MemoryStore::new( &nativelink_config::stores::MemoryStore::default(), )); - let store_owned = VerifyStore::new( + let store = VerifyStore::new( &nativelink_config::stores::VerifyStore { backend: nativelink_config::stores::StoreConfig::memory( nativelink_config::stores::MemoryStore::default(), @@ -195,9 +192,8 @@ mod verify_store_tests { verify_size: false, verify_hash: true, }, - inner_store.clone(), + Store::new(inner_store.clone()), ); - let store = Pin::new(&store_owned); /// This value is sha256("123"). const HASH: &str = "a665a45920422f9d417e4867efdc4fb8a04a1f3fff1fa07e998e86f7f7a27ae3"; @@ -206,7 +202,7 @@ mod verify_store_tests { let result = store.update_oneshot(digest, VALUE.into()).await; assert_eq!(result, Ok(()), "Expected success, got: {:?}", result); assert_eq!( - Pin::new(inner_store.as_ref()).has(digest).await, + inner_store.has(digest).await, Ok(Some(VALUE.len())), "Expected data to exist in store after update" ); @@ -218,7 +214,7 @@ mod verify_store_tests { let inner_store = Arc::new(MemoryStore::new( &nativelink_config::stores::MemoryStore::default(), )); - let store_owned = VerifyStore::new( + let store = VerifyStore::new( &nativelink_config::stores::VerifyStore { backend: nativelink_config::stores::StoreConfig::memory( nativelink_config::stores::MemoryStore::default(), @@ -226,9 +222,8 @@ mod verify_store_tests { verify_size: false, verify_hash: true, }, - inner_store.clone(), + Store::new(inner_store.clone()), ); - let store = Pin::new(&store_owned); /// This value is sha256("12"). const HASH: &str = "6b51d431df5d7f141cbececcf79edf3dd861c3b4069f0b11661a3eefacbba918"; @@ -245,7 +240,7 @@ mod verify_store_tests { "Error should contain '{expected_err}', got: {err:?}" ); assert_eq!( - Pin::new(inner_store.as_ref()).has(digest).await, + inner_store.has(digest).await, Ok(None), "Expected data to not exist in store after update" ); @@ -257,7 +252,7 @@ mod verify_store_tests { let inner_store = Arc::new(MemoryStore::new( &nativelink_config::stores::MemoryStore::default(), )); - let store_owned = VerifyStore::new( + let store = VerifyStore::new( &nativelink_config::stores::VerifyStore { backend: nativelink_config::stores::StoreConfig::memory( nativelink_config::stores::MemoryStore::default(), @@ -265,9 +260,8 @@ mod verify_store_tests { verify_size: false, verify_hash: true, }, - inner_store.clone(), + Store::new(inner_store.clone()), ); - let store = Pin::new(&store_owned); /// This value is blake3("123"). const HASH: &str = "b3d4f8803f7e24b8f389b072e75477cdbcfbe074080fb5e500e53e26e054158e"; @@ -282,7 +276,7 @@ mod verify_store_tests { assert_eq!(result, Ok(()), "Expected success, got: {:?}", result); assert_eq!( - Pin::new(inner_store.as_ref()).has(digest).await, + inner_store.has(digest).await, Ok(Some(VALUE.len())), "Expected data to exist in store after update" ); @@ -294,7 +288,7 @@ mod verify_store_tests { let inner_store = Arc::new(MemoryStore::new( &nativelink_config::stores::MemoryStore::default(), )); - let store_owned = VerifyStore::new( + let store = VerifyStore::new( &nativelink_config::stores::VerifyStore { backend: nativelink_config::stores::StoreConfig::memory( nativelink_config::stores::MemoryStore::default(), @@ -302,9 +296,8 @@ mod verify_store_tests { verify_size: false, verify_hash: true, }, - inner_store.clone(), + Store::new(inner_store.clone()), ); - let store = Pin::new(&store_owned); /// This value is blake3("12"). const HASH: &str = "b944a0a3b20cf5927e594ff306d256d16cd5b0ba3e27b3285f40d7ef5e19695b"; @@ -329,7 +322,7 @@ mod verify_store_tests { "Error should contain '{expected_err}', got: {err:?}" ); assert_eq!( - Pin::new(inner_store.as_ref()).has(digest).await, + inner_store.has(digest).await, Ok(None), "Expected data to not exist in store after update" ); diff --git a/nativelink-util/src/health_utils.rs b/nativelink-util/src/health_utils.rs index 1a1cb3172..1f117bcda 100644 --- a/nativelink-util/src/health_utils.rs +++ b/nativelink-util/src/health_utils.rs @@ -200,7 +200,7 @@ macro_rules! default_health_status_indicator { &self, namespace: std::borrow::Cow<'static, str>, ) -> nativelink_util::health_utils::HealthStatus { - Store::check_health(Pin::new(self), namespace).await + StoreDriver::check_health(Pin::new(self), namespace).await } } }; diff --git a/nativelink-util/src/store_trait.rs b/nativelink-util/src/store_trait.rs index b399dfbac..6347333eb 100644 --- a/nativelink-util/src/store_trait.rs +++ b/nativelink-util/src/store_trait.rs @@ -12,17 +12,18 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::borrow::Cow; +use std::borrow::{BorrowMut, Cow}; use std::collections::hash_map::DefaultHasher as StdHasher; use std::hash::{Hash, Hasher}; use std::ops::Deref; use std::pin::Pin; +use std::ptr::addr_eq; use std::sync::{Arc, OnceLock}; use async_trait::async_trait; use bytes::{Bytes, BytesMut}; use futures::future::{select, Either}; -use futures::{join, try_join, FutureExt}; +use futures::{join, try_join, Future, FutureExt}; use nativelink_error::{error_if, make_err, Code, Error, ResultExt}; use rand::rngs::StdRng; use rand::{RngCore, SeedableRng}; @@ -74,7 +75,7 @@ pub enum UploadSizeInfo { /// Utility to send all the data to the store from a file. // Note: This is not inlined because some code may want to bypass any underlying // optimizations that may be present in the inner store. -pub async fn slow_update_store_with_file( +pub async fn slow_update_store_with_file( store: Pin<&S>, digest: DigestInfo, file: &mut fs::ResumeableFileSlot, @@ -128,11 +129,6 @@ pub async fn slow_update_store_with_file( } } -// TODO(allada) When 1.76.0 stabalizes more we can use `core::ptr::addr_eq` instead. -fn addr_eq(p: *const T, q: *const U) -> bool { - std::ptr::eq(p as *const (), q as *const ()) -} - /// Optimizations that stores may want to expose to the callers. /// This is useful for specific cases when the store can optimize the processing /// of the data being processed. @@ -148,16 +144,99 @@ pub enum StoreOptimizations { NoopDownloads, } -#[async_trait] -pub trait Store: Sync + Send + Unpin + HealthStatusIndicator + 'static { +#[derive(Clone)] +#[repr(transparent)] +pub struct Store { + inner: Arc, +} + +impl Store { + pub fn new(inner: Arc) -> Self { + Self { inner } + } +} + +impl Store { + /// Returns the immediate inner store driver. + /// Note: This does not recursively try to resolve underlying store drivers + /// like `.inner_store()` does. + #[inline] + pub fn into_inner(self) -> Arc { + self.inner + } + + /// Gets the underlying store for the given digest. + /// A caller might want to use this to obtain a reference to the "real" underlying store + /// (if applicable) and check if it implements some special traits that allow optimizations. + /// Note: If the store performs complex operations on the data, it should return itself. + #[inline] + pub fn inner_store(&self, digest: Option) -> &dyn StoreDriver { + self.inner.inner_store(digest) + } + + /// Tries to cast the underlying store to the given type. + #[inline] + pub fn downcast_ref(&self, maybe_digest: Option) -> Option<&U> { + self.inner.inner_store(maybe_digest).as_any().downcast_ref() + } + + /// Register any metrics that this store wants to expose to the Prometheus. + #[inline] + pub fn register_metrics(&self, registry: &mut Registry) { + self.inner.clone().register_metrics(registry) + } + + /// Register health checks used to monitor the store. + #[inline] + pub fn register_health(&self, registry: &mut HealthRegistryBuilder) { + self.inner.clone().register_health(registry) + } +} + +impl StoreLike for Store { + #[inline] + fn as_store_driver(&self) -> &'_ dyn StoreDriver { + self.inner.as_ref() + } + + fn as_pin(&self) -> Pin<&Self> { + Pin::new(self) + } +} + +impl StoreLike for T +where + T: StoreDriver + Sized, +{ + #[inline] + fn as_store_driver(&self) -> &'_ dyn StoreDriver { + self + } + + fn as_pin(&self) -> Pin<&Self> { + Pin::new(self) + } +} + +pub trait StoreLike: Send + Sync + Sized { + /// Returns the immediate inner store driver. + fn as_store_driver(&self) -> &'_ dyn StoreDriver; + + /// Utility function to return a pinned reference to self. + fn as_pin(&self) -> Pin<&Self>; + + /// Utility function to return a pinned reference to the store driver. + #[inline] + fn as_store_driver_pin(&self) -> Pin<&'_ dyn StoreDriver> { + Pin::new(self.as_store_driver()) + } + /// Look up a digest in the store and return None if it does not exist in /// the store, or Some(size) if it does. /// Note: On an AC store the size will be incorrect and should not be used! #[inline] - async fn has(self: Pin<&Self>, digest: DigestInfo) -> Result, Error> { - let mut result = [None]; - self.has_with_results(&[digest], &mut result).await?; - Ok(result[0]) + fn has(&self, digest: DigestInfo) -> impl Future, Error>> + '_ { + self.as_store_driver_pin().has(digest) } /// Look up a list of digests in the store and return a result for each in @@ -165,6 +244,133 @@ pub trait Store: Sync + Send + Unpin + HealthStatusIndicator + 'static { /// exist in the store, or Some(size) if it does. /// Note: On an AC store the size will be incorrect and should not be used! #[inline] + fn has_many<'b>( + &'b self, + digests: &'b [DigestInfo], + ) -> impl Future>, Error>> + Send + 'b { + self.as_store_driver_pin().has_many(digests) + } + + /// The implementation of the above has and has_many functions. See their + /// documentation for details. + #[inline] + fn has_with_results<'b>( + &'b self, + digests: &'b [DigestInfo], + results: &'b mut [Option], + ) -> impl Future> + Send + 'b { + self.as_store_driver_pin() + .has_with_results(digests, results) + } + + /// Sends the data to the store. + #[inline] + fn update( + &self, + digest: DigestInfo, + reader: DropCloserReadHalf, + upload_size: UploadSizeInfo, + ) -> impl Future> + Send + '_ { + self.as_store_driver_pin() + .update(digest, reader, upload_size) + } + + /// Any optimizations the store might want to expose to the callers. + /// By default, no optimizations are exposed. + #[inline] + fn optimized_for(&self, optimization: StoreOptimizations) -> bool { + self.as_store_driver_pin().optimized_for(optimization) + } + + /// Specialized version of `.update()` which takes a `ResumeableFileSlot`. + /// This is useful if the underlying store can optimize the upload process + /// when it knows the data is coming from a file. + #[inline] + fn update_with_whole_file( + &self, + digest: DigestInfo, + file: fs::ResumeableFileSlot, + upload_size: UploadSizeInfo, + ) -> impl Future, Error>> + Send + '_ { + self.as_store_driver_pin() + .update_with_whole_file(digest, file, upload_size) + } + + /// Utility to send all the data to the store when you have all the bytes. + #[inline] + fn update_oneshot( + &self, + digest: DigestInfo, + data: Bytes, + ) -> impl Future> + Send + '_ { + self.as_store_driver_pin().update_oneshot(digest, data) + } + + /// Retreives part of the data from the store and writes it to the given writer. + #[inline] + fn get_part<'b>( + &'b self, + digest: DigestInfo, + mut writer: impl BorrowMut + Send + 'b, + offset: usize, + length: Option, + ) -> impl Future> + Send + 'b { + // Note: We need to capture `writer` just in case the caller + // expects the drop() method to be called on it when the future + // is done due to the complex interaction between the DropCloserWriteHalf + // and the DropCloserReadHalf during drop(). + async move { + self.as_store_driver_pin() + .get_part(digest, writer.borrow_mut(), offset, length) + .await + } + } + + /// Utility that works the same as `.get_part()`, but writes all the data. + #[inline] + fn get( + &self, + digest: DigestInfo, + writer: DropCloserWriteHalf, + ) -> impl Future> + Send + '_ { + self.as_store_driver_pin().get(digest, writer) + } + + /// Utility that will return all the bytes at once instead of in a streaming manner. + #[inline] + fn get_part_unchunked( + &self, + digest: DigestInfo, + offset: usize, + length: Option, + ) -> impl Future> + Send + '_ { + self.as_store_driver_pin() + .get_part_unchunked(digest, offset, length) + } + + /// Default implementation of the health check. Some stores may want to override this + /// in situations where the default implementation is not sufficient. + #[inline] + fn check_health( + &self, + namespace: Cow<'static, str>, + ) -> impl Future + Send + '_ { + self.as_store_driver_pin().check_health(namespace) + } +} + +#[async_trait] +pub trait StoreDriver: Sync + Send + Unpin + HealthStatusIndicator + 'static { + /// See: `StoreLike::has()` for details. + #[inline] + async fn has(self: Pin<&Self>, digest: DigestInfo) -> Result, Error> { + let mut result = [None]; + self.has_with_results(&[digest], &mut result).await?; + Ok(result[0]) + } + + /// See: `StoreLike::has_many()` for details. + #[inline] async fn has_many( self: Pin<&Self>, digests: &[DigestInfo], @@ -174,14 +380,14 @@ pub trait Store: Sync + Send + Unpin + HealthStatusIndicator + 'static { Ok(results) } - /// The implementation of the above has and has_many functions. See their - /// documentation for details. + /// See: `StoreLike::has_with_results()` for details. async fn has_with_results( self: Pin<&Self>, digests: &[DigestInfo], results: &mut [Option], ) -> Result<(), Error>; + /// See: `StoreLike::update()` for details. async fn update( self: Pin<&Self>, digest: DigestInfo, @@ -189,15 +395,12 @@ pub trait Store: Sync + Send + Unpin + HealthStatusIndicator + 'static { upload_size: UploadSizeInfo, ) -> Result<(), Error>; - /// Any optimizations the store might want to expose to the callers. - /// By default, no optimizations are exposed. + /// See: `StoreLike::optimized_for()` for details. fn optimized_for(&self, _optimization: StoreOptimizations) -> bool { false } - /// Specialized version of `.update()` which takes a `ResumeableFileSlot`. - /// This is useful if the underlying store can optimize the upload process - /// when it knows the data is coming from a file. + /// See: `StoreLike::update_with_whole_file()` for details. async fn update_with_whole_file( self: Pin<&Self>, digest: DigestInfo, @@ -218,7 +421,7 @@ pub trait Store: Sync + Send + Unpin + HealthStatusIndicator + 'static { Ok(Some(file)) } - // Utility to send all the data to the store when you have all the bytes. + /// See: `StoreLike::update_oneshot()` for details. async fn update_oneshot( self: Pin<&Self>, digest: DigestInfo, @@ -248,8 +451,8 @@ pub trait Store: Sync + Send + Unpin + HealthStatusIndicator + 'static { Ok(()) } - /// Retreives part of the data from the store and writes it to the given writer. - async fn get_part_ref( + /// See: `StoreLike::get_part()` for details. + async fn get_part( self: Pin<&Self>, digest: DigestInfo, writer: &mut DropCloserWriteHalf, @@ -257,49 +460,17 @@ pub trait Store: Sync + Send + Unpin + HealthStatusIndicator + 'static { length: Option, ) -> Result<(), Error>; - /// Same as `get_part_ref`, but takes ownership of the writer. This is preferred - /// when the writer is definitly not going to be needed after the function returns. - /// This is useful because the read half of the writer will block until the writer - /// is dropped or EOF is sent. If the writer was passed as a reference, and the - /// reader was being waited with the `.get_part()`, it could deadlock if the writer - /// is not dropped or EOF sent. `.get_part_ref()` should be used when the writer - /// might be used after the function returns. - #[inline] - async fn get_part( - self: Pin<&Self>, - digest: DigestInfo, - mut writer: DropCloserWriteHalf, - offset: usize, - length: Option, - ) -> Result<(), Error> { - self.get_part_ref(digest, &mut writer, offset, length).await - } - - /// Utility that works the same as ``.get_part()`, but writes all the data. + /// See: `StoreLike::get()` for details. #[inline] async fn get( self: Pin<&Self>, digest: DigestInfo, - writer: DropCloserWriteHalf, - ) -> Result<(), Error> { - self.get_part(digest, writer, 0, None).await - } - - /// Utility for when `self` is an `Arc` to make the code easier to write. - #[inline] - async fn get_part_arc( - self: Arc, - digest: DigestInfo, - writer: DropCloserWriteHalf, - offset: usize, - length: Option, + mut writer: DropCloserWriteHalf, ) -> Result<(), Error> { - Pin::new(self.as_ref()) - .get_part(digest, writer, offset, length) - .await + self.get_part(digest, &mut writer, 0, None).await } - // Utility that will return all the bytes at once instead of in a streaming manner. + /// See: `StoreLike::get_part_unchunked()` for details. async fn get_part_unchunked( self: Pin<&Self>, digest: DigestInfo, @@ -309,19 +480,20 @@ pub trait Store: Sync + Send + Unpin + HealthStatusIndicator + 'static { // TODO(blaise.bruer) This is extremely inefficient, since we have exactly // what we need here. Maybe we could instead make a version of the stream // that can take objects already fully in memory instead? - let (tx, mut rx) = make_buf_channel_pair(); + let (mut tx, mut rx) = make_buf_channel_pair(); let (data_res, get_part_res) = join!( rx.consume(length), - self.get_part(digest, tx, offset, length), + // We use a closure here to ensure that the `tx` is dropped when the + // future is done. + async move { self.get_part(digest, &mut tx, offset, length).await }, ); get_part_res .err_tip(|| "Failed to get_part in get_part_unchunked") .merge(data_res.err_tip(|| "Failed to read stream to completion in get_part_unchunked")) } - // Default implementation of the health check. Some stores may want to override this - // in situations where the default implementation is not sufficient. + /// See: `StoreLike::check_health()` for details. async fn check_health(self: Pin<&Self>, namespace: Cow<'static, str>) -> HealthStatus { let digest_data_size = default_digest_size_health_check(); let mut digest_data = vec![0u8; digest_data_size]; @@ -399,12 +571,8 @@ pub trait Store: Sync + Send + Unpin + HealthStatusIndicator + 'static { HealthStatus::new_ok(self.get_ref(), "Successfully store health check".into()) } - /// Gets the underlying store for the given digest. - /// A caller might want to use this to obtain a reference to the "real" underlying store - /// (if applicable) and check if it implements some special traits that allow optimizations. - /// Note: If the store performs complex operations on the data, it should return itself. - fn inner_store(&self, _digest: Option) -> &'_ dyn Store; - fn inner_store_arc(self: Arc, _digest: Option) -> Arc; + /// See: `StoreLike::inner_store()` for details. + fn inner_store(&self, _digest: Option) -> &dyn StoreDriver; /// Returns an Any variation of whatever Self is. fn as_any(&self) -> &(dyn std::any::Any + Sync + Send + 'static); diff --git a/nativelink-worker/src/local_worker.rs b/nativelink-worker/src/local_worker.rs index 7e4d433dd..7b4237d15 100644 --- a/nativelink-worker/src/local_worker.rs +++ b/nativelink-worker/src/local_worker.rs @@ -380,14 +380,15 @@ pub struct LocalWorker { /// `FastSlowStore` and will be checked at runtime. pub async fn new_local_worker( config: Arc, - cas_store: Arc, - ac_store: Option>, - historical_store: Arc, + cas_store: Store, + ac_store: Option, + historical_store: Store, ) -> Result, Error> { - let any_store = cas_store.inner_store_arc(None).as_any_arc(); - let fast_slow_store = any_store.downcast::().map_err(|_| { - make_input_err!("Expected store for LocalWorker's store to be a FastSlowStore") - })?; + let fast_slow_store = cas_store + .downcast_ref::(None) + .err_tip(|| "Expected store for LocalWorker's store to be a FastSlowStore")? + .get_arc() + .err_tip(|| "FastSlowStore's Arc doesn't exist")?; if let Ok(path) = fs::canonicalize(&config.work_directory).await { fs::remove_dir_all(path) diff --git a/nativelink-worker/src/running_actions_manager.rs b/nativelink-worker/src/running_actions_manager.rs index 7539c38d3..10fc69bac 100644 --- a/nativelink-worker/src/running_actions_manager.rs +++ b/nativelink-worker/src/running_actions_manager.rs @@ -64,7 +64,7 @@ use nativelink_util::digest_hasher::{DigestHasher, DigestHasherFunc}; use nativelink_util::metrics_utils::{ AsyncCounterWrapper, CollectorState, CounterWithTime, MetricsComponent, }; -use nativelink_util::store_trait::{Store, UploadSizeInfo}; +use nativelink_util::store_trait::{Store, StoreLike, UploadSizeInfo}; use nativelink_util::{background_spawn, spawn, spawn_blocking}; use parking_lot::Mutex; use prost::Message; @@ -121,7 +121,7 @@ struct SideChannelInfo { // of the future. So we need to force this function to return a dynamic future instead. // see: https://github.com/rust-lang/rust/issues/78649 pub fn download_to_directory<'a>( - cas_store: Pin<&'a FastSlowStore>, + cas_store: &'a FastSlowStore, filesystem_store: Pin<&'a FilesystemStore>, digest: &'a DigestInfo, current_directory: &'a str, @@ -256,7 +256,7 @@ fn is_executable(metadata: &std::fs::Metadata, _full_path: &impl AsRef) -> } async fn upload_file( - cas_store: Pin<&dyn Store>, + cas_store: Pin<&impl StoreLike>, full_path: impl AsRef + Debug, hasher: DigestHasherFunc, metadata: std::fs::Metadata, @@ -364,7 +364,7 @@ async fn upload_symlink( } fn upload_directory<'a, P: AsRef + Debug + Send + Sync + Clone + 'a>( - cas_store: Pin<&'a dyn Store>, + cas_store: Pin<&'a impl StoreLike>, full_dir_path: P, full_work_directory: &'a str, hasher: DigestHasherFunc, @@ -664,10 +664,9 @@ impl RunningActionImpl { } let command = { // Download and build out our input files/folders. Also fetch and decode our Command. - let cas_store_pin = Pin::new(self.running_actions_manager.cas_store.as_ref()); let command_fut = self.metrics().get_proto_command_from_store.wrap(async { get_and_decode_digest::( - cas_store_pin, + self.running_actions_manager.cas_store.as_ref(), &self.action_info.command_digest, ) .await @@ -683,7 +682,7 @@ impl RunningActionImpl { self.metrics() .download_to_directory .wrap(download_to_directory( - cas_store_pin, + &self.running_actions_manager.cas_store, filesystem_store_pin, &self.action_info.input_root_digest, &self.work_directory, @@ -1029,7 +1028,7 @@ impl RunningActionImpl { state.execution_metadata.clone(), ) }; - let cas_store = Pin::new(self.running_actions_manager.cas_store.as_ref()); + let cas_store = self.running_actions_manager.cas_store.as_ref(); let hasher = self.action_info.unique_qualifier.digest_function; enum OutputType { None, @@ -1073,7 +1072,7 @@ impl RunningActionImpl { if metadata.is_file() { return Ok(OutputType::File( - upload_file(cas_store, &full_path, hasher, metadata) + upload_file(cas_store.as_pin(), &full_path, hasher, metadata) .await .map(|mut file_info| { file_info.name_or_path = NameOrPath::Path(entry); @@ -1086,7 +1085,7 @@ impl RunningActionImpl { }; if metadata.is_dir() { Ok(OutputType::Directory( - upload_directory(cas_store, &full_path, work_directory, hasher) + upload_directory(cas_store.as_pin(), &full_path, work_directory, hasher) .and_then(|(root_dir, children)| async move { let tree = ProtoTree { root: Some(root_dir), @@ -1094,7 +1093,7 @@ impl RunningActionImpl { }; let tree_digest = serialize_and_upload_message( &tree, - cas_store, + cas_store.as_pin(), &mut hasher.hasher(), ) .await @@ -1389,8 +1388,8 @@ pub struct ExecutionConfiguration { struct UploadActionResults { upload_ac_results_strategy: UploadCacheResultsStrategy, upload_historical_results_strategy: UploadCacheResultsStrategy, - ac_store: Option>, - historical_store: Arc, + ac_store: Option, + historical_store: Store, success_message_template: Template, failure_message_template: Template, } @@ -1398,8 +1397,8 @@ struct UploadActionResults { impl UploadActionResults { fn new( config: &UploadActionResultConfig, - ac_store: Option>, - historical_store: Arc, + ac_store: Option, + historical_store: Store, ) -> Result { let upload_historical_results_strategy = config .upload_historical_results_strategy @@ -1489,12 +1488,11 @@ impl UploadActionResults { action_result: ProtoActionResult, hasher: DigestHasherFunc, ) -> Result<(), Error> { - let Some(ac_store) = self.ac_store.as_deref() else { + let Some(ac_store) = self.ac_store.as_ref() else { return Ok(()); }; // If we are a GrpcStore we shortcut here, as this is a special store. - let any_store = ac_store.inner_store(Some(action_digest)).as_any(); - if let Some(grpc_store) = any_store.downcast_ref::() { + if let Some(grpc_store) = ac_store.downcast_ref::(Some(action_digest)) { let update_action_request = UpdateActionResultRequest { // This is populated by `update_action_result`. instance_name: String::new(), @@ -1515,7 +1513,7 @@ impl UploadActionResults { .encode(&mut store_data) .err_tip(|| "Encoding ActionResult for caching")?; - Pin::new(ac_store) + ac_store .update_oneshot(action_digest, store_data.split().freeze()) .await .err_tip(|| "Caching ActionResult") @@ -1533,7 +1531,7 @@ impl UploadActionResults { action_digest: Some(action_digest.into()), execute_response: Some(execute_response.clone()), }, - Pin::new(self.historical_store.as_ref()), + self.historical_store.as_pin(), &mut hasher.hasher(), ) .await @@ -1625,8 +1623,8 @@ pub struct RunningActionsManagerArgs<'a> { pub root_action_directory: String, pub execution_configuration: ExecutionConfiguration, pub cas_store: Arc, - pub ac_store: Option>, - pub historical_store: Arc, + pub ac_store: Option, + pub historical_store: Store, pub upload_action_result_config: &'a UploadActionResultConfig, pub max_action_timeout: Duration, pub timeout_handled_externally: bool, @@ -1656,17 +1654,15 @@ impl RunningActionsManagerImpl { callbacks: Callbacks, ) -> Result { // Sadly because of some limitations of how Any works we need to clone more times than optimal. - let any_store = args + let filesystem_store = args .cas_store .fast_store() - .clone() - .inner_store_arc(None) - .as_any_arc(); - let filesystem_store = any_store.downcast::().map_err(|_| { - make_input_err!( + .downcast_ref::(None) + .err_tip(|| { "Expected FilesystemStore store for .fast_store() in RunningActionsManagerImpl" - ) - })?; + })? + .get_arc() + .err_tip(|| "FilesystemStore's internal Arc was lost")?; let (action_done_tx, _) = watch::channel(()); Ok(Self { root_action_directory: args.root_action_directory, @@ -1727,10 +1723,9 @@ impl RunningActionsManagerImpl { .err_tip(|| "Expected action_digest to exist on StartExecute")? .try_into()?; let load_start_timestamp = (self.callbacks.now_fn)(); - let action = - get_and_decode_digest::(Pin::new(self.cas_store.as_ref()), &action_digest) - .await - .err_tip(|| "During start_action")?; + let action = get_and_decode_digest::(self.cas_store.as_ref(), &action_digest) + .await + .err_tip(|| "During start_action")?; let action_info = ActionInfo::try_from_action_and_execute_request_with_salt( execute_request, action, diff --git a/nativelink-worker/tests/local_worker_test.rs b/nativelink-worker/tests/local_worker_test.rs index 40f610825..f9a89db92 100644 --- a/nativelink-worker/tests/local_worker_test.rs +++ b/nativelink-worker/tests/local_worker_test.rs @@ -32,11 +32,12 @@ mod utils { use nativelink_config::cas_server::{LocalWorkerConfig, WorkerProperty}; use nativelink_error::{make_err, make_input_err, Code, Error}; use nativelink_macro::nativelink_test; +use nativelink_proto::build::bazel::remote::execution::v2::digest_function; use nativelink_proto::build::bazel::remote::execution::v2::platform::Property; use nativelink_proto::com::github::trace_machina::nativelink::remote_execution::update_for_worker::Update; use nativelink_proto::com::github::trace_machina::nativelink::remote_execution::{ - execute_result, ConnectionResult, ExecuteResult, StartExecute, SupportedProperties, - UpdateForWorker, + execute_result, ConnectionResult, ExecuteResult, KillActionRequest, StartExecute, + SupportedProperties, UpdateForWorker, }; use nativelink_store::fast_slow_store::FastSlowStore; use nativelink_store::filesystem_store::FilesystemStore; @@ -47,6 +48,7 @@ use nativelink_util::action_messages::{ use nativelink_util::common::{encode_stream_proto, fs, DigestInfo}; use nativelink_util::digest_hasher::DigestHasherFunc; use nativelink_util::platform_properties::PlatformProperties; +use nativelink_util::store_trait::Store; use nativelink_worker::local_worker::new_local_worker; use prost::Message; use rand::{thread_rng, Rng}; @@ -72,8 +74,6 @@ fn make_temp_path(data: &str) -> String { #[cfg(test)] mod local_worker_tests { - use nativelink_proto::build::bazel::remote::execution::v2::digest_function; - use nativelink_proto::com::github::trace_machina::nativelink::remote_execution::KillActionRequest; use pretty_assertions::assert_eq; use super::*; // Must be declared in every module. @@ -420,7 +420,7 @@ mod local_worker_tests { #[nativelink_test] async fn new_local_worker_creates_work_directory_test() -> Result<(), Box> { - let cas_store = Arc::new(FastSlowStore::new( + let cas_store = Store::new(FastSlowStore::new( &nativelink_config::stores::FastSlowStore { // Note: These are not needed for this test, so we put dummy memory stores here. fast: nativelink_config::stores::StoreConfig::memory( @@ -430,7 +430,7 @@ mod local_worker_tests { nativelink_config::stores::MemoryStore::default(), ), }, - Arc::new( + Store::new( ::new(&nativelink_config::stores::FilesystemStore { content_path: make_temp_path("content_path"), temp_path: make_temp_path("temp_path"), @@ -438,13 +438,13 @@ mod local_worker_tests { }) .await?, ), - Arc::new(MemoryStore::new( + Store::new(Arc::new(MemoryStore::new( &nativelink_config::stores::MemoryStore::default(), - )), + ))), )); - let ac_store = Arc::new(MemoryStore::new( + let ac_store = Store::new(Arc::new(MemoryStore::new( &nativelink_config::stores::MemoryStore::default(), - )); + ))); let work_directory = make_temp_path("foo"); new_local_worker( Arc::new(LocalWorkerConfig { @@ -468,7 +468,7 @@ mod local_worker_tests { #[nativelink_test] async fn new_local_worker_removes_work_directory_before_start_test( ) -> Result<(), Box> { - let cas_store = Arc::new(FastSlowStore::new( + let cas_store = Store::new(FastSlowStore::new( &nativelink_config::stores::FastSlowStore { // Note: These are not needed for this test, so we put dummy memory stores here. fast: nativelink_config::stores::StoreConfig::memory( @@ -478,7 +478,7 @@ mod local_worker_tests { nativelink_config::stores::MemoryStore::default(), ), }, - Arc::new( + Store::new( ::new(&nativelink_config::stores::FilesystemStore { content_path: make_temp_path("content_path"), temp_path: make_temp_path("temp_path"), @@ -486,13 +486,13 @@ mod local_worker_tests { }) .await?, ), - Arc::new(MemoryStore::new( + Store::new(Arc::new(MemoryStore::new( &nativelink_config::stores::MemoryStore::default(), - )), + ))), )); - let ac_store = Arc::new(MemoryStore::new( + let ac_store = Store::new(Arc::new(MemoryStore::new( &nativelink_config::stores::MemoryStore::default(), - )); + ))); let work_directory = make_temp_path("foo"); fs::create_dir_all(format!("{}/{}", work_directory, "another_dir")).await?; let mut file = diff --git a/nativelink-worker/tests/running_actions_manager_test.rs b/nativelink-worker/tests/running_actions_manager_test.rs index 936db3949..8af415f7d 100644 --- a/nativelink-worker/tests/running_actions_manager_test.rs +++ b/nativelink-worker/tests/running_actions_manager_test.rs @@ -20,7 +20,6 @@ use std::fs::Permissions; use std::io::{Cursor, Write}; #[cfg(target_family = "unix")] use std::os::unix::fs::{MetadataExt, PermissionsExt}; -use std::pin::Pin; use std::str::from_utf8; use std::sync::atomic::{AtomicBool, AtomicI64, AtomicU64, Ordering}; use std::sync::{Arc, Mutex}; @@ -49,7 +48,7 @@ use nativelink_util::action_messages::{ }; use nativelink_util::common::{fs, DigestInfo}; use nativelink_util::digest_hasher::{DigestHasher, DigestHasherFunc}; -use nativelink_util::store_trait::Store; +use nativelink_util::store_trait::{Store, StoreLike}; use nativelink_worker::running_actions_manager::{ download_to_directory, Callbacks, ExecutionConfiguration, RunningAction, RunningActionImpl, RunningActionsManager, RunningActionsManagerArgs, RunningActionsManagerImpl, @@ -80,10 +79,10 @@ fn make_temp_path(data: &str) -> String { async fn setup_stores() -> Result< ( - Pin>, - Pin>, - Pin>, - Pin>, + Arc, + Arc, + Arc, + Arc, ), Error, > { @@ -94,17 +93,17 @@ async fn setup_stores() -> Result< ..Default::default() }; let slow_config = nativelink_config::stores::MemoryStore::default(); - let fast_store = Pin::new(Arc::new(FilesystemStore::new(&fast_config).await?)); - let slow_store = Pin::new(Arc::new(MemoryStore::new(&slow_config))); - let ac_store = Pin::new(Arc::new(MemoryStore::new(&slow_config))); - let cas_store = Pin::new(Arc::new(FastSlowStore::new( + let fast_store = FilesystemStore::new(&fast_config).await?; + let slow_store = Arc::new(MemoryStore::new(&slow_config)); + let ac_store = Arc::new(MemoryStore::new(&slow_config)); + let cas_store = FastSlowStore::new( &nativelink_config::stores::FastSlowStore { fast: nativelink_config::stores::StoreConfig::filesystem(fast_config), slow: nativelink_config::stores::StoreConfig::memory(slow_config), }, - Pin::into_inner(fast_store.clone()), - Pin::into_inner(slow_store.clone()), - ))); + Store::new(fast_store.clone()), + Store::new(slow_store.clone()), + ); Ok((fast_store, slow_store, cas_store, ac_store)) } @@ -214,7 +213,7 @@ mod running_actions_manager_tests { .err_tip(|| format!("Could not make download_dir : {download_dir}"))?; download_to_directory( cas_store.as_ref(), - fast_store.as_ref(), + fast_store.as_pin(), &root_directory_digest, &download_dir, ) @@ -319,7 +318,7 @@ mod running_actions_manager_tests { .err_tip(|| format!("Could not make download_dir : {download_dir}"))?; download_to_directory( cas_store.as_ref(), - fast_store.as_ref(), + fast_store.as_pin(), &root_directory_digest, &download_dir, ) @@ -393,7 +392,7 @@ mod running_actions_manager_tests { .err_tip(|| format!("Could not make download_dir : {download_dir}"))?; download_to_directory( cas_store.as_ref(), - fast_store.as_ref(), + fast_store.as_pin(), &root_directory_digest, &download_dir, ) @@ -434,9 +433,9 @@ mod running_actions_manager_tests { RunningActionsManagerArgs { root_action_directory, execution_configuration: ExecutionConfiguration::default(), - cas_store: Pin::into_inner(cas_store.clone()), - ac_store: Some(Pin::into_inner(ac_store.clone())), - historical_store: Pin::into_inner(cas_store.clone()), + cas_store: cas_store.clone(), + ac_store: Some(Store::new(ac_store.clone())), + historical_store: Store::new(cas_store.clone()), upload_action_result_config: &nativelink_config::cas_server::UploadActionResultConfig { upload_ac_results_strategy: @@ -460,7 +459,7 @@ mod running_actions_manager_tests { }; let command_digest = serialize_and_upload_message( &command, - cas_store.as_ref(), + cas_store.as_pin(), &mut DigestHasherFunc::Sha256.hasher(), ) .await?; @@ -471,7 +470,7 @@ mod running_actions_manager_tests { digest: Some( serialize_and_upload_message( &Directory::default(), - cas_store.as_ref(), + cas_store.as_pin(), &mut DigestHasherFunc::Sha256.hasher(), ) .await? @@ -480,7 +479,7 @@ mod running_actions_manager_tests { }], ..Default::default() }, - cas_store.as_ref(), + cas_store.as_pin(), &mut DigestHasherFunc::Sha256.hasher(), ) .await?; @@ -491,7 +490,7 @@ mod running_actions_manager_tests { }; let action_digest = serialize_and_upload_message( &action, - cas_store.as_ref(), + cas_store.as_pin(), &mut DigestHasherFunc::Sha256.hasher(), ) .await?; @@ -549,9 +548,9 @@ mod running_actions_manager_tests { RunningActionsManagerArgs { root_action_directory, execution_configuration: ExecutionConfiguration::default(), - cas_store: Pin::into_inner(cas_store.clone()), - ac_store: Some(Pin::into_inner(ac_store.clone())), - historical_store: Pin::into_inner(cas_store.clone()), + cas_store: cas_store.clone(), + ac_store: Some(Store::new(ac_store.clone())), + historical_store: Store::new(cas_store.clone()), upload_action_result_config: &nativelink_config::cas_server::UploadActionResultConfig { upload_ac_results_strategy: @@ -577,7 +576,7 @@ mod running_actions_manager_tests { }; let command_digest = serialize_and_upload_message( &command, - cas_store.as_ref(), + cas_store.as_pin(), &mut DigestHasherFunc::Sha256.hasher(), ) .await?; @@ -588,7 +587,7 @@ mod running_actions_manager_tests { digest: Some( serialize_and_upload_message( &Directory::default(), - cas_store.as_ref(), + cas_store.as_pin(), &mut DigestHasherFunc::Sha256.hasher(), ) .await? @@ -597,7 +596,7 @@ mod running_actions_manager_tests { }], ..Default::default() }, - cas_store.as_ref(), + cas_store.as_pin(), &mut DigestHasherFunc::Sha256.hasher(), ) .await?; @@ -608,7 +607,7 @@ mod running_actions_manager_tests { }; let action_digest = serialize_and_upload_message( &action, - cas_store.as_ref(), + cas_store.as_pin(), &mut DigestHasherFunc::Sha256.hasher(), ) .await?; @@ -666,9 +665,9 @@ mod running_actions_manager_tests { RunningActionsManagerArgs { root_action_directory, execution_configuration: ExecutionConfiguration::default(), - cas_store: Pin::into_inner(cas_store.clone()), - ac_store: Some(Pin::into_inner(ac_store.clone())), - historical_store: Pin::into_inner(cas_store.clone()), + cas_store: cas_store.clone(), + ac_store: Some(Store::new(ac_store.clone())), + historical_store: Store::new(cas_store.clone()), upload_action_result_config: &nativelink_config::cas_server::UploadActionResultConfig { upload_ac_results_strategy: @@ -709,7 +708,7 @@ mod running_actions_manager_tests { }; let command_digest = serialize_and_upload_message( &command, - cas_store.as_ref(), + cas_store.as_pin(), &mut DigestHasherFunc::Blake3.hasher(), ) .await?; @@ -720,7 +719,7 @@ mod running_actions_manager_tests { digest: Some( serialize_and_upload_message( &Directory::default(), - cas_store.as_ref(), + cas_store.as_pin(), &mut DigestHasherFunc::Blake3.hasher(), ) .await? @@ -729,7 +728,7 @@ mod running_actions_manager_tests { }], ..Default::default() }, - cas_store.as_ref(), + cas_store.as_pin(), &mut DigestHasherFunc::Blake3.hasher(), ) .await?; @@ -740,7 +739,7 @@ mod running_actions_manager_tests { }; let action_digest = serialize_and_upload_message( &action, - cas_store.as_ref(), + cas_store.as_pin(), &mut DigestHasherFunc::Blake3.hasher(), ) .await?; @@ -838,9 +837,9 @@ mod running_actions_manager_tests { RunningActionsManagerArgs { root_action_directory, execution_configuration: ExecutionConfiguration::default(), - cas_store: Pin::into_inner(cas_store.clone()), - ac_store: Some(Pin::into_inner(ac_store.clone())), - historical_store: Pin::into_inner(cas_store.clone()), + cas_store: cas_store.clone(), + ac_store: Some(Store::new(ac_store.clone())), + historical_store: Store::new(cas_store.clone()), upload_action_result_config: &nativelink_config::cas_server::UploadActionResultConfig { upload_ac_results_strategy: @@ -881,7 +880,7 @@ mod running_actions_manager_tests { }; let command_digest = serialize_and_upload_message( &command, - cas_store.as_ref(), + cas_store.as_pin(), &mut DigestHasherFunc::Sha256.hasher(), ) .await?; @@ -892,7 +891,7 @@ mod running_actions_manager_tests { digest: Some( serialize_and_upload_message( &Directory::default(), - cas_store.as_ref(), + cas_store.as_pin(), &mut DigestHasherFunc::Sha256.hasher(), ) .await? @@ -901,7 +900,7 @@ mod running_actions_manager_tests { }], ..Default::default() }, - cas_store.as_ref(), + cas_store.as_pin(), &mut DigestHasherFunc::Sha256.hasher(), ) .await?; @@ -912,7 +911,7 @@ mod running_actions_manager_tests { }; let action_digest = serialize_and_upload_message( &action, - cas_store.as_ref(), + cas_store.as_pin(), &mut DigestHasherFunc::Sha256.hasher(), ) .await?; @@ -1012,9 +1011,9 @@ mod running_actions_manager_tests { RunningActionsManagerArgs { root_action_directory, execution_configuration: ExecutionConfiguration::default(), - cas_store: Pin::into_inner(cas_store.clone()), - ac_store: Some(Pin::into_inner(ac_store.clone())), - historical_store: Pin::into_inner(cas_store.clone()), + cas_store: cas_store.clone(), + ac_store: Some(Store::new(ac_store.clone())), + historical_store: Store::new(cas_store.clone()), upload_action_result_config: &nativelink_config::cas_server::UploadActionResultConfig { upload_ac_results_strategy: @@ -1051,13 +1050,13 @@ mod running_actions_manager_tests { }; let command_digest = serialize_and_upload_message( &command, - cas_store.as_ref(), + cas_store.as_pin(), &mut DigestHasherFunc::Sha256.hasher(), ) .await?; let input_root_digest = serialize_and_upload_message( &Directory::default(), - cas_store.as_ref(), + cas_store.as_pin(), &mut DigestHasherFunc::Sha256.hasher(), ) .await?; @@ -1068,7 +1067,7 @@ mod running_actions_manager_tests { }; let action_digest = serialize_and_upload_message( &action, - cas_store.as_ref(), + cas_store.as_pin(), &mut DigestHasherFunc::Sha256.hasher(), ) .await?; @@ -1212,9 +1211,9 @@ mod running_actions_manager_tests { RunningActionsManagerArgs { root_action_directory: root_action_directory.clone(), execution_configuration: ExecutionConfiguration::default(), - cas_store: Pin::into_inner(cas_store.clone()), - ac_store: Some(Pin::into_inner(ac_store.clone())), - historical_store: Pin::into_inner(cas_store.clone()), + cas_store: cas_store.clone(), + ac_store: Some(Store::new(ac_store.clone())), + historical_store: Store::new(cas_store.clone()), upload_action_result_config: &nativelink_config::cas_server::UploadActionResultConfig { upload_ac_results_strategy: @@ -1246,13 +1245,13 @@ mod running_actions_manager_tests { }; let command_digest = serialize_and_upload_message( &command, - cas_store.as_ref(), + cas_store.as_pin(), &mut DigestHasherFunc::Sha256.hasher(), ) .await?; let input_root_digest = serialize_and_upload_message( &Directory::default(), - cas_store.as_ref(), + cas_store.as_pin(), &mut DigestHasherFunc::Sha256.hasher(), ) .await?; @@ -1263,7 +1262,7 @@ mod running_actions_manager_tests { }; let action_digest = serialize_and_upload_message( &action, - cas_store.as_ref(), + cas_store.as_pin(), &mut DigestHasherFunc::Sha256.hasher(), ) .await?; @@ -1340,9 +1339,9 @@ mod running_actions_manager_tests { Arc::new(RunningActionsManagerImpl::new(RunningActionsManagerArgs { root_action_directory: root_action_directory.clone(), execution_configuration: ExecutionConfiguration::default(), - cas_store: Pin::into_inner(cas_store.clone()), - ac_store: Some(Pin::into_inner(ac_store.clone())), - historical_store: Pin::into_inner(cas_store.clone()), + cas_store: cas_store.clone(), + ac_store: Some(Store::new(ac_store.clone())), + historical_store: Store::new(cas_store.clone()), upload_action_result_config: &nativelink_config::cas_server::UploadActionResultConfig { upload_ac_results_strategy: @@ -1376,13 +1375,13 @@ mod running_actions_manager_tests { }; let command_digest = serialize_and_upload_message( &command, - cas_store.as_ref(), + cas_store.as_pin(), &mut DigestHasherFunc::Sha256.hasher(), ) .await?; let input_root_digest = serialize_and_upload_message( &Directory::default(), - cas_store.as_ref(), + cas_store.as_pin(), &mut DigestHasherFunc::Sha256.hasher(), ) .await?; @@ -1393,7 +1392,7 @@ mod running_actions_manager_tests { }; let action_digest = serialize_and_upload_message( &action, - cas_store.as_ref(), + cas_store.as_pin(), &mut DigestHasherFunc::Sha256.hasher(), ) .await?; @@ -1500,9 +1499,9 @@ exit 0 entrypoint: Some(test_wrapper_script.into_string().unwrap()), additional_environment: None, }, - cas_store: Pin::into_inner(cas_store.clone()), - ac_store: Some(Pin::into_inner(ac_store.clone())), - historical_store: Pin::into_inner(cas_store.clone()), + cas_store: cas_store.clone(), + ac_store: Some(Store::new(ac_store.clone())), + historical_store: Store::new(cas_store.clone()), upload_action_result_config: &nativelink_config::cas_server::UploadActionResultConfig { upload_ac_results_strategy: @@ -1523,13 +1522,13 @@ exit 0 }; let command_digest = serialize_and_upload_message( &command, - cas_store.as_ref(), + cas_store.as_pin(), &mut DigestHasherFunc::Sha256.hasher(), ) .await?; let input_root_digest = serialize_and_upload_message( &Directory::default(), - cas_store.as_ref(), + cas_store.as_pin(), &mut DigestHasherFunc::Sha256.hasher(), ) .await?; @@ -1540,7 +1539,7 @@ exit 0 }; let action_digest = serialize_and_upload_message( &action, - cas_store.as_ref(), + cas_store.as_pin(), &mut DigestHasherFunc::Sha256.hasher(), ) .await?; @@ -1657,9 +1656,9 @@ exit 0 ), ])), }, - cas_store: Pin::into_inner(cas_store.clone()), - ac_store: Some(Pin::into_inner(ac_store.clone())), - historical_store: Pin::into_inner(cas_store.clone()), + cas_store: cas_store.clone(), + ac_store: Some(Store::new(ac_store.clone())), + historical_store: Store::new(cas_store.clone()), upload_action_result_config: &nativelink_config::cas_server::UploadActionResultConfig { upload_ac_results_strategy: @@ -1680,13 +1679,13 @@ exit 0 }; let command_digest = serialize_and_upload_message( &command, - cas_store.as_ref(), + cas_store.as_pin(), &mut DigestHasherFunc::Sha256.hasher(), ) .await?; let input_root_digest = serialize_and_upload_message( &Directory::default(), - cas_store.as_ref(), + cas_store.as_pin(), &mut DigestHasherFunc::Sha256.hasher(), ) .await?; @@ -1708,7 +1707,7 @@ exit 0 }; let action_digest = serialize_and_upload_message( &action, - cas_store.as_ref(), + cas_store.as_pin(), &mut DigestHasherFunc::Sha256.hasher(), ) .await?; @@ -1813,9 +1812,9 @@ exit 1 EnvironmentSource::side_channel_file, )])), }, - cas_store: Pin::into_inner(cas_store.clone()), - ac_store: Some(Pin::into_inner(ac_store.clone())), - historical_store: Pin::into_inner(cas_store.clone()), + cas_store: cas_store.clone(), + ac_store: Some(Store::new(ac_store.clone())), + historical_store: Store::new(cas_store.clone()), upload_action_result_config: &nativelink_config::cas_server::UploadActionResultConfig { upload_ac_results_strategy: @@ -1833,13 +1832,13 @@ exit 1 }; let command_digest = serialize_and_upload_message( &command, - cas_store.as_ref(), + cas_store.as_pin(), &mut DigestHasherFunc::Sha256.hasher(), ) .await?; let input_root_digest = serialize_and_upload_message( &Directory::default(), - cas_store.as_ref(), + cas_store.as_pin(), &mut DigestHasherFunc::Sha256.hasher(), ) .await?; @@ -1850,7 +1849,7 @@ exit 1 }; let action_digest = serialize_and_upload_message( &action, - cas_store.as_ref(), + cas_store.as_pin(), &mut DigestHasherFunc::Sha256.hasher(), ) .await?; @@ -1888,9 +1887,9 @@ exit 1 Arc::new(RunningActionsManagerImpl::new(RunningActionsManagerArgs { root_action_directory: String::new(), execution_configuration: ExecutionConfiguration::default(), - cas_store: Pin::into_inner(cas_store.clone()), - ac_store: Some(Pin::into_inner(ac_store.clone())), - historical_store: Pin::into_inner(cas_store.clone()), + cas_store: cas_store.clone(), + ac_store: Some(Store::new(ac_store.clone())), + historical_store: Store::new(cas_store.clone()), upload_action_result_config: &nativelink_config::cas_server::UploadActionResultConfig { upload_ac_results_strategy: @@ -1961,9 +1960,9 @@ exit 1 Arc::new(RunningActionsManagerImpl::new(RunningActionsManagerArgs { root_action_directory: String::new(), execution_configuration: ExecutionConfiguration::default(), - cas_store: Pin::into_inner(cas_store.clone()), - ac_store: Some(Pin::into_inner(ac_store.clone())), - historical_store: Pin::into_inner(cas_store.clone()), + cas_store: cas_store.clone(), + ac_store: Some(Store::new(ac_store.clone())), + historical_store: Store::new(cas_store.clone()), upload_action_result_config: &nativelink_config::cas_server::UploadActionResultConfig { upload_ac_results_strategy: @@ -2033,9 +2032,9 @@ exit 1 Arc::new(RunningActionsManagerImpl::new(RunningActionsManagerArgs { root_action_directory: String::new(), execution_configuration: ExecutionConfiguration::default(), - cas_store: Pin::into_inner(cas_store.clone()), - ac_store: Some(Pin::into_inner(ac_store.clone())), - historical_store: Pin::into_inner(cas_store.clone()), + cas_store: cas_store.clone(), + ac_store: Some(Store::new(ac_store.clone())), + historical_store: Store::new(cas_store.clone()), upload_action_result_config: &nativelink_config::cas_server::UploadActionResultConfig { upload_historical_results_strategy: Some( @@ -2134,9 +2133,9 @@ exit 1 Arc::new(RunningActionsManagerImpl::new(RunningActionsManagerArgs { root_action_directory: String::new(), execution_configuration: ExecutionConfiguration::default(), - cas_store: Pin::into_inner(cas_store.clone()), - ac_store: Some(Pin::into_inner(ac_store.clone())), - historical_store: Pin::into_inner(cas_store.clone()), + cas_store: cas_store.clone(), + ac_store: Some(Store::new(ac_store.clone())), + historical_store: Store::new(cas_store.clone()), upload_action_result_config: &nativelink_config::cas_server::UploadActionResultConfig { upload_historical_results_strategy: Some( @@ -2174,9 +2173,9 @@ exit 1 let running_actions_manager = Arc::new(RunningActionsManagerImpl::new(RunningActionsManagerArgs { root_action_directory: String::new(), execution_configuration: ExecutionConfiguration::default(), - cas_store: Pin::into_inner(cas_store.clone()), - ac_store: Some(Pin::into_inner(ac_store.clone())), - historical_store: Pin::into_inner(cas_store.clone()), + cas_store: cas_store.clone(), + ac_store: Some(Store::new(ac_store.clone())), + historical_store: Store::new(cas_store.clone()), upload_action_result_config: &nativelink_config::cas_server::UploadActionResultConfig { upload_historical_results_strategy: Some( nativelink_config::cas_server::UploadCacheResultsStrategy::failures_only, @@ -2240,9 +2239,9 @@ exit 1 Arc::new(RunningActionsManagerImpl::new(RunningActionsManagerArgs { root_action_directory: String::new(), execution_configuration: ExecutionConfiguration::default(), - cas_store: Pin::into_inner(cas_store.clone()), - ac_store: Some(Pin::into_inner(ac_store.clone())), - historical_store: Pin::into_inner(cas_store.clone()), + cas_store: cas_store.clone(), + ac_store: Some(Store::new(ac_store.clone())), + historical_store: Store::new(cas_store.clone()), upload_action_result_config: &nativelink_config::cas_server::UploadActionResultConfig { upload_ac_results_strategy: @@ -2317,13 +2316,13 @@ exit 1 }; let command_digest = serialize_and_upload_message( &command, - cas_store.as_ref(), + cas_store.as_pin(), &mut DigestHasherFunc::Sha256.hasher(), ) .await?; let input_root_digest = serialize_and_upload_message( &Directory::default(), - cas_store.as_ref(), + cas_store.as_pin(), &mut DigestHasherFunc::Sha256.hasher(), ) .await?; @@ -2345,7 +2344,7 @@ exit 1 }; let action_digest = serialize_and_upload_message( &action, - cas_store.as_ref(), + cas_store.as_pin(), &mut DigestHasherFunc::Sha256.hasher(), ) .await?; @@ -2354,9 +2353,9 @@ exit 1 RunningActionsManagerArgs { root_action_directory: root_action_directory.clone(), execution_configuration: ExecutionConfiguration::default(), - cas_store: Pin::into_inner(cas_store.clone()), - ac_store: Some(Pin::into_inner(ac_store.clone())), - historical_store: Pin::into_inner(cas_store.clone()), + cas_store: cas_store.clone(), + ac_store: Some(Store::new(ac_store.clone())), + historical_store: Store::new(cas_store.clone()), upload_action_result_config: &nativelink_config::cas_server::UploadActionResultConfig { upload_ac_results_strategy: @@ -2423,7 +2422,7 @@ exit 1 }; let action_digest = serialize_and_upload_message( &action, - cas_store.as_ref(), + cas_store.as_pin(), &mut DigestHasherFunc::Sha256.hasher(), ) .await?; @@ -2432,9 +2431,9 @@ exit 1 RunningActionsManagerArgs { root_action_directory: root_action_directory.clone(), execution_configuration: ExecutionConfiguration::default(), - cas_store: Pin::into_inner(cas_store.clone()), - ac_store: Some(Pin::into_inner(ac_store.clone())), - historical_store: Pin::into_inner(cas_store.clone()), + cas_store: cas_store.clone(), + ac_store: Some(Store::new(ac_store.clone())), + historical_store: Store::new(cas_store.clone()), upload_action_result_config: &nativelink_config::cas_server::UploadActionResultConfig { upload_ac_results_strategy: @@ -2501,7 +2500,7 @@ exit 1 }; let action_digest = serialize_and_upload_message( &action, - cas_store.as_ref(), + cas_store.as_pin(), &mut DigestHasherFunc::Sha256.hasher(), ) .await?; @@ -2510,9 +2509,9 @@ exit 1 RunningActionsManagerArgs { root_action_directory: root_action_directory.clone(), execution_configuration: ExecutionConfiguration::default(), - cas_store: Pin::into_inner(cas_store.clone()), - ac_store: Some(Pin::into_inner(ac_store.clone())), - historical_store: Pin::into_inner(cas_store.clone()), + cas_store: cas_store.clone(), + ac_store: Some(Store::new(ac_store.clone())), + historical_store: Store::new(cas_store.clone()), upload_action_result_config: &nativelink_config::cas_server::UploadActionResultConfig { upload_ac_results_strategy: @@ -2586,9 +2585,9 @@ exit 1 RunningActionsManagerArgs { root_action_directory: root_action_directory.clone(), execution_configuration: ExecutionConfiguration::default(), - cas_store: Pin::into_inner(cas_store.clone()), - ac_store: Some(Pin::into_inner(ac_store.clone())), - historical_store: Pin::into_inner(cas_store.clone()), + cas_store: cas_store.clone(), + ac_store: Some(Store::new(ac_store.clone())), + historical_store: Store::new(cas_store.clone()), upload_action_result_config: &nativelink_config::cas_server::UploadActionResultConfig { upload_ac_results_strategy: @@ -2632,13 +2631,13 @@ exit 1 }; let command_digest = serialize_and_upload_message( &command, - cas_store.as_ref(), + cas_store.as_pin(), &mut DigestHasherFunc::Sha256.hasher(), ) .await?; let input_root_digest = serialize_and_upload_message( &Directory::default(), - cas_store.as_ref(), + cas_store.as_pin(), &mut DigestHasherFunc::Sha256.hasher(), ) .await?; @@ -2649,7 +2648,7 @@ exit 1 }; let action_digest = serialize_and_upload_message( &action, - cas_store.as_ref(), + cas_store.as_pin(), &mut DigestHasherFunc::Sha256.hasher(), ) .await?; @@ -2709,9 +2708,9 @@ exit 1 RunningActionsManagerArgs { root_action_directory: root_action_directory.clone(), execution_configuration: ExecutionConfiguration::default(), - cas_store: Pin::into_inner(cas_store.clone()), - ac_store: Some(Pin::into_inner(ac_store.clone())), - historical_store: Pin::into_inner(cas_store.clone()), + cas_store: cas_store.clone(), + ac_store: Some(Store::new(ac_store.clone())), + historical_store: Store::new(cas_store.clone()), upload_action_result_config: &nativelink_config::cas_server::UploadActionResultConfig { upload_ac_results_strategy: @@ -2750,13 +2749,13 @@ exit 1 }; let command_digest = serialize_and_upload_message( &command, - cas_store.as_ref(), + cas_store.as_pin(), &mut DigestHasherFunc::Sha256.hasher(), ) .await?; let input_root_digest = serialize_and_upload_message( &Directory::default(), - cas_store.as_ref(), + cas_store.as_pin(), &mut DigestHasherFunc::Sha256.hasher(), ) .await?; @@ -2767,7 +2766,7 @@ exit 1 }; let action_digest = serialize_and_upload_message( &action, - cas_store.as_ref(), + cas_store.as_pin(), &mut DigestHasherFunc::Sha256.hasher(), ) .await?; @@ -2868,10 +2867,10 @@ exit 1 let running_actions_manager = Arc::new(RunningActionsManagerImpl::new_with_callbacks( RunningActionsManagerArgs { root_action_directory, - cas_store: Pin::into_inner(cas_store.clone()), - ac_store: Some(Pin::into_inner(ac_store.clone())), + cas_store: cas_store.clone(), + ac_store: Some(Store::new(ac_store.clone())), execution_configuration: ExecutionConfiguration::default(), - historical_store: Pin::into_inner(cas_store.clone()), + historical_store: Store::new(cas_store.clone()), upload_action_result_config: &nativelink_config::cas_server::UploadActionResultConfig { upload_ac_results_strategy: @@ -2901,13 +2900,13 @@ exit 1 }; let command_digest = serialize_and_upload_message( &command, - cas_store.as_ref(), + cas_store.as_pin(), &mut DigestHasherFunc::Sha256.hasher(), ) .await?; let input_root_digest = serialize_and_upload_message( &Directory::default(), - cas_store.as_ref(), + cas_store.as_pin(), &mut DigestHasherFunc::Sha256.hasher(), ) .await?; @@ -2918,7 +2917,7 @@ exit 1 }; let action_digest = serialize_and_upload_message( &action, - cas_store.as_ref(), + cas_store.as_pin(), &mut DigestHasherFunc::Sha256.hasher(), ) .await?; @@ -2961,9 +2960,9 @@ exit 1 Arc::new(RunningActionsManagerImpl::new(RunningActionsManagerArgs { root_action_directory: root_action_directory.clone(), execution_configuration: ExecutionConfiguration::default(), - cas_store: Pin::into_inner(cas_store.clone()), - ac_store: Some(Pin::into_inner(ac_store.clone())), - historical_store: Pin::into_inner(cas_store.clone()), + cas_store: cas_store.clone(), + ac_store: Some(Store::new(ac_store.clone())), + historical_store: Store::new(cas_store.clone()), upload_action_result_config: &nativelink_config::cas_server::UploadActionResultConfig { upload_ac_results_strategy: @@ -2989,13 +2988,13 @@ exit 1 }; let command_digest = serialize_and_upload_message( &command, - cas_store.as_ref(), + cas_store.as_pin(), &mut DigestHasherFunc::Sha256.hasher(), ) .await?; let input_root_digest = serialize_and_upload_message( &Directory::default(), - cas_store.as_ref(), + cas_store.as_pin(), &mut DigestHasherFunc::Sha256.hasher(), ) .await?; @@ -3006,7 +3005,7 @@ exit 1 }; let action_digest = serialize_and_upload_message( &action, - cas_store.as_ref(), + cas_store.as_pin(), &mut DigestHasherFunc::Sha256.hasher(), ) .await?; @@ -3066,9 +3065,9 @@ exit 1 RunningActionsManagerArgs { root_action_directory, execution_configuration: ExecutionConfiguration::default(), - cas_store: Pin::into_inner(cas_store.clone()), - ac_store: Some(Pin::into_inner(ac_store.clone())), - historical_store: Pin::into_inner(cas_store.clone()), + cas_store: cas_store.clone(), + ac_store: Some(Store::new(ac_store.clone())), + historical_store: Store::new(cas_store.clone()), upload_action_result_config: &nativelink_config::cas_server::UploadActionResultConfig { upload_ac_results_strategy: @@ -3109,7 +3108,7 @@ exit 1 }; let command_digest = serialize_and_upload_message( &command, - cas_store.as_ref(), + cas_store.as_pin(), &mut DigestHasherFunc::Sha256.hasher(), ) .await?; @@ -3120,7 +3119,7 @@ exit 1 digest: Some( serialize_and_upload_message( &Directory::default(), - cas_store.as_ref(), + cas_store.as_pin(), &mut DigestHasherFunc::Sha256.hasher(), ) .await? @@ -3129,7 +3128,7 @@ exit 1 }], ..Default::default() }, - cas_store.as_ref(), + cas_store.as_pin(), &mut DigestHasherFunc::Sha256.hasher(), ) .await?; @@ -3140,7 +3139,7 @@ exit 1 }; let action_digest = serialize_and_upload_message( &action, - cas_store.as_ref(), + cas_store.as_pin(), &mut DigestHasherFunc::Sha256.hasher(), ) .await?; @@ -3248,9 +3247,9 @@ exit 1 RunningActionsManagerArgs { root_action_directory, execution_configuration: Default::default(), - cas_store: Pin::into_inner(cas_store.clone()), - ac_store: Some(Pin::into_inner(ac_store.clone())), - historical_store: Pin::into_inner(cas_store.clone()), + cas_store: cas_store.clone(), + ac_store: Some(Store::new(ac_store.clone())), + historical_store: Store::new(cas_store.clone()), upload_action_result_config: &nativelink_config::cas_server::UploadActionResultConfig { upload_ac_results_strategy: @@ -3286,13 +3285,13 @@ exit 1 }; let command_digest = serialize_and_upload_message( &command, - cas_store.as_ref(), + cas_store.as_pin(), &mut DigestHasherFunc::Sha256.hasher(), ) .await?; let input_root_digest = serialize_and_upload_message( &Directory::default(), - cas_store.as_ref(), + cas_store.as_pin(), &mut DigestHasherFunc::Sha256.hasher(), ) .await?; @@ -3313,7 +3312,7 @@ exit 1 }; let action_digest = serialize_and_upload_message( &action, - cas_store.as_ref(), + cas_store.as_pin(), &mut DigestHasherFunc::Sha256.hasher(), ) .await?;