Skip to content

Commit 88bc8c1

Browse files
authoredMay 14, 2024
Merge pull request #992 from Lorak-mmk/remove_with_current_subscriber
Remove `.with_current_subscriber()` calls
2 parents c25c54d + 370b00a commit 88bc8c1

File tree

13 files changed

+132
-34
lines changed

13 files changed

+132
-34
lines changed
 

‎Cargo.lock.msrv

+31
Some generated files are not rendered by default. Learn more about customizing how changed files appear on GitHub.

‎docs/source/logging/logging.md

+49-3
Original file line numberDiff line numberDiff line change
@@ -1,9 +1,15 @@
11
# Logging
22

33
The driver uses the [tracing](https://github.com/tokio-rs/tracing) crate for all logs.\
4-
To view the logs you have to create a `tracing` subscriber to which all logs will be written.
4+
There are two ways to view the logs:
5+
- Create a `tracing` subscriber to which all logs will be written (recommended).
6+
- Enable `log` feature on `tracing` crate and use some logger from `log` ecosystem. \
7+
Only do this if you can't use `tracing` subscriber for some reason.
8+
9+
## Using tracing subscriber
10+
11+
To print the logs you can use the default subscriber:
512

6-
To just print the logs you can use the default subscriber:
713
```rust
814
# extern crate scylla;
915
# extern crate tokio;
@@ -45,4 +51,44 @@ To start this example execute:
4551
RUST_LOG=info cargo run
4652
```
4753

48-
The full [example](https://github.com/scylladb/scylla-rust-driver/tree/main/examples/logging.rs) is available in the `examples` folder
54+
The full [example](https://github.com/scylladb/scylla-rust-driver/tree/main/examples/logging.rs) is available in the `examples` folder.
55+
You can run it from main folder of driver repository using `RUST_LOG=trace SCYLLA_URI=<scylla_ip>:9042 cargo run --example logging`.
56+
57+
## Using log
58+
59+
To collect tracing events using log collector you first need to enable `log` feature on `tracing` crate.
60+
You can use `cargo add tracing -F log` or edit `Cargo.toml`:
61+
```toml
62+
tracing = { version = "0.1.40" , features = ["log"] }
63+
```
64+
then you can setup `env_logger` os some other logger and it will output logs from the driver:
65+
66+
```rust
67+
# extern crate scylla;
68+
# extern crate tokio;
69+
# extern crate tracing;
70+
# extern crate env_logger;
71+
# use std::error::Error;
72+
# use scylla::{Session, SessionBuilder};
73+
use tracing::info;
74+
75+
#[tokio::main]
76+
async fn main() -> Result<(), Box<dyn Error>> {
77+
// Setup `log` collector that uses RUST_LOG env variable to configure
78+
// verbosity.
79+
env_logger::init();
80+
81+
let uri = std::env::var("SCYLLA_URI").unwrap_or_else(|_| "127.0.0.1:9042".to_string());
82+
info!("Connecting to {}", uri);
83+
84+
let session: Session = SessionBuilder::new().known_node(uri).build().await?;
85+
session.query("CREATE KEYSPACE IF NOT EXISTS examples_ks WITH REPLICATION = {'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1}", &[]).await?;
86+
87+
session.query("USE examples_ks", &[]).await?;
88+
89+
Ok(())
90+
}
91+
```
92+
93+
The full [example](https://github.com/scylladb/scylla-rust-driver/tree/main/examples/logging_log.rs) is available in the `examples` folder.
94+
You can run it from main folder of driver repository using `RUST_LOG=trace SCYLLA_URI=<scylla_ip>:9042 cargo run --example logging_log`.

‎examples/Cargo.toml

+6-1
Original file line numberDiff line numberDiff line change
@@ -12,7 +12,7 @@ rustyline = "9"
1212
rustyline-derive = "0.6"
1313
scylla = {path = "../scylla", features = ["ssl", "cloud", "chrono", "time", "num-bigint-03", "num-bigint-04", "bigdecimal-04"]}
1414
tokio = {version = "1.1.0", features = ["full"]}
15-
tracing = "0.1.25"
15+
tracing = { version = "0.1.25" , features = ["log"] }
1616
tracing-subscriber = { version = "0.3.14", features = ["env-filter"] }
1717
chrono = { version = "0.4", default-features = false }
1818
time = { version = "0.3.22" }
@@ -21,6 +21,7 @@ tower = "0.4"
2121
stats_alloc = "0.1"
2222
clap = { version = "3.2.4", features = ["derive"] }
2323
rand = "0.8.5"
24+
env_logger = "0.10"
2425

2526
[[example]]
2627
name = "auth"
@@ -34,6 +35,10 @@ path = "basic.rs"
3435
name = "logging"
3536
path = "logging.rs"
3637

38+
[[example]]
39+
name = "logging_log"
40+
path = "logging_log.rs"
41+
3742
[[example]]
3843
name = "tls"
3944
path = "tls.rs"

‎examples/logging_log.rs

+27
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,27 @@
1+
use anyhow::Result;
2+
use scylla::transport::session::Session;
3+
use scylla::SessionBuilder;
4+
use std::env;
5+
use tracing::info;
6+
7+
// To run this example, and view logged messages, RUST_LOG env var needs to be set
8+
// This can be done using shell command presented below
9+
// RUST_LOG=info cargo run --example logging_log
10+
#[tokio::main]
11+
async fn main() -> Result<()> {
12+
// Driver uses `tracing` for logging purposes, but it's possible to use `log`
13+
// ecosystem to view the messages. This requires adding `tracing` crate to
14+
// dependencies and enabling its "log" feature. Then you will be able to use
15+
// loggers like `env_logger` to see driver's messages.
16+
env_logger::init();
17+
18+
let uri = env::var("SCYLLA_URI").unwrap_or_else(|_| "127.0.0.1:9042".to_string());
19+
info!("Connecting to {}", uri);
20+
21+
let session: Session = SessionBuilder::new().known_node(uri).build().await?;
22+
session.query("CREATE KEYSPACE IF NOT EXISTS examples_ks WITH REPLICATION = {'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1}", &[]).await?;
23+
24+
session.query("USE examples_ks", &[]).await?;
25+
26+
Ok(())
27+
}

‎scylla-proxy/examples/cmdline.rs

+1-2
Original file line numberDiff line numberDiff line change
@@ -13,7 +13,6 @@ use std::{
1313
};
1414

1515
use scylla_proxy::{Node, Proxy, ShardAwareness};
16-
use tracing::instrument::WithSubscriber;
1716

1817
fn init_logger() {
1918
tracing_subscriber::fmt::fmt()
@@ -53,7 +52,7 @@ async fn main() {
5352
None,
5453
None,
5554
)]);
56-
let running_proxy = proxy.run().with_current_subscriber().await.unwrap();
55+
let running_proxy = proxy.run().await.unwrap();
5756

5857
pause().await;
5958
running_proxy.finish().await.unwrap();

‎scylla-proxy/examples/identity_proxy.rs

+1-2
Original file line numberDiff line numberDiff line change
@@ -1,7 +1,6 @@
11
use std::{net::SocketAddr, str::FromStr};
22

33
use scylla_proxy::{Node, Proxy, ShardAwareness};
4-
use tracing::instrument::WithSubscriber;
54

65
fn init_logger() {
76
tracing_subscriber::fmt::fmt()
@@ -30,7 +29,7 @@ async fn main() {
3029
.build(),
3130
)
3231
.build();
33-
let running_proxy = proxy.run().with_current_subscriber().await.unwrap();
32+
let running_proxy = proxy.run().await.unwrap();
3433

3534
pause().await;
3635
running_proxy.finish().await.unwrap();

‎scylla-proxy/examples/identity_shard_aware_proxy.rs

+1-2
Original file line numberDiff line numberDiff line change
@@ -1,7 +1,6 @@
11
use std::{net::SocketAddr, str::FromStr};
22

33
use scylla_proxy::{Node, Proxy, ShardAwareness};
4-
use tracing::instrument::WithSubscriber;
54

65
fn init_logger() {
76
tracing_subscriber::fmt::fmt()
@@ -27,7 +26,7 @@ async fn main() {
2726
None,
2827
None,
2928
)]);
30-
let running_proxy = proxy.run().with_current_subscriber().await.unwrap();
29+
let running_proxy = proxy.run().await.unwrap();
3130

3231
pause().await;
3332
running_proxy.finish().await.unwrap();

‎scylla/src/transport/cluster.rs

+2-3
Original file line numberDiff line numberDiff line change
@@ -24,7 +24,6 @@ use std::collections::{HashMap, HashSet};
2424
use std::net::SocketAddr;
2525
use std::sync::Arc;
2626
use std::time::Duration;
27-
use tracing::instrument::WithSubscriber;
2827
use tracing::{debug, warn};
2928
use uuid::Uuid;
3029

@@ -206,7 +205,7 @@ impl Cluster {
206205
};
207206

208207
let (fut, worker_handle) = worker.work().remote_handle();
209-
tokio::spawn(fut.with_current_subscriber());
208+
tokio::spawn(fut);
210209

211210
let result = Cluster {
212211
data: cluster_data,
@@ -647,7 +646,7 @@ impl ClusterWorker {
647646

648647
let cluster_data = self.cluster_data.load_full();
649648
let use_keyspace_future = Self::handle_use_keyspace_request(cluster_data, request);
650-
tokio::spawn(use_keyspace_future.with_current_subscriber());
649+
tokio::spawn(use_keyspace_future);
651650
},
652651
None => return, // If use_keyspace_channel was closed then cluster was dropped, we can stop working
653652
}

‎scylla/src/transport/connection.rs

+2-3
Original file line numberDiff line numberDiff line change
@@ -13,7 +13,6 @@ use tokio::io::{split, AsyncRead, AsyncWrite, AsyncWriteExt, BufReader, BufWrite
1313
use tokio::net::{TcpSocket, TcpStream};
1414
use tokio::sync::{mpsc, oneshot};
1515
use tokio::time::Instant;
16-
use tracing::instrument::WithSubscriber;
1716
use tracing::{debug, error, trace, warn};
1817
use uuid::Uuid;
1918

@@ -1090,7 +1089,7 @@ impl Connection {
10901089
node_address,
10911090
)
10921091
.remote_handle();
1093-
tokio::task::spawn(task.with_current_subscriber());
1092+
tokio::task::spawn(task);
10941093
return Ok(handle);
10951094
}
10961095

@@ -1104,7 +1103,7 @@ impl Connection {
11041103
node_address,
11051104
)
11061105
.remote_handle();
1107-
tokio::task::spawn(task.with_current_subscriber());
1106+
tokio::task::spawn(task);
11081107
Ok(handle)
11091108
}
11101109

‎scylla/src/transport/connection_pool.rs

+8-12
Original file line numberDiff line numberDiff line change
@@ -27,7 +27,6 @@ use std::sync::{Arc, RwLock, Weak};
2727
use std::time::Duration;
2828

2929
use tokio::sync::{broadcast, mpsc, Notify};
30-
use tracing::instrument::WithSubscriber;
3130
use tracing::{debug, error, trace, warn};
3231

3332
/// The target size of a per-node connection pool.
@@ -212,7 +211,7 @@ impl NodeConnectionPool {
212211

213212
let conns = refiller.get_shared_connections();
214213
let (fut, refiller_handle) = refiller.run(use_keyspace_request_receiver).remote_handle();
215-
tokio::spawn(fut.with_current_subscriber());
214+
tokio::spawn(fut);
216215

217216
Self {
218217
conns,
@@ -1138,17 +1137,14 @@ impl PoolRefiller {
11381137
Err(QueryError::IoError(io_error.unwrap()))
11391138
};
11401139

1141-
tokio::task::spawn(
1142-
async move {
1143-
let res = fut.await;
1144-
match &res {
1145-
Ok(()) => debug!("[{}] Successfully changed current keyspace", address),
1146-
Err(err) => warn!("[{}] Failed to change keyspace: {:?}", address, err),
1147-
}
1148-
let _ = response_sender.send(res);
1140+
tokio::task::spawn(async move {
1141+
let res = fut.await;
1142+
match &res {
1143+
Ok(()) => debug!("[{}] Successfully changed current keyspace", address),
1144+
Err(err) => warn!("[{}] Failed to change keyspace: {:?}", address, err),
11491145
}
1150-
.with_current_subscriber(),
1151-
);
1146+
let _ = response_sender.send(res);
1147+
});
11521148
}
11531149

11541150
// Requires the keyspace to be set

‎scylla/src/transport/iterator.rs

+1-2
Original file line numberDiff line numberDiff line change
@@ -16,7 +16,6 @@ use scylla_cql::types::serialize::row::SerializedValues;
1616
use std::result::Result;
1717
use thiserror::Error;
1818
use tokio::sync::mpsc;
19-
use tracing::instrument::WithSubscriber;
2019

2120
use super::errors::QueryError;
2221
use super::execution_profile::ExecutionProfileInner;
@@ -387,7 +386,7 @@ impl RowIterator {
387386
worker_task: impl Future<Output = PageSendAttemptedProof> + Send + 'static,
388387
mut receiver: mpsc::Receiver<Result<ReceivedPage, QueryError>>,
389388
) -> Result<RowIterator, QueryError> {
390-
tokio::task::spawn(worker_task.with_current_subscriber());
389+
tokio::task::spawn(worker_task);
391390

392391
// This unwrap is safe because:
393392
// - The future returned by worker.work sends at least one item

‎scylla/src/transport/load_balancing/default.rs

+2-2
Original file line numberDiff line numberDiff line change
@@ -2269,7 +2269,7 @@ mod latency_awareness {
22692269
use itertools::Either;
22702270
use scylla_cql::errors::{DbError, QueryError};
22712271
use tokio::time::{Duration, Instant};
2272-
use tracing::{instrument::WithSubscriber, trace, warn};
2272+
use tracing::{trace, warn};
22732273
use uuid::Uuid;
22742274

22752275
use crate::{load_balancing::NodeRef, routing::Shard, transport::node::Node};
@@ -2454,7 +2454,7 @@ mod latency_awareness {
24542454
}
24552455
}
24562456
.remote_handle();
2457-
tokio::task::spawn(updater_fut.with_current_subscriber());
2457+
tokio::task::spawn(updater_fut);
24582458

24592459
Self {
24602460
_updater_handle: Some(updater_handle),

‎scylla/tests/integration/utils.rs

+1-2
Original file line numberDiff line numberDiff line change
@@ -3,7 +3,6 @@ use std::collections::HashMap;
33
use std::env;
44
use std::net::SocketAddr;
55
use std::str::FromStr;
6-
use tracing::instrument::WithSubscriber;
76

87
use scylla_proxy::{Node, Proxy, ProxyError, RunningProxy, ShardAwareness};
98

@@ -53,7 +52,7 @@ where
5352
);
5453

5554
let translation_map = proxy.translation_map();
56-
let running_proxy = proxy.run().with_current_subscriber().await.unwrap();
55+
let running_proxy = proxy.run().await.unwrap();
5756

5857
let running_proxy = test(
5958
[proxy1_uri, proxy2_uri, proxy3_uri],

0 commit comments

Comments
 (0)