From c74afeb29106d58a42bd8b377576acd3653a4b1d Mon Sep 17 00:00:00 2001 From: James Kay Date: Thu, 5 Sep 2024 17:20:03 +0100 Subject: [PATCH] Browser time (#2452) * `linera-base`: abstract away from `tokio::time` for the Web * `linera-rpc`: remove path prefix on `Duration` Co-authored-by: Janito Vaqueiro Ferreira Filho Signed-off-by: James Kay --------- Signed-off-by: James Kay Co-authored-by: Janito Vaqueiro Ferreira Filho --- Cargo.lock | 3 +-- examples/Cargo.lock | 1 - linera-base/Cargo.toml | 5 ++++- linera-base/src/lib.rs | 10 +--------- linera-base/src/prometheus_util.rs | 4 ++-- linera-base/src/time.rs | 16 +++++++++++++++ linera-client/src/chain_listener.rs | 2 +- linera-client/src/client_context.rs | 9 +++------ linera-client/src/util.rs | 5 +++-- linera-core/Cargo.toml | 3 --- linera-core/benches/client_benchmarks.rs | 3 +-- linera-core/src/chain_worker/config.rs | 4 ++-- linera-core/src/unit_tests/worker_tests.rs | 4 ++-- linera-core/src/updater.rs | 10 +--------- linera-core/src/worker.rs | 2 +- linera-indexer/example/tests/test.rs | 6 +++--- linera-indexer/lib/src/service.rs | 6 +++--- linera-rpc/src/grpc/client.rs | 5 +++-- linera-rpc/src/grpc/pool.rs | 3 +-- linera-rpc/src/grpc/server.rs | 6 ++++-- linera-rpc/src/grpc/transport.rs | 4 ++-- linera-rpc/src/node_provider.rs | 3 +-- linera-rpc/src/simple/client.rs | 20 +++++++++---------- linera-rpc/src/simple/server.rs | 8 +++++--- linera-rpc/tests/transport.rs | 3 +-- linera-service/benches/transfers.rs | 3 +-- linera-service/src/benchmark.rs | 4 ++-- linera-service/src/cli_wrappers/local_net.rs | 10 +++++----- linera-service/src/cli_wrappers/wallet.rs | 6 +++--- linera-service/src/grpc_proxy.rs | 2 +- linera-service/src/linera/net_up_utils.rs | 6 ++---- linera-service/src/node_service.rs | 2 +- linera-service/src/proxy.rs | 4 ++-- linera-service/tests/linera_net_tests.rs | 9 +++++---- linera-service/tests/local_net_tests.rs | 2 +- linera-storage-service/src/child.rs | 10 ++++------ linera-storage/src/db_storage.rs | 2 +- .../benches/reentrant_collection_view.rs | 3 +-- linera-views/src/context.rs | 2 +- 39 files changed, 101 insertions(+), 109 deletions(-) create mode 100644 linera-base/src/time.rs diff --git a/Cargo.lock b/Cargo.lock index 884953f592de..b9b182c1585e 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -4236,6 +4236,7 @@ dependencies = [ "tracing-subscriber", "tracing-web", "wasm-bindgen-futures", + "wasmtimer", "web-time", "zstd", ] @@ -4333,7 +4334,6 @@ dependencies = [ "async-graphql", "async-trait", "bcs", - "cfg-if", "cfg_aliases", "clap", "counter", @@ -4367,7 +4367,6 @@ dependencies = [ "tracing", "tracing-subscriber", "trait-variant", - "wasmtimer", ] [[package]] diff --git a/examples/Cargo.lock b/examples/Cargo.lock index f0f62699009d..ed60d3cd12f4 100644 --- a/examples/Cargo.lock +++ b/examples/Cargo.lock @@ -3423,7 +3423,6 @@ dependencies = [ "async-graphql", "async-trait", "bcs", - "cfg-if", "cfg_aliases", "clap", "dashmap 5.5.3", diff --git a/linera-base/Cargo.toml b/linera-base/Cargo.toml index 920afad0998c..1d8cbd706159 100644 --- a/linera-base/Cargo.toml +++ b/linera-base/Cargo.toml @@ -21,8 +21,9 @@ web = [ "rand/std", "rand/std_rng", "tracing-web", - "web-time", + "wasmtimer", "wasm-bindgen-futures", + "web-time", ] [dependencies] @@ -52,9 +53,11 @@ serde_json.workspace = true sha3.workspace = true test-strategy = { workspace = true, optional = true } thiserror.workspace = true +tokio = { workspace = true, features = ["time"] } tracing.workspace = true tracing-subscriber = { workspace = true, features = ["json", "fmt", "ansi"] } wasm-bindgen-futures = { workspace = true, optional = true } +wasmtimer = { workspace = true, optional = true } web-time = { workspace = true, optional = true } [target.'cfg(target_arch = "wasm32")'.dependencies] diff --git a/linera-base/src/lib.rs b/linera-base/src/lib.rs index 6c86b4633835..7f744d7c132f 100644 --- a/linera-base/src/lib.rs +++ b/linera-base/src/lib.rs @@ -23,6 +23,7 @@ pub mod ownership; pub mod prometheus_util; #[cfg(not(chain))] pub mod task; +pub mod time; pub mod tracing; #[cfg(test)] mod unit_tests; @@ -31,15 +32,6 @@ pub use graphql::BcsHexParseError; #[doc(hidden)] pub use {async_graphql, bcs, hex}; -cfg_if::cfg_if! { - if #[cfg(web)] { - #[cfg(web)] - pub use web_time as time; - } else { - pub use std::time; - } -} - /// A macro for asserting that a condition is true, returning an error if it is not. /// /// # Examples diff --git a/linera-base/src/prometheus_util.rs b/linera-base/src/prometheus_util.rs index bd74b6c95a7a..bc6d9be54488 100644 --- a/linera-base/src/prometheus_util.rs +++ b/linera-base/src/prometheus_util.rs @@ -3,13 +3,13 @@ //! This module defines util functions for interacting with Prometheus (logging metrics, etc) -use std::time::Instant; - use prometheus::{ histogram_opts, register_histogram_vec, register_int_counter_vec, Error, HistogramVec, IntCounterVec, Opts, }; +use crate::time::Instant; + const LINERA_NAMESPACE: &str = "linera"; /// Wrapper arount prometheus register_int_counter_vec! macro which also sets the linera namespace diff --git a/linera-base/src/time.rs b/linera-base/src/time.rs new file mode 100644 index 000000000000..2b44b7a93814 --- /dev/null +++ b/linera-base/src/time.rs @@ -0,0 +1,16 @@ +// Copyright (c) Zefchain Labs, Inc. +// SPDX-License-Identifier: Apache-2.0 + +/*! +Abstractions over time that can be used natively or on the Web. + */ + +cfg_if::cfg_if! { + if #[cfg(web)] { + pub use web_time::*; + pub use wasmtimer::tokio as timer; + } else { + pub use std::time::*; + pub use tokio::time as timer; + } +} diff --git a/linera-client/src/chain_listener.rs b/linera-client/src/chain_listener.rs index de1fba4cc5de..91bb62accd25 100644 --- a/linera-client/src/chain_listener.rs +++ b/linera-client/src/chain_listener.rs @@ -276,7 +276,7 @@ where async fn maybe_sleep(delay_ms: u64) { if delay_ms > 0 { - tokio::time::sleep(Duration::from_millis(delay_ms)).await; + linera_base::time::timer::sleep(Duration::from_millis(delay_ms)).await; } } } diff --git a/linera-client/src/client_context.rs b/linera-client/src/client_context.rs index d71e01180476..374e44d52085 100644 --- a/linera-client/src/client_context.rs +++ b/linera-client/src/client_context.rs @@ -1,11 +1,7 @@ // Copyright (c) Zefchain Labs, Inc. // SPDX-License-Identifier: Apache-2.0 -use std::{ - collections::BTreeMap, - sync::Arc, - time::{Duration, Instant}, -}; +use std::{collections::BTreeMap, sync::Arc}; use async_trait::async_trait; use futures::Future; @@ -14,6 +10,7 @@ use linera_base::{ data_types::{BlockHeight, Timestamp}, identifiers::{Account, ChainId}, ownership::ChainOwnership, + time::{Duration, Instant}, }; use linera_chain::data_types::Certificate; use linera_core::{ @@ -618,7 +615,7 @@ where let chain_client = self.make_chain_client(chain_id); async move { for i in 0..5 { - tokio::time::sleep(Duration::from_secs(i)).await; + linera_base::time::timer::sleep(Duration::from_secs(i)).await; chain_client.process_inbox().await?; let chain_state = chain_client.chain_state_view().await?; if chain_state diff --git a/linera-client/src/util.rs b/linera-client/src/util.rs index e9ea49122747..e3a4e4d173aa 100644 --- a/linera-client/src/util.rs +++ b/linera-client/src/util.rs @@ -1,13 +1,14 @@ // Copyright (c) Zefchain Labs, Inc. // SPDX-License-Identifier: Apache-2.0 -use std::{collections::HashSet, num::ParseIntError, str::FromStr, time::Duration}; +use std::{collections::HashSet, num::ParseIntError, str::FromStr}; use futures::future; use linera_base::{ crypto::CryptoError, data_types::{TimeDelta, Timestamp}, identifiers::ChainId, + time::Duration, }; use linera_core::{data_types::RoundTimeout, node::NotificationStream, worker::Reason}; use tokio_stream::StreamExt as _; @@ -36,7 +37,7 @@ pub async fn wait_for_next_round(stream: &mut NotificationStream, timeout: Round }); future::select( Box::pin(stream.next()), - Box::pin(tokio::time::sleep( + Box::pin(linera_base::time::timer::sleep( timeout.timestamp.duration_since(Timestamp::now()), )), ) diff --git a/linera-core/Cargo.toml b/linera-core/Cargo.toml index 11eb8b56d327..35ee0ea4451f 100644 --- a/linera-core/Cargo.toml +++ b/linera-core/Cargo.toml @@ -45,7 +45,6 @@ web = [ "linera-execution/web", "linera-storage/web", "linera-views/web", - "wasmtimer", ] [dependencies] @@ -53,7 +52,6 @@ anyhow = { workspace = true, optional = true } async-graphql.workspace = true async-trait.workspace = true bcs.workspace = true -cfg-if.workspace = true clap.workspace = true dashmap.workspace = true futures.workspace = true @@ -77,7 +75,6 @@ tokio-stream.workspace = true tonic.workspace = true tracing.workspace = true trait-variant.workspace = true -wasmtimer = { workspace = true, optional = true } [target.'cfg(not(target_arch = "wasm32"))'.dependencies] linera-storage-service.workspace = true diff --git a/linera-core/benches/client_benchmarks.rs b/linera-core/benches/client_benchmarks.rs index 2056fa2b90cb..2409b764d9f0 100644 --- a/linera-core/benches/client_benchmarks.rs +++ b/linera-core/benches/client_benchmarks.rs @@ -1,12 +1,11 @@ // Copyright (c) Zefchain Labs, Inc. // SPDX-License-Identifier: Apache-2.0 -use std::time::Duration; - use criterion::{criterion_group, criterion_main, measurement::Measurement, BatchSize, Criterion}; use linera_base::{ data_types::Amount, identifiers::{Account, ChainDescription}, + time::Duration, }; use linera_core::{ client, diff --git a/linera-core/src/chain_worker/config.rs b/linera-core/src/chain_worker/config.rs index 4fe2cab25eb1..a652ee2cea5d 100644 --- a/linera-core/src/chain_worker/config.rs +++ b/linera-core/src/chain_worker/config.rs @@ -3,9 +3,9 @@ //! Configuration parameters for the chain worker. -use std::{sync::Arc, time::Duration}; +use std::sync::Arc; -use linera_base::crypto::KeyPair; +use linera_base::{crypto::KeyPair, time::Duration}; /// Configuration parameters for the [`ChainWorkerState`][`super::state::ChainWorkerState`]. #[derive(Clone, Default)] diff --git a/linera-core/src/unit_tests/worker_tests.rs b/linera-core/src/unit_tests/worker_tests.rs index 14b3cc39ac5a..7bc9ed9e9a1e 100644 --- a/linera-core/src/unit_tests/worker_tests.rs +++ b/linera-core/src/unit_tests/worker_tests.rs @@ -3635,7 +3635,7 @@ where } drop(worker); - tokio::time::sleep(Duration::from_millis(10)).await; + linera_base::time::timer::sleep(Duration::from_millis(10)).await; application.assert_no_more_expected_calls(); application.assert_no_active_instances(); @@ -3805,7 +3805,7 @@ where } drop(worker); - tokio::time::sleep(Duration::from_millis(10)).await; + linera_base::time::timer::sleep(Duration::from_millis(10)).await; application.assert_no_more_expected_calls(); application.assert_no_active_instances(); diff --git a/linera-core/src/updater.rs b/linera-core/src/updater.rs index 1095d55a111a..c361eabddb6b 100644 --- a/linera-core/src/updater.rs +++ b/linera-core/src/updater.rs @@ -14,7 +14,7 @@ use linera_base::{ data_types::{BlockHeight, Round}, ensure, identifiers::ChainId, - time::{Duration, Instant}, + time::{timer::timeout, Duration, Instant}, }; use linera_chain::data_types::{BlockProposal, Certificate, LiteVote}; use linera_execution::committee::{Committee, ValidatorName}; @@ -28,14 +28,6 @@ use crate::{ node::{CrossChainMessageDelivery, LocalValidatorNode, NodeError}, }; -cfg_if::cfg_if! { - if #[cfg(web)] { - use wasmtimer::tokio::timeout; - } else { - use tokio::time::timeout; - } -} - /// The amount of time we wait for additional validators to contribute to the result, as a fraction /// of how long it took to reach a quorum. const GRACE_PERIOD: f64 = 0.2; diff --git a/linera-core/src/worker.rs b/linera-core/src/worker.rs index 71677aa1023d..ab70f8c546fa 100644 --- a/linera-core/src/worker.rs +++ b/linera-core/src/worker.rs @@ -17,6 +17,7 @@ use linera_base::{ data_types::{ArithmeticError, Blob, BlockHeight, Round, UserApplicationDescription}, doc_scalar, identifiers::{BlobId, ChainId, Owner, UserApplicationId}, + time::timer::{sleep, timeout}, }; use linera_chain::{ data_types::{ @@ -33,7 +34,6 @@ use thiserror::Error; use tokio::{ sync::{mpsc, oneshot, OwnedRwLockReadGuard}, task::JoinSet, - time::{sleep, timeout}, }; use tracing::{error, instrument, trace, warn, Instrument as _}; #[cfg(with_metrics)] diff --git a/linera-indexer/example/tests/test.rs b/linera-indexer/example/tests/test.rs index 1f5610a544a2..56329617c7ad 100644 --- a/linera-indexer/example/tests/test.rs +++ b/linera-indexer/example/tests/test.rs @@ -48,7 +48,7 @@ async fn run_indexer(path_provider: &PathProvider) -> Child { let child = command.spawn().unwrap(); let client = reqwest_client(); for i in 0..10 { - tokio::time::sleep(Duration::from_secs(i)).await; + linera_base::time::timer::sleep(Duration::from_secs(i)).await; let request = client .get(format!("http://localhost:{}/", port)) .send() @@ -118,9 +118,9 @@ async fn test_end_to_end_operations_indexer(config: impl LineraNetConfig) { let chain1 = ChainId::root(1); for _ in 0..10 { transfer(&req_client, chain0, chain1, "0.1").await; - tokio::time::sleep(Duration::from_millis(TRANSFER_DELAY_MILLIS)).await; + linera_base::time::timer::sleep(Duration::from_millis(TRANSFER_DELAY_MILLIS)).await; } - tokio::time::sleep(Duration::from_secs(2)).await; + linera_base::time::timer::sleep(Duration::from_secs(2)).await; // checking indexer state let variables = block::Variables { diff --git a/linera-indexer/lib/src/service.rs b/linera-indexer/lib/src/service.rs index ab8af4299450..8cc5fa508a84 100644 --- a/linera-indexer/lib/src/service.rs +++ b/linera-indexer/lib/src/service.rs @@ -3,8 +3,6 @@ //! This module defines the service client for the indexer. -use std::time::Duration; - use async_tungstenite::{ tokio::connect_async, tungstenite::{client::IntoClientRequest, http::HeaderValue}, @@ -15,7 +13,9 @@ use futures::{ }; use graphql_client::reqwest::post_graphql; use graphql_ws_client::{graphql::StreamingOperation, GraphQLClientClientBuilder}; -use linera_base::{crypto::CryptoHash, data_types::BlockHeight, identifiers::ChainId}; +use linera_base::{ + crypto::CryptoHash, data_types::BlockHeight, identifiers::ChainId, time::Duration, +}; use linera_chain::data_types::HashedCertificateValue; use linera_core::worker::Reason; use linera_service_graphql_client::{block, chains, notifications, Block, Chains, Notifications}; diff --git a/linera-rpc/src/grpc/client.rs b/linera-rpc/src/grpc/client.rs index 9c94f16fb886..fa8922f3553e 100644 --- a/linera-rpc/src/grpc/client.rs +++ b/linera-rpc/src/grpc/client.rs @@ -1,13 +1,14 @@ // Copyright (c) Zefchain Labs, Inc. // SPDX-License-Identifier: Apache-2.0 -use std::{iter, time::Duration}; +use std::iter; use futures::{future, stream, StreamExt}; use linera_base::{ crypto::CryptoHash, data_types::{Blob, BlobContent}, identifiers::{BlobId, ChainId}, + time::Duration, }; use linera_chain::data_types::{self, Certificate, CertificateValue, HashedCertificateValue}; #[cfg(web)] @@ -248,7 +249,7 @@ impl ValidatorNode for GrpcClient { let delay = notification_retry_delay.saturating_mul(retry_count); retry_count += 1; future::Either::Right(async move { - tokio::time::sleep(delay).await; + linera_base::time::timer::sleep(delay).await; true }) }) diff --git a/linera-rpc/src/grpc/pool.rs b/linera-rpc/src/grpc/pool.rs index a73480d61700..056a993308bd 100644 --- a/linera-rpc/src/grpc/pool.rs +++ b/linera-rpc/src/grpc/pool.rs @@ -1,9 +1,8 @@ // Copyright (c) Zefchain Labs, Inc. // SPDX-License-Identifier: Apache-2.0 -use std::time::Duration; - use dashmap::DashMap; +use linera_base::time::Duration; use super::{transport, GrpcError}; diff --git a/linera-rpc/src/grpc/server.rs b/linera-rpc/src/grpc/server.rs index 64b62b33e293..e9eda342bbd1 100644 --- a/linera-rpc/src/grpc/server.rs +++ b/linera-rpc/src/grpc/server.rs @@ -372,8 +372,10 @@ where for i in 0..cross_chain_max_retries { // Delay increases linearly with the attempt number. - tokio::time::sleep(cross_chain_sender_delay + cross_chain_retry_delay * i) - .await; + linera_base::time::timer::sleep( + cross_chain_sender_delay + cross_chain_retry_delay * i, + ) + .await; let result = || async { let cross_chain_request = cross_chain_request.clone().try_into()?; diff --git a/linera-rpc/src/grpc/transport.rs b/linera-rpc/src/grpc/transport.rs index 0013c5923616..706d53c25d43 100644 --- a/linera-rpc/src/grpc/transport.rs +++ b/linera-rpc/src/grpc/transport.rs @@ -5,8 +5,8 @@ use crate::NodeOptions; #[derive(Clone, Debug, Default)] pub struct Options { - pub connect_timeout: Option, - pub timeout: Option, + pub connect_timeout: Option, + pub timeout: Option, } impl From<&'_ NodeOptions> for Options { diff --git a/linera-rpc/src/node_provider.rs b/linera-rpc/src/node_provider.rs index df7e43631dc6..20bb7285b8ae 100644 --- a/linera-rpc/src/node_provider.rs +++ b/linera-rpc/src/node_provider.rs @@ -1,8 +1,7 @@ // Copyright (c) Zefchain Labs, Inc. // SPDX-License-Identifier: Apache-2.0 -use std::time::Duration; - +use linera_base::time::Duration; use linera_core::node::{LocalValidatorNodeProvider, NodeError}; #[cfg(with_simple_network)] diff --git a/linera-rpc/src/simple/client.rs b/linera-rpc/src/simple/client.rs index 007330687a86..82cfaff2b0ab 100644 --- a/linera-rpc/src/simple/client.rs +++ b/linera-rpc/src/simple/client.rs @@ -2,7 +2,7 @@ // Copyright (c) Zefchain Labs, Inc. // SPDX-License-Identifier: Apache-2.0 -use std::{future::Future, time::Duration}; +use std::future::Future; use async_trait::async_trait; use futures::{sink::SinkExt, stream::StreamExt}; @@ -10,6 +10,7 @@ use linera_base::{ crypto::CryptoHash, data_types::{Blob, BlobContent}, identifiers::{BlobId, ChainId}, + time::{timer, Duration}, }; use linera_chain::data_types::{ BlockProposal, Certificate, CertificateValue, HashedCertificateValue, LiteCertificate, @@ -19,7 +20,6 @@ use linera_core::{ node::{CrossChainMessageDelivery, NodeError, NotificationStream, ValidatorNode}, }; use linera_version::VersionInfo; -use tokio::time; use super::{codec, transport::TransportProtocol}; use crate::{ @@ -51,11 +51,11 @@ impl SimpleClient { let address = format!("{}:{}", self.network.host, self.network.port); let mut stream = self.network.protocol.connect(address).await?; // Send message - time::timeout(self.send_timeout, stream.send(message)) + timer::timeout(self.send_timeout, stream.send(message)) .await .map_err(|timeout| codec::Error::Io(timeout.into()))??; // Wait for reply - time::timeout(self.recv_timeout, stream.next()) + timer::timeout(self.recv_timeout, stream.next()) .await .map_err(|timeout| codec::Error::Io(timeout.into()))? .transpose()? @@ -165,15 +165,15 @@ impl ValidatorNode for SimpleClient { #[derive(Clone)] pub struct SimpleMassClient { pub network: ValidatorPublicNetworkPreConfig, - send_timeout: std::time::Duration, - recv_timeout: std::time::Duration, + send_timeout: Duration, + recv_timeout: Duration, } impl SimpleMassClient { pub fn new( network: ValidatorPublicNetworkPreConfig, - send_timeout: std::time::Duration, - recv_timeout: std::time::Duration, + send_timeout: Duration, + recv_timeout: Duration, ) -> Self { Self { network, @@ -208,7 +208,7 @@ impl mass_client::MassClient for SimpleMassClient { } Some(request) => request, }; - let status = time::timeout(self.send_timeout, stream.send(request)).await; + let status = timer::timeout(self.send_timeout, stream.send(request)).await; if let Err(error) = status { tracing::error!("Failed to send request: {}", error); continue; @@ -218,7 +218,7 @@ impl mass_client::MassClient for SimpleMassClient { if requests.len() % 5000 == 0 && requests.len() > 0 { tracing::info!("In flight {} Remaining {}", in_flight, requests.len()); } - match time::timeout(self.recv_timeout, stream.next()).await { + match timer::timeout(self.recv_timeout, stream.next()).await { Ok(Some(Ok(message))) => { in_flight -= 1; responses.push(message); diff --git a/linera-rpc/src/simple/server.rs b/linera-rpc/src/simple/server.rs index bf75fb291097..3d451e9e5bec 100644 --- a/linera-rpc/src/simple/server.rs +++ b/linera-rpc/src/simple/server.rs @@ -1,10 +1,9 @@ // Copyright (c) Zefchain Labs, Inc. // SPDX-License-Identifier: Apache-2.0 -use std::time::Duration; - use async_trait::async_trait; use futures::{channel::mpsc, stream::StreamExt}; +use linera_base::time::Duration; use linera_core::{ node::NodeError, worker::{NetworkActions, WorkerError, WorkerState}, @@ -107,7 +106,10 @@ where // Send the cross-chain query and retry if needed. for i in 0..cross_chain_max_retries { // Delay increases linearly with the attempt number. - tokio::time::sleep(cross_chain_sender_delay + cross_chain_retry_delay * i).await; + linera_base::time::timer::sleep( + cross_chain_sender_delay + cross_chain_retry_delay * i, + ) + .await; let status = pool.send_message_to(message.clone(), &remote_address).await; match status { diff --git a/linera-rpc/tests/transport.rs b/linera-rpc/tests/transport.rs index e16dcffcaefc..b6418d5b1d3c 100644 --- a/linera-rpc/tests/transport.rs +++ b/linera-rpc/tests/transport.rs @@ -9,8 +9,7 @@ wasm_bindgen_test::wasm_bindgen_test_configure!(run_in_browser); #[ignore] // this test currently must be run manually, as it requires a Linera proxy to be running on 127.0.0.1:9000. async fn client() { - use std::time::Duration; - + use linera_base::time::Duration; use linera_core::node::LocalValidatorNode as _; use linera_rpc::config::*; diff --git a/linera-service/benches/transfers.rs b/linera-service/benches/transfers.rs index 206816044106..f17e23eed832 100644 --- a/linera-service/benches/transfers.rs +++ b/linera-service/benches/transfers.rs @@ -1,8 +1,6 @@ // Copyright (c) Zefchain Labs, Inc. // SPDX-License-Identifier: Apache-2.0 -use std::time::{Duration, Instant}; - use criterion::{criterion_group, criterion_main, Criterion}; use futures::{ stream::{self, FuturesUnordered}, @@ -11,6 +9,7 @@ use futures::{ use linera_base::{ data_types::Amount, identifiers::{Account, ChainId, Owner}, + time::{Duration, Instant}, }; use linera_execution::system::Recipient; use linera_sdk::test::{ActiveChain, TestValidator}; diff --git a/linera-service/src/benchmark.rs b/linera-service/src/benchmark.rs index a58cdcb9141d..5995db5e118b 100644 --- a/linera-service/src/benchmark.rs +++ b/linera-service/src/benchmark.rs @@ -10,6 +10,7 @@ use linera_base::{ async_graphql::InputType, data_types::Amount, identifiers::{Account, AccountOwner, ApplicationId, ChainId, Owner}, + time::timer::Instant, }; use linera_sdk::abis::fungible::{self, FungibleTokenAbi, InitialState, Parameters}; use linera_service::cli_wrappers::{ @@ -19,7 +20,6 @@ use linera_service::cli_wrappers::{ use port_selector::random_free_tcp_port; use rand::{Rng as _, SeedableRng}; use serde_json::Value; -use tokio::time::Instant; use tracing::info; #[derive(clap::Parser)] @@ -235,7 +235,7 @@ async fn benchmark_with_fungible( .await?, ); for i in 0.. { - tokio::time::sleep(Duration::from_secs(i)).await; + linera_base::time::timer::sleep(Duration::from_secs(i)).await; let actual_balance = app.get_amount(&AccountOwner::User(context.owner)).await; if actual_balance == expected_balance { diff --git a/linera-service/src/cli_wrappers/local_net.rs b/linera-service/src/cli_wrappers/local_net.rs index 7cb0d9013b8c..7ba2fd3cd7eb 100644 --- a/linera-service/src/cli_wrappers/local_net.rs +++ b/linera-service/src/cli_wrappers/local_net.rs @@ -563,7 +563,7 @@ impl LocalNet { } Network::Tcp | Network::Udp => { info!("Letting validator proxy {validator} start"); - tokio::time::sleep(Duration::from_secs(2)).await; + linera_base::time::timer::sleep(Duration::from_secs(2)).await; } } Ok(child) @@ -574,9 +574,9 @@ impl LocalNet { .context("endpoint should always parse")? .connect_lazy(); let mut client = HealthClient::new(connection); - tokio::time::sleep(Duration::from_millis(100)).await; + linera_base::time::timer::sleep(Duration::from_millis(100)).await; for i in 0..10 { - tokio::time::sleep(Duration::from_millis(i * 500)).await; + linera_base::time::timer::sleep(Duration::from_millis(i * 500)).await; let result = client.check(HealthCheckRequest::default()).await; if result.is_ok() && result.unwrap().get_ref().status() == ServingStatus::Serving { info!("Successfully started {nickname}"); @@ -630,7 +630,7 @@ impl LocalNet { if i_try == max_try { bail!("Failed to initialize after {} attempts", max_try); } - let one_second = std::time::Duration::from_secs(1); + let one_second = linera_base::time::Duration::from_secs(1); std::thread::sleep(one_second); } self.set_init.insert(key); @@ -661,7 +661,7 @@ impl LocalNet { } Network::Tcp | Network::Udp => { info!("Letting validator server {validator}:{shard} start"); - tokio::time::sleep(Duration::from_secs(2)).await; + linera_base::time::timer::sleep(Duration::from_secs(2)).await; } } Ok(child) diff --git a/linera-service/src/cli_wrappers/wallet.rs b/linera-service/src/cli_wrappers/wallet.rs index ad55f754df1e..423dabe5b325 100644 --- a/linera-service/src/cli_wrappers/wallet.rs +++ b/linera-service/src/cli_wrappers/wallet.rs @@ -390,7 +390,7 @@ impl ClientWrapper { .spawn_into()?; let client = reqwest_client(); for i in 0..10 { - tokio::time::sleep(Duration::from_secs(i)).await; + linera_base::time::timer::sleep(Duration::from_secs(i)).await; let request = client .get(format!("http://localhost:{}/", port)) .send() @@ -445,7 +445,7 @@ impl ClientWrapper { .spawn_into()?; let client = reqwest_client(); for i in 0..10 { - tokio::time::sleep(Duration::from_secs(i)).await; + linera_base::time::timer::sleep(Duration::from_secs(i)).await; let request = client .get(format!("http://localhost:{}/", port)) .send() @@ -951,7 +951,7 @@ impl NodeService { let n_try = 15; let query = query.as_ref(); for i in 0..n_try { - tokio::time::sleep(Duration::from_secs(i)).await; + linera_base::time::timer::sleep(Duration::from_secs(i)).await; let url = format!("http://localhost:{}/", self.port); let client = reqwest_client(); let response = client diff --git a/linera-service/src/grpc_proxy.rs b/linera-service/src/grpc_proxy.rs index b9f1375be30b..12c44957ce25 100644 --- a/linera-service/src/grpc_proxy.rs +++ b/linera-service/src/grpc_proxy.rs @@ -129,7 +129,7 @@ where fn call(&mut self, request: tonic::codegen::http::Request) -> Self::Future { #[cfg(with_metrics)] - let start = std::time::Instant::now(); + let start = linera_base::time::Instant::now(); let future = self.service.call(request); async move { let response = future.await?; diff --git a/linera-service/src/linera/net_up_utils.rs b/linera-service/src/linera/net_up_utils.rs index 12e293848539..5fea22b2fecd 100644 --- a/linera-service/src/linera/net_up_utils.rs +++ b/linera-service/src/linera/net_up_utils.rs @@ -1,10 +1,8 @@ // Copyright (c) Zefchain Labs, Inc. // SPDX-License-Identifier: Apache-2.0 -use std::time::Duration; - use colored::Colorize as _; -use linera_base::data_types::Amount; +use linera_base::{data_types::Amount, time::Duration}; use linera_client::storage::StorageConfig; use linera_execution::ResourceControlPolicy; use linera_service::{ @@ -137,7 +135,7 @@ async fn net_up( .expect("Initialized clients should always have a default chain"); // Make time to (hopefully) display the message after the tracing logs. - tokio::time::sleep(Duration::from_secs(1)).await; + linera_base::time::timer::sleep(Duration::from_secs(1)).await; // Create the wallet for the initial "root" chains. info!("Local test network successfully started."); diff --git a/linera-service/src/node_service.rs b/linera-service/src/node_service.rs index ed3222a354c5..17b4e377417b 100644 --- a/linera-service/src/node_service.rs +++ b/linera-service/src/node_service.rs @@ -1140,7 +1140,7 @@ pub async fn wait_for_next_round(stream: &mut NotificationStream, timeout: Round }); future::select( Box::pin(stream.next()), - Box::pin(tokio::time::sleep( + Box::pin(linera_base::time::timer::sleep( timeout.timestamp.duration_since(Timestamp::now()), )), ) diff --git a/linera-service/src/proxy.rs b/linera-service/src/proxy.rs index a1a274560c3e..d45d1b990140 100644 --- a/linera-service/src/proxy.rs +++ b/linera-service/src/proxy.rs @@ -276,8 +276,8 @@ where recv_timeout: Duration, ) -> Result> { let mut connection = protocol.connect((shard.host, shard.port)).await?; - tokio::time::timeout(send_timeout, connection.send(message)).await??; - let message = tokio::time::timeout(recv_timeout, connection.next()) + linera_base::time::timer::timeout(send_timeout, connection.send(message)).await??; + let message = linera_base::time::timer::timeout(recv_timeout, connection.next()) .await? .transpose()?; Ok(message) diff --git a/linera-service/tests/linera_net_tests.rs b/linera-service/tests/linera_net_tests.rs index 3dd4c4a96c34..05931a04aa8a 100644 --- a/linera-service/tests/linera_net_tests.rs +++ b/linera-service/tests/linera_net_tests.rs @@ -545,8 +545,7 @@ async fn test_wasm_end_to_end_counter_publish_create(config: impl LineraNetConfi #[cfg_attr(feature = "remote-net", test_case(RemoteNetTestingConfig::new(None) ; "remote_net_grpc"))] #[test_log::test(tokio::test)] async fn test_wasm_end_to_end_social_user_pub_sub(config: impl LineraNetConfig) -> Result<()> { - use std::time::Instant; - + use linera_base::time::Instant; use social::SocialAbi; let _guard = INTEGRATION_TEST_GUARD.lock().await; tracing::info!("Starting test {}", test_name!()); @@ -619,7 +618,9 @@ async fn test_wasm_end_to_end_social_user_pub_sub(config: impl LineraNetConfig) }); let deadline = Instant::now() + Duration::from_secs(20); loop { - let result = tokio::time::timeout(deadline - Instant::now(), notifications.next()).await?; + let result = + linera_base::time::timer::timeout(deadline - Instant::now(), notifications.next()) + .await?; anyhow::ensure!(result.transpose()?.is_some(), "Failed to confirm post"); let response = app2.query(query).await?; if response == expected_response { @@ -2482,7 +2483,7 @@ async fn test_open_chain_node_service(config: impl LineraNetConfig) -> Result<() // Verify that the default chain now has 6 and the new one has 4 tokens. for i in 0..10 { - tokio::time::sleep(Duration::from_secs(i)).await; + linera_base::time::timer::sleep(Duration::from_secs(i)).await; let balance1 = app1.get_amount(&owner).await; let balance2 = app2.get_amount(&owner).await; if balance1 == Amount::from_tokens(6) && balance2 == Amount::from_tokens(4) { diff --git a/linera-service/tests/local_net_tests.rs b/linera-service/tests/local_net_tests.rs index 8901da85bbb6..dbc459dc0048 100644 --- a/linera-service/tests/local_net_tests.rs +++ b/linera-service/tests/local_net_tests.rs @@ -233,7 +233,7 @@ async fn test_end_to_end_retry_notification_stream(config: LocalNetConfig) -> Re client1 .transfer(Amount::from_tokens(1), chain, ChainId::root(9)) .await?; - tokio::time::sleep(Duration::from_secs(i)).await; + linera_base::time::timer::sleep(Duration::from_secs(i)).await; height += 1; let response = node_service2 .query_node(format!( diff --git a/linera-storage-service/src/child.rs b/linera-storage-service/src/child.rs index fbb92ef0049b..a9f54d12b565 100644 --- a/linera-storage-service/src/child.rs +++ b/linera-storage-service/src/child.rs @@ -1,10 +1,8 @@ // Copyright (c) Zefchain Labs, Inc. // SPDX-License-Identifier: Apache-2.0 -use std::time::Duration; - use anyhow::{bail, Result}; -use linera_base::command::CommandExt; +use linera_base::{command::CommandExt, time::Duration}; use port_selector::random_free_tcp_port; use tokio::process::{Child, Command}; @@ -16,7 +14,7 @@ pub async fn get_free_port() -> Result { if let Some(port) = port { return Ok(port); } - tokio::time::sleep(Duration::from_secs(i)).await; + linera_base::time::timer::sleep(Duration::from_secs(i)).await; } bail!("Failed to obtain a port"); } @@ -63,7 +61,7 @@ impl StorageService { if storage_service_check_absence(&self.endpoint).await? { return Ok(()); } - tokio::time::sleep(Duration::from_secs(i)).await; + linera_base::time::timer::sleep(Duration::from_secs(i)).await; } bail!("Failed to start child server"); } @@ -80,7 +78,7 @@ impl StorageService { if result.is_ok() { return Ok(guard); } - tokio::time::sleep(Duration::from_secs(i)).await; + linera_base::time::timer::sleep(Duration::from_secs(i)).await; } bail!("Failed to start child server"); } diff --git a/linera-storage/src/db_storage.rs b/linera-storage/src/db_storage.rs index 3eacd3d11586..fc6367858413 100644 --- a/linera-storage/src/db_storage.rs +++ b/linera-storage/src/db_storage.rs @@ -238,7 +238,7 @@ impl Clock for WallClock { } async fn sleep(&self, delta: TimeDelta) { - tokio::time::sleep(delta.as_duration()).await + linera_base::time::timer::sleep(delta.as_duration()).await } async fn sleep_until(&self, timestamp: Timestamp) { diff --git a/linera-views/benches/reentrant_collection_view.rs b/linera-views/benches/reentrant_collection_view.rs index a45c33c691df..3fd9799fbcfb 100644 --- a/linera-views/benches/reentrant_collection_view.rs +++ b/linera-views/benches/reentrant_collection_view.rs @@ -1,9 +1,8 @@ // Copyright (c) Zefchain Labs, Inc. // SPDX-License-Identifier: Apache-2.0 -use std::time::{Duration, Instant}; - use criterion::{black_box, criterion_group, criterion_main, Criterion}; +use linera_base::time::{Duration, Instant}; use linera_views::{ batch::Batch, context::{create_test_memory_context, Context, MemoryContext}, diff --git a/linera-views/src/context.rs b/linera-views/src/context.rs index 5f92e92af9e0..570aeb72133f 100644 --- a/linera-views/src/context.rs +++ b/linera-views/src/context.rs @@ -4,10 +4,10 @@ use std::{ fmt::{Debug, Display}, future::Future, - time::{Duration, Instant}, }; use async_trait::async_trait; +use linera_base::time::{Duration, Instant}; use serde::{de::DeserializeOwned, Serialize}; use crate::{