Skip to content

Commit 5959012

Browse files
authored
Add single-file clickbench benchmark (#2597)
This is the original thing we wanted. It didn't really work on our runners at the time but hopefully as we're rebuilding some of that infra we can go back to running it, and it'll be nice to at least run it locally.
1 parent e1f1ba3 commit 5959012

File tree

2 files changed

+130
-55
lines changed

2 files changed

+130
-55
lines changed

bench-vortex/src/bin/clickbench.rs

+13-45
Original file line numberDiff line numberDiff line change
@@ -1,25 +1,23 @@
1-
use std::fs::{self, File};
2-
use std::path::PathBuf;
1+
use std::fs::{self};
32
use std::time::{Duration, Instant};
43

5-
use bench_vortex::clickbench::{self, HITS_SCHEMA, clickbench_queries};
4+
use bench_vortex::clickbench::{self, Flavor, HITS_SCHEMA, clickbench_queries};
65
use bench_vortex::display::{DisplayFormat, RatioMode, print_measurements_json, render_table};
76
use bench_vortex::measurements::QueryMeasurement;
87
use bench_vortex::metrics::export_plan_spans;
98
use bench_vortex::{
109
Format, IdempotentPath as _, default_env_filter, execute_physical_plan,
11-
feature_flagged_allocator, get_session_with_cache, idempotent, physical_plan,
10+
feature_flagged_allocator, get_session_with_cache, physical_plan,
1211
};
1312
use clap::Parser;
1413
use datafusion_physical_plan::display::DisplayableExecutionPlan;
1514
use indicatif::ProgressBar;
1615
use itertools::Itertools;
17-
use log::{info, warn};
18-
use rayon::iter::{IntoParallelIterator, ParallelIterator as _};
16+
use log::warn;
1917
use tokio::runtime::Builder;
2018
use tracing::info_span;
2119
use tracing_futures::Instrument;
22-
use vortex::error::{VortexExpect, vortex_panic};
20+
use vortex::error::vortex_panic;
2321

2422
feature_flagged_allocator!();
2523

@@ -46,9 +44,11 @@ struct Args {
4644
emulate_object_store: bool,
4745
#[arg(long)]
4846
export_spans: bool,
47+
#[arg(long, value_enum, default_value_t = Flavor::Partitioned)]
48+
flavor: Flavor,
4949
}
5050

51-
fn main() {
51+
fn main() -> anyhow::Result<()> {
5252
let args = Args::parse();
5353

5454
// Capture `RUST_LOG` configuration
@@ -95,45 +95,10 @@ fn main() {
9595
None => Builder::new_multi_thread().enable_all().build(),
9696
}
9797
.expect("Failed building the Runtime");
98-
let basepath = "clickbench".to_data_path();
98+
let basepath = format!("clickbench_{}", args.flavor).to_data_path();
9999
let client = reqwest::blocking::Client::default();
100100

101-
// The clickbench-provided file is missing some higher-level type info, so we reprocess it
102-
// to add that info, see https://github.com/ClickHouse/ClickBench/issues/7.
103-
(0_u32..100).into_par_iter().for_each(|idx| {
104-
let output_path = basepath.join("parquet").join(format!("hits_{idx}.parquet"));
105-
idempotent(&output_path, |output_path| {
106-
info!("Downloading file {idx}");
107-
let url = format!("https://pub-3ba949c0f0354ac18db1f0f14f0a2c52.r2.dev/clickbench/parquet_many/hits_{idx}.parquet");
108-
109-
110-
let make_req = || client.get(&url).send();
111-
let mut output = None;
112-
113-
for attempt in 1..4 {
114-
match make_req() {
115-
Ok(r) => {
116-
output = Some(r.error_for_status());
117-
break;
118-
},
119-
Err(e) => {
120-
warn!("Request for file {idx} timed out, retying for the {attempt} time");
121-
output = Some(Err(e));
122-
}
123-
}
124-
125-
// Very basic backoff mechanism
126-
std::thread::sleep(Duration::from_secs(attempt));
127-
}
128-
129-
let mut response = output.vortex_expect("Must have value here")?;
130-
let mut file = File::create(output_path)?;
131-
response.copy_to(&mut file)?;
132-
133-
anyhow::Ok(PathBuf::from(output_path))
134-
})
135-
.unwrap();
136-
});
101+
args.flavor.download(&client, basepath.as_path())?;
137102

138103
let queries = match args.queries.clone() {
139104
None => clickbench_queries(),
@@ -151,6 +116,7 @@ fn main() {
151116
let session_context = get_session_with_cache(args.emulate_object_store);
152117
let context = session_context.clone();
153118
let mut plans = Vec::new();
119+
154120
match format {
155121
Format::Parquet => runtime.block_on(async {
156122
clickbench::register_parquet_files(
@@ -254,4 +220,6 @@ fn main() {
254220
}
255221
DisplayFormat::GhJson => print_measurements_json(all_measurements).unwrap(),
256222
}
223+
224+
Ok(())
257225
}

bench-vortex/src/clickbench.rs

+117-10
Original file line numberDiff line numberDiff line change
@@ -1,15 +1,21 @@
1-
use std::path::Path;
1+
use std::fs::File;
2+
use std::path::{Path, PathBuf};
23
use std::sync::{Arc, LazyLock};
4+
use std::time::Duration;
35

46
use arrow_schema::{DataType, Field, Schema, TimeUnit};
7+
use clap::ValueEnum;
58
use datafusion::datasource::file_format::parquet::ParquetFormat;
69
use datafusion::datasource::listing::{
710
ListingOptions, ListingTable, ListingTableConfig, ListingTableUrl,
811
};
912
use datafusion::prelude::{ParquetReadOptions, SessionContext};
1013
use futures::{StreamExt, TryStreamExt, stream};
14+
use rayon::iter::{IntoParallelIterator, ParallelIterator};
15+
use reqwest::IntoUrl;
16+
use reqwest::blocking::Response;
1117
use tokio::fs::{OpenOptions, create_dir_all};
12-
use tracing::info;
18+
use tracing::{info, warn};
1319
use vortex::TryIntoArray;
1420
use vortex::dtype::DType;
1521
use vortex::dtype::arrow::FromArrowType;
@@ -19,7 +25,7 @@ use vortex::layout::{LayoutRegistry, LayoutRegistryExt};
1925
use vortex::stream::ArrayStreamAdapter;
2026
use vortex_datafusion::persistent::VortexFormat;
2127

22-
use crate::idempotent_async;
28+
use crate::{idempotent, idempotent_async};
2329

2430
pub static HITS_SCHEMA: LazyLock<Schema> = LazyLock::new(|| {
2531
use DataType::*;
@@ -147,20 +153,36 @@ pub async fn register_vortex_files(
147153
schema: &Schema,
148154
) -> anyhow::Result<()> {
149155
let vortex_dir = input_path.join("vortex");
156+
let parquet_path = input_path.join("parquet");
150157
create_dir_all(&vortex_dir).await?;
151158

152-
stream::iter(0..100)
153-
.map(|idx| {
154-
let parquet_file_path = input_path
155-
.join("parquet")
156-
.join(format!("hits_{idx}.parquet"));
157-
let output_path = vortex_dir.join(format!("hits_{idx}.{VORTEX_FILE_EXTENSION}"));
159+
let parquet_inputs =
160+
std::fs::read_dir(parquet_path.clone())?.collect::<std::io::Result<Vec<_>>>()?;
161+
info!(
162+
"Found {} parquet files in {}",
163+
parquet_inputs.len(),
164+
parquet_path.to_str().unwrap()
165+
);
166+
167+
let iter = parquet_inputs
168+
.iter()
169+
.filter(|entry| entry.path().extension().is_some_and(|e| e == "parquet"));
170+
171+
stream::iter(iter)
172+
.map(|dir_entry| {
173+
let filename = {
174+
let mut temp = dir_entry.path();
175+
temp.set_extension("");
176+
temp.file_name().unwrap().to_str().unwrap().to_string()
177+
};
178+
let parquet_file_path = parquet_path.join(format!("{filename}.parquet"));
179+
let output_path = vortex_dir.join(format!("{filename}.{VORTEX_FILE_EXTENSION}"));
158180
let session = session.clone();
159181

160182
tokio::spawn(async move {
161183
let output_path = output_path.clone();
162184
idempotent_async(&output_path, move |vtx_file| async move {
163-
info!("Processing file {idx}");
185+
info!("Processing file '{filename}'");
164186
let record_batches = session
165187
.read_parquet(
166188
parquet_file_path.to_str().unwrap(),
@@ -259,3 +281,88 @@ pub fn clickbench_queries() -> Vec<(usize, String)> {
259281
.enumerate()
260282
.collect()
261283
}
284+
285+
#[derive(ValueEnum, Default, Clone, Copy, Debug, Hash, PartialEq, Eq)]
286+
pub enum Flavor {
287+
#[default]
288+
Partitioned,
289+
Single,
290+
}
291+
292+
impl std::fmt::Display for Flavor {
293+
fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
294+
write!(f, "{}", self.to_possible_value().unwrap().get_name())
295+
}
296+
}
297+
298+
impl Flavor {
299+
pub fn download(
300+
&self,
301+
client: &reqwest::blocking::Client,
302+
basepath: impl AsRef<Path>,
303+
) -> anyhow::Result<()> {
304+
let basepath = basepath.as_ref();
305+
match self {
306+
Flavor::Single => {
307+
let output_path = basepath.join("parquet").join("hits.parquet");
308+
idempotent(&output_path, |output_path| {
309+
info!("Downloading single clickbench file");
310+
let url = "https://pub-3ba949c0f0354ac18db1f0f14f0a2c52.r2.dev/clickbench/parquet_single/hits.parquet";
311+
let mut response = retry_get(client, url)?;
312+
let mut file = File::create(output_path)?;
313+
response.copy_to(&mut file)?;
314+
315+
anyhow::Ok(())
316+
})?;
317+
318+
Ok(())
319+
}
320+
Flavor::Partitioned => {
321+
// The clickbench-provided file is missing some higher-level type info, so we reprocess it
322+
// to add that info, see https://github.com/ClickHouse/ClickBench/issues/7.
323+
(0_u32..100).into_par_iter().for_each(|idx| {
324+
let output_path = basepath.join("parquet").join(format!("hits_{idx}.parquet"));
325+
idempotent(&output_path, |output_path| {
326+
info!("Downloading file {idx}");
327+
let url = format!("https://pub-3ba949c0f0354ac18db1f0f14f0a2c52.r2.dev/clickbench/parquet_many/hits_{idx}.parquet");
328+
let mut response = retry_get(client, url)?;
329+
let mut file = File::create(output_path)?;
330+
response.copy_to(&mut file)?;
331+
332+
anyhow::Ok(PathBuf::from(output_path))
333+
})
334+
.unwrap();
335+
});
336+
337+
Ok(())
338+
}
339+
}
340+
}
341+
}
342+
343+
fn retry_get(client: &reqwest::blocking::Client, url: impl IntoUrl) -> anyhow::Result<Response> {
344+
let url = url.as_str();
345+
let make_req = || client.get(url).send();
346+
347+
let mut output = None;
348+
349+
for attempt in 1..4 {
350+
match make_req().and_then(|r| r.error_for_status()) {
351+
Ok(r) => {
352+
output = Some(r);
353+
break;
354+
}
355+
Err(e) => {
356+
warn!("Request errored with {e}, retying for the {attempt} time");
357+
}
358+
}
359+
360+
// Very basic backoff mechanism
361+
std::thread::sleep(Duration::from_secs(attempt));
362+
}
363+
364+
match output {
365+
Some(v) => Ok(v),
366+
None => anyhow::bail!("Exahusted retry attempts for {url}"),
367+
}
368+
}

0 commit comments

Comments
 (0)