Skip to content

Commit

Permalink
perf(query): improve SELECTs performance (#169)
Browse files Browse the repository at this point in the history
* Improve performance of `RowCursor::next()`.
* Add `RowCursor::{decoded_bytes,received_bytes}`  methods.
  • Loading branch information
loyd authored Oct 20, 2024
1 parent b45ff06 commit b4cbb39
Show file tree
Hide file tree
Showing 22 changed files with 569 additions and 490 deletions.
9 changes: 8 additions & 1 deletion CHANGELOG.md
Original file line number Diff line number Diff line change
@@ -1,16 +1,23 @@
# Changelog
All notable changes to this project will be documented in this file.

The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/),
The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.1.0/),
and this project adheres to [Semantic Versioning](https://semver.org/spec/v2.0.0.html).

<!-- next-header -->

## [Unreleased] - ReleaseDate
### Added
- query/cursor: add `RowCursor::{decoded_bytes,received_bytes}` methods ([#169]).

### Changed
- query/cursor: improve performance of `RowCursor::next()` ([#169]).

### Fixed
- mock: work with the advanced time via `tokio::time::advance()` ([#165]).

[#165]: https://github.com/ClickHouse/clickhouse-rs/pull/165
[#169]: https://github.com/ClickHouse/clickhouse-rs/pull/169

## [0.13.0] - 2024-09-27
### Added
Expand Down
2 changes: 1 addition & 1 deletion Cargo.toml
Original file line number Diff line number Diff line change
Expand Up @@ -93,7 +93,7 @@ url = "2.1.1"
futures = "0.3.5"
futures-channel = "0.3.30"
static_assertions = "1.1"
sealed = "0.5"
sealed = "0.6"
sha-1 = { version = "0.10", optional = true }
serde_json = { version = "1.0.68", optional = true }
lz4_flex = { version = "0.11.3", default-features = false, features = [
Expand Down
42 changes: 42 additions & 0 deletions benches/README.md
Original file line number Diff line number Diff line change
@@ -0,0 +1,42 @@
# Benchmarks

All cases are run with `cargo bench --bench <case>`.

## With a mocked server

These benchmarks are run against a mocked server, which is a simple HTTP server that responds with a fixed response. This is useful to measure the overhead of the client itself:
* `select` checks throughput of `Client::query()`.
* `insert` checks throughput of `Client::insert()` and `Client::inserter()` (if the `inserter` features is enabled).

### How to collect perf data

The crate's code runs on the thread with the name `testee`:
```bash
cargo bench --bench <name> &
perf record -p `ps -AT | grep testee | awk '{print $2}'` --call-graph dwarf,65528 --freq 5000 -g -- sleep 5
perf script > perf.script
```

Then upload the `perf.script` file to [Firefox Profiler](https://profiler.firefox.com).

## With a running ClickHouse server

These benchmarks are run against a real ClickHouse server, so it must be started:
```bash
docker run -d -p 8123:8123 -p 9000:9000 --name ch clickhouse/clickhouse-server

cargo bench --bench <case>
```

Cases:
* `select_numbers` measures time of running a big SELECT query to the `system.numbers_mt` table.

### How to collect perf data

```bash
cargo bench --bench <name> &
perf record -p `ps -AT | grep <name> | awk '{print $2}'` --call-graph dwarf,65528 --freq 5000 -g -- sleep 5
perf script > perf.script
```

Then upload the `perf.script` file to [Firefox Profiler](https://profiler.firefox.com).
86 changes: 76 additions & 10 deletions benches/common.rs
Original file line number Diff line number Diff line change
@@ -1,6 +1,14 @@
#![allow(dead_code)] // typical for common test/bench modules :(

use std::{convert::Infallible, future::Future, net::SocketAddr, thread};
use std::{
convert::Infallible,
future::Future,
net::SocketAddr,
pin::Pin,
sync::atomic::{AtomicU32, Ordering},
thread,
time::Duration,
};

use bytes::Bytes;
use futures::stream::StreamExt;
Expand All @@ -11,7 +19,13 @@ use hyper::{
service, Request, Response,
};
use hyper_util::rt::{TokioIo, TokioTimer};
use tokio::{net::TcpListener, runtime};
use tokio::{
net::TcpListener,
runtime,
sync::{mpsc, oneshot},
};

use clickhouse::error::Result;

pub(crate) struct ServerHandle;

Expand All @@ -38,14 +52,7 @@ where
}
};

thread::spawn(move || {
runtime::Builder::new_current_thread()
.enable_all()
.build()
.unwrap()
.block_on(serving);
});

run_on_st_runtime("server", serving);
ServerHandle
}

Expand All @@ -57,3 +64,62 @@ pub(crate) async fn skip_incoming(request: Request<Incoming>) {
result.unwrap();
}
}

pub(crate) struct RunnerHandle {
tx: mpsc::UnboundedSender<Run>,
}

struct Run {
future: Pin<Box<dyn Future<Output = Result<Duration>> + Send>>,
callback: oneshot::Sender<Result<Duration>>,
}

impl RunnerHandle {
pub(crate) fn run(
&self,
f: impl Future<Output = Result<Duration>> + Send + 'static,
) -> Duration {
let (tx, rx) = oneshot::channel();

self.tx
.send(Run {
future: Box::pin(f),
callback: tx,
})
.unwrap();

rx.blocking_recv().unwrap().unwrap()
}
}

pub(crate) fn start_runner() -> RunnerHandle {
let (tx, mut rx) = mpsc::unbounded_channel::<Run>();

run_on_st_runtime("testee", async move {
while let Some(run) = rx.recv().await {
let result = run.future.await;
let _ = run.callback.send(result);
}
});

RunnerHandle { tx }
}

fn run_on_st_runtime(name: &str, f: impl Future + Send + 'static) {
let name = name.to_string();
thread::Builder::new()
.name(name.clone())
.spawn(move || {
let no = AtomicU32::new(0);
runtime::Builder::new_current_thread()
.enable_all()
.thread_name_fn(move || {
let no = no.fetch_add(1, Ordering::Relaxed);
format!("{name}-{no}")
})
.build()
.unwrap()
.block_on(f);
})
.unwrap();
}
16 changes: 9 additions & 7 deletions benches/insert.rs
Original file line number Diff line number Diff line change
@@ -1,11 +1,14 @@
use std::{future::Future, mem, time::Duration};
use std::{
future::Future,
mem,
time::{Duration, Instant},
};

use bytes::Bytes;
use criterion::{black_box, criterion_group, criterion_main, Criterion, Throughput};
use http_body_util::Empty;
use hyper::{body::Incoming, Request, Response};
use serde::Serialize;
use tokio::{runtime::Runtime, time::Instant};

use clickhouse::{error::Result, Client, Compression, Row};

Expand Down Expand Up @@ -76,30 +79,29 @@ async fn run_inserter<const WITH_PERIOD: bool>(client: Client, iters: u64) -> Re

fn run<F>(c: &mut Criterion, name: &str, port: u16, f: impl Fn(Client, u64) -> F)
where
F: Future<Output = Result<Duration>>,
F: Future<Output = Result<Duration>> + Send + 'static,
{
let addr = format!("127.0.0.1:{port}").parse().unwrap();
let _server = common::start_server(addr, serve);
let runner = common::start_runner();

let mut group = c.benchmark_group(name);
group.throughput(Throughput::Bytes(mem::size_of::<SomeRow>() as u64));
group.bench_function("no compression", |b| {
b.iter_custom(|iters| {
let rt = Runtime::new().unwrap();
let client = Client::default()
.with_url(format!("http://{addr}"))
.with_compression(Compression::None);
rt.block_on((f)(client, iters)).unwrap()
runner.run((f)(client, iters))
})
});
#[cfg(feature = "lz4")]
group.bench_function("lz4", |b| {
b.iter_custom(|iters| {
let rt = Runtime::new().unwrap();
let client = Client::default()
.with_url(format!("http://{addr}"))
.with_compression(Compression::Lz4);
rt.block_on((f)(client, iters)).unwrap()
runner.run((f)(client, iters))
})
});
group.finish();
Expand Down
23 changes: 11 additions & 12 deletions benches/select.rs
Original file line number Diff line number Diff line change
@@ -1,4 +1,8 @@
use std::{convert::Infallible, mem};
use std::{
convert::Infallible,
mem,
time::{Duration, Instant},
};

use bytes::Bytes;
use criterion::{black_box, criterion_group, criterion_main, Criterion, Throughput};
Expand All @@ -9,7 +13,6 @@ use hyper::{
Request, Response,
};
use serde::Deserialize;
use tokio::{runtime::Runtime, time::Instant};

use clickhouse::{error::Result, Client, Compression, Row};

Expand Down Expand Up @@ -47,6 +50,7 @@ fn select(c: &mut Criterion) {
let addr = "127.0.0.1:6543".parse().unwrap();
let chunk = prepare_chunk();
let _server = common::start_server(addr, move |req| serve(req, chunk.clone()));
let runner = common::start_runner();

#[allow(dead_code)]
#[derive(Debug, Row, Deserialize)]
Expand All @@ -57,7 +61,8 @@ fn select(c: &mut Criterion) {
d: u32,
}

async fn run(client: Client, iters: u64) -> Result<()> {
async fn run(client: Client, iters: u64) -> Result<Duration> {
let start = Instant::now();
let mut cursor = client
.query("SELECT ?fields FROM some")
.fetch::<SomeRow>()?;
Expand All @@ -66,32 +71,26 @@ fn select(c: &mut Criterion) {
black_box(cursor.next().await?);
}

Ok(())
Ok(start.elapsed())
}

let mut group = c.benchmark_group("select");
group.throughput(Throughput::Bytes(mem::size_of::<SomeRow>() as u64));
group.bench_function("no compression", |b| {
b.iter_custom(|iters| {
let rt = Runtime::new().unwrap();
let client = Client::default()
.with_url(format!("http://{addr}"))
.with_compression(Compression::None);
let start = Instant::now();
rt.block_on(run(client, iters)).unwrap();
start.elapsed()
runner.run(run(client, iters))
})
});
#[cfg(feature = "lz4")]
group.bench_function("lz4", |b| {
b.iter_custom(|iters| {
let rt = Runtime::new().unwrap();
let client = Client::default()
.with_url(format!("http://{addr}"))
.with_compression(Compression::Lz4);
let start = Instant::now();
rt.block_on(run(client, iters)).unwrap();
start.elapsed()
runner.run(run(client, iters))
})
});
group.finish();
Expand Down
33 changes: 22 additions & 11 deletions benches/select_numbers.rs
Original file line number Diff line number Diff line change
@@ -1,15 +1,24 @@
use serde::Deserialize;

use clickhouse::{error::Result, Client, Compression, Row};
use clickhouse::{Client, Compression, Row};

#[derive(Row, Deserialize)]
struct Data {
no: u64,
}

async fn bench() -> u64 {
async fn bench(name: &str, compression: Compression) {
let start = std::time::Instant::now();
let (sum, dec_mbytes, rec_mbytes) = tokio::spawn(do_bench(compression)).await.unwrap();
assert_eq!(sum, 124999999750000000);
let elapsed = start.elapsed();
let throughput = dec_mbytes / elapsed.as_secs_f64();
println!("{name:>8} {elapsed:>7.3?} {throughput:>4.0} MiB/s {rec_mbytes:>4.0} MiB");
}

async fn do_bench(compression: Compression) -> (u64, f64, f64) {
let client = Client::default()
.with_compression(Compression::None)
.with_compression(compression)
.with_url("http://localhost:8123");

let mut cursor = client
Expand All @@ -22,15 +31,17 @@ async fn bench() -> u64 {
sum += row.no;
}

sum
let dec_bytes = cursor.decoded_bytes();
let dec_mbytes = dec_bytes as f64 / 1024.0 / 1024.0;
let recv_bytes = cursor.received_bytes();
let recv_mbytes = recv_bytes as f64 / 1024.0 / 1024.0;
(sum, dec_mbytes, recv_mbytes)
}

#[tokio::main]
async fn main() -> Result<()> {
println!("Started");
let start = std::time::Instant::now();
let sum = tokio::spawn(bench()).await.unwrap();
let elapsed = start.elapsed();
println!("Done: elapsed={elapsed:?} sum={sum}");
Ok(())
async fn main() {
println!("compress elapsed throughput received");
bench("none", Compression::None).await;
#[cfg(feature = "lz4")]
bench("lz4", Compression::Lz4).await;
}
Loading

0 comments on commit b4cbb39

Please sign in to comment.