Line data Source code
1 : use std::{sync::Arc, time::SystemTime};
2 :
3 : use anyhow::Context;
4 : use bytes::{buf::Writer, BufMut, BytesMut};
5 : use chrono::{Datelike, Timelike};
6 : use futures::{Stream, StreamExt};
7 : use parquet::{
8 : basic::Compression,
9 : file::{
10 : metadata::RowGroupMetaDataPtr,
11 : properties::{WriterProperties, WriterPropertiesPtr, DEFAULT_PAGE_SIZE},
12 : writer::SerializedFileWriter,
13 : },
14 : record::RecordWriter,
15 : };
16 : use remote_storage::{GenericRemoteStorage, RemotePath, TimeoutOrCancel};
17 : use tokio::{sync::mpsc, time};
18 : use tokio_util::sync::CancellationToken;
19 : use tracing::{debug, info, Span};
20 : use utils::backoff;
21 :
22 : use crate::{
23 : config::{remote_storage_from_toml, OptRemoteStorageConfig},
24 : context::LOG_CHAN_DISCONNECT,
25 : };
26 :
27 : use super::{RequestMonitoring, LOG_CHAN};
28 :
29 12 : #[derive(clap::Args, Clone, Debug)]
30 : pub struct ParquetUploadArgs {
31 : /// Storage location to upload the parquet files to.
32 : /// Encoded as toml (same format as pageservers), eg
33 : /// `{bucket_name='the-bucket',bucket_region='us-east-1',prefix_in_bucket='proxy',endpoint='http://minio:9000'}`
34 : #[clap(long, default_value = "{}", value_parser = remote_storage_from_toml)]
35 0 : parquet_upload_remote_storage: OptRemoteStorageConfig,
36 :
37 : #[clap(long, default_value = "{}", value_parser = remote_storage_from_toml)]
38 0 : parquet_upload_disconnect_events_remote_storage: OptRemoteStorageConfig,
39 :
40 : /// How many rows to include in a row group
41 6 : #[clap(long, default_value_t = 8192)]
42 0 : parquet_upload_row_group_size: usize,
43 :
44 : /// How large each column page should be in bytes
45 6 : #[clap(long, default_value_t = DEFAULT_PAGE_SIZE)]
46 0 : parquet_upload_page_size: usize,
47 :
48 : /// How large the total parquet file should be in bytes
49 6 : #[clap(long, default_value_t = 100_000_000)]
50 0 : parquet_upload_size: i64,
51 :
52 : /// How long to wait before forcing a file upload
53 : #[clap(long, default_value = "20m", value_parser = humantime::parse_duration)]
54 0 : parquet_upload_maximum_duration: tokio::time::Duration,
55 :
56 : /// What level of compression to use
57 6 : #[clap(long, default_value_t = Compression::UNCOMPRESSED)]
58 0 : parquet_upload_compression: Compression,
59 : }
60 :
61 : // Occasional network issues and such can cause remote operations to fail, and
62 : // that's expected. If a upload fails, we log it at info-level, and retry.
63 : // But after FAILED_UPLOAD_WARN_THRESHOLD retries, we start to log it at WARN
64 : // level instead, as repeated failures can mean a more serious problem. If it
65 : // fails more than FAILED_UPLOAD_RETRIES times, we give up
66 : pub const FAILED_UPLOAD_WARN_THRESHOLD: u32 = 3;
67 : pub const FAILED_UPLOAD_MAX_RETRIES: u32 = 10;
68 :
69 : // the parquet crate leaves a lot to be desired...
70 : // what follows is an attempt to write parquet files with minimal allocs.
71 : // complication: parquet is a columnar format, while we want to write in as rows.
72 : // design:
73 : // * we batch up to 1024 rows, then flush them into a 'row group'
74 : // * after each rowgroup write, we check the length of the file and upload to s3 if large enough
75 :
76 7524222 : #[derive(parquet_derive::ParquetRecordWriter)]
77 : pub struct RequestData {
78 : region: &'static str,
79 : protocol: &'static str,
80 : /// Must be UTC. The derive macro doesn't like the timezones
81 : timestamp: chrono::NaiveDateTime,
82 : session_id: uuid::Uuid,
83 : peer_addr: String,
84 : username: Option<String>,
85 : application_name: Option<String>,
86 : endpoint_id: Option<String>,
87 : database: Option<String>,
88 : project: Option<String>,
89 : branch: Option<String>,
90 : auth_method: Option<&'static str>,
91 : error: Option<&'static str>,
92 : /// Success is counted if we form a HTTP response with sql rows inside
93 : /// Or if we make it to proxy_pass
94 : success: bool,
95 : /// Indicates if the cplane started the new compute node for this request.
96 : cold_start_info: &'static str,
97 : /// Tracks time from session start (HTTP request/libpq TCP handshake)
98 : /// Through to success/failure
99 : duration_us: u64,
100 : /// If the session was successful after the disconnect, will be created one more event with filled `disconnect_timestamp`.
101 : disconnect_timestamp: Option<chrono::NaiveDateTime>,
102 : }
103 :
104 : impl From<&RequestMonitoring> for RequestData {
105 0 : fn from(value: &RequestMonitoring) -> Self {
106 0 : Self {
107 0 : session_id: value.session_id,
108 0 : peer_addr: value.peer_addr.to_string(),
109 0 : timestamp: value.first_packet.naive_utc(),
110 0 : username: value.user.as_deref().map(String::from),
111 0 : application_name: value.application.as_deref().map(String::from),
112 0 : endpoint_id: value.endpoint_id.as_deref().map(String::from),
113 0 : database: value.dbname.as_deref().map(String::from),
114 0 : project: value.project.as_deref().map(String::from),
115 0 : branch: value.branch.as_deref().map(String::from),
116 0 : auth_method: value.auth_method.as_ref().map(|x| match x {
117 0 : super::AuthMethod::Web => "web",
118 0 : super::AuthMethod::ScramSha256 => "scram_sha_256",
119 0 : super::AuthMethod::ScramSha256Plus => "scram_sha_256_plus",
120 0 : super::AuthMethod::Cleartext => "cleartext",
121 0 : }),
122 0 : protocol: value.protocol.as_str(),
123 0 : region: value.region,
124 0 : error: value.error_kind.as_ref().map(|e| e.to_metric_label()),
125 0 : success: value.success,
126 0 : cold_start_info: value.cold_start_info.as_str(),
127 0 : duration_us: SystemTime::from(value.first_packet)
128 0 : .elapsed()
129 0 : .unwrap_or_default()
130 0 : .as_micros() as u64, // 584 millenia... good enough
131 0 : disconnect_timestamp: value.disconnect_timestamp.map(|x| x.naive_utc()),
132 0 : }
133 0 : }
134 : }
135 :
136 : /// Parquet request context worker
137 : ///
138 : /// It listened on a channel for all completed requests, extracts the data and writes it into a parquet file,
139 : /// then uploads a completed batch to S3
140 0 : pub async fn worker(
141 0 : cancellation_token: CancellationToken,
142 0 : config: ParquetUploadArgs,
143 0 : ) -> anyhow::Result<()> {
144 0 : let Some(remote_storage_config) = config.parquet_upload_remote_storage else {
145 0 : tracing::warn!("parquet request upload: no s3 bucket configured");
146 0 : return Ok(());
147 : };
148 :
149 0 : let (tx, mut rx) = mpsc::unbounded_channel();
150 0 : LOG_CHAN.set(tx.downgrade()).unwrap();
151 0 :
152 0 : // setup row stream that will close on cancellation
153 0 : let cancellation_token2 = cancellation_token.clone();
154 0 : tokio::spawn(async move {
155 0 : cancellation_token2.cancelled().await;
156 : // dropping this sender will cause the channel to close only once
157 : // all the remaining inflight requests have been completed.
158 0 : drop(tx);
159 0 : });
160 0 : let rx = futures::stream::poll_fn(move |cx| rx.poll_recv(cx));
161 0 : let rx = rx.map(RequestData::from);
162 :
163 0 : let storage =
164 0 : GenericRemoteStorage::from_config(&remote_storage_config).context("remote storage init")?;
165 :
166 0 : let properties = WriterProperties::builder()
167 0 : .set_data_page_size_limit(config.parquet_upload_page_size)
168 0 : .set_compression(config.parquet_upload_compression);
169 0 :
170 0 : let parquet_config = ParquetConfig {
171 0 : propeties: Arc::new(properties.build()),
172 0 : rows_per_group: config.parquet_upload_row_group_size,
173 0 : file_size: config.parquet_upload_size,
174 0 : max_duration: config.parquet_upload_maximum_duration,
175 0 :
176 0 : #[cfg(any(test, feature = "testing"))]
177 0 : test_remote_failures: 0,
178 0 : };
179 :
180 : // TODO(anna): consider moving this to a separate function.
181 0 : if let Some(disconnect_events_storage_config) =
182 0 : config.parquet_upload_disconnect_events_remote_storage
183 : {
184 0 : let (tx_disconnect, mut rx_disconnect) = mpsc::unbounded_channel();
185 0 : LOG_CHAN_DISCONNECT.set(tx_disconnect.downgrade()).unwrap();
186 0 :
187 0 : // setup row stream that will close on cancellation
188 0 : tokio::spawn(async move {
189 0 : cancellation_token.cancelled().await;
190 : // dropping this sender will cause the channel to close only once
191 : // all the remaining inflight requests have been completed.
192 0 : drop(tx_disconnect);
193 0 : });
194 0 : let rx_disconnect = futures::stream::poll_fn(move |cx| rx_disconnect.poll_recv(cx));
195 0 : let rx_disconnect = rx_disconnect.map(RequestData::from);
196 :
197 0 : let storage_disconnect =
198 0 : GenericRemoteStorage::from_config(&disconnect_events_storage_config)
199 0 : .context("remote storage for disconnect events init")?;
200 0 : let parquet_config_disconnect = parquet_config.clone();
201 : tokio::try_join!(
202 : worker_inner(storage, rx, parquet_config),
203 : worker_inner(storage_disconnect, rx_disconnect, parquet_config_disconnect)
204 : )
205 0 : .map(|_| ())
206 : } else {
207 0 : worker_inner(storage, rx, parquet_config).await
208 : }
209 0 : }
210 :
211 : #[derive(Clone, Debug)]
212 : struct ParquetConfig {
213 : propeties: WriterPropertiesPtr,
214 : rows_per_group: usize,
215 : file_size: i64,
216 :
217 : max_duration: tokio::time::Duration,
218 :
219 : #[cfg(any(test, feature = "testing"))]
220 : test_remote_failures: u64,
221 : }
222 :
223 10 : async fn worker_inner(
224 10 : storage: GenericRemoteStorage,
225 10 : rx: impl Stream<Item = RequestData>,
226 10 : config: ParquetConfig,
227 10 : ) -> anyhow::Result<()> {
228 : #[cfg(any(test, feature = "testing"))]
229 10 : let storage = if config.test_remote_failures > 0 {
230 4 : GenericRemoteStorage::unreliable_wrapper(storage, config.test_remote_failures)
231 : } else {
232 6 : storage
233 : };
234 :
235 10 : let mut rx = std::pin::pin!(rx);
236 10 :
237 10 : let mut rows = Vec::with_capacity(config.rows_per_group);
238 :
239 10 : let schema = rows.as_slice().schema()?;
240 10 : let buffer = BytesMut::new();
241 10 : let w = buffer.writer();
242 10 : let mut w = SerializedFileWriter::new(w, schema.clone(), config.propeties.clone())?;
243 :
244 10 : let mut last_upload = time::Instant::now();
245 10 :
246 10 : let mut len = 0;
247 418010 : while let Some(row) = rx.next().await {
248 418000 : rows.push(row);
249 418000 : let force = last_upload.elapsed() > config.max_duration;
250 418000 : if rows.len() == config.rows_per_group || force {
251 210 : let rg_meta;
252 210 : (rows, w, rg_meta) = flush_rows(rows, w).await?;
253 210 : len += rg_meta.compressed_size();
254 417790 : }
255 418000 : if len > config.file_size || force {
256 56 : last_upload = time::Instant::now();
257 233 : let file = upload_parquet(w, len, &storage).await?;
258 56 : w = SerializedFileWriter::new(file, schema.clone(), config.propeties.clone())?;
259 56 : len = 0;
260 417944 : }
261 : }
262 :
263 10 : if !rows.is_empty() {
264 2 : let rg_meta;
265 2 : (_, w, rg_meta) = flush_rows(rows, w).await?;
266 2 : len += rg_meta.compressed_size();
267 8 : }
268 :
269 10 : if !w.flushed_row_groups().is_empty() {
270 24 : let _: Writer<BytesMut> = upload_parquet(w, len, &storage).await?;
271 4 : }
272 :
273 10 : Ok(())
274 10 : }
275 :
276 212 : async fn flush_rows<W>(
277 212 : rows: Vec<RequestData>,
278 212 : mut w: SerializedFileWriter<W>,
279 212 : ) -> anyhow::Result<(
280 212 : Vec<RequestData>,
281 212 : SerializedFileWriter<W>,
282 212 : RowGroupMetaDataPtr,
283 212 : )>
284 212 : where
285 212 : W: std::io::Write + Send + 'static,
286 212 : {
287 212 : let span = Span::current();
288 212 : let (mut rows, w, rg_meta) = tokio::task::spawn_blocking(move || {
289 212 : let _enter = span.enter();
290 :
291 212 : let mut rg = w.next_row_group()?;
292 212 : rows.as_slice().write_to_row_group(&mut rg)?;
293 212 : let rg_meta = rg.close()?;
294 :
295 212 : let size = rg_meta.compressed_size();
296 212 : let compression = rg_meta.compressed_size() as f64 / rg_meta.total_byte_size() as f64;
297 212 :
298 212 : debug!(size, compression, "flushed row group to parquet file");
299 :
300 212 : Ok::<_, parquet::errors::ParquetError>((rows, w, rg_meta))
301 212 : })
302 212 : .await
303 212 : .unwrap()?;
304 :
305 212 : rows.clear();
306 212 : Ok((rows, w, rg_meta))
307 212 : }
308 :
309 62 : async fn upload_parquet(
310 62 : mut w: SerializedFileWriter<Writer<BytesMut>>,
311 62 : len: i64,
312 62 : storage: &GenericRemoteStorage,
313 62 : ) -> anyhow::Result<Writer<BytesMut>> {
314 62 : let len_uncompressed = w
315 62 : .flushed_row_groups()
316 62 : .iter()
317 212 : .map(|rg| rg.total_byte_size())
318 62 : .sum::<i64>();
319 :
320 : // I don't know how compute intensive this is, although it probably isn't much... better be safe than sorry.
321 : // finish method only available on the fork: https://github.com/apache/arrow-rs/issues/5253
322 62 : let (mut buffer, metadata) =
323 62 : tokio::task::spawn_blocking(move || -> parquet::errors::Result<_> {
324 62 : let metadata = w.finish()?;
325 62 : let buffer = std::mem::take(w.inner_mut().get_mut());
326 62 : Ok((buffer, metadata))
327 62 : })
328 62 : .await
329 62 : .unwrap()?;
330 :
331 62 : let data = buffer.split().freeze();
332 62 :
333 62 : let compression = len as f64 / len_uncompressed as f64;
334 62 : let size = data.len();
335 62 : let now = chrono::Utc::now();
336 62 : let id = uuid::Uuid::new_v7(uuid::Timestamp::from_unix(
337 62 : uuid::NoContext,
338 62 : // we won't be running this in 1970. this cast is ok
339 62 : now.timestamp() as u64,
340 62 : now.timestamp_subsec_nanos(),
341 62 : ));
342 62 :
343 62 : info!(
344 : %id,
345 : rows = metadata.num_rows,
346 0 : size, compression, "uploading request parquet file"
347 : );
348 :
349 62 : let year = now.year();
350 62 : let month = now.month();
351 62 : let day = now.day();
352 62 : let hour = now.hour();
353 : // segment files by time for S3 performance
354 62 : let path = RemotePath::from_string(&format!(
355 62 : "{year:04}/{month:02}/{day:02}/{hour:02}/requests_{id}.parquet"
356 62 : ))?;
357 62 : let cancel = CancellationToken::new();
358 62 : backoff::retry(
359 86 : || async {
360 86 : let stream = futures::stream::once(futures::future::ready(Ok(data.clone())));
361 86 : storage
362 86 : .upload(stream, data.len(), &path, None, &cancel)
363 195 : .await
364 86 : },
365 62 : TimeoutOrCancel::caused_by_cancel,
366 62 : FAILED_UPLOAD_WARN_THRESHOLD,
367 62 : FAILED_UPLOAD_MAX_RETRIES,
368 62 : "request_data_upload",
369 62 : // we don't want cancellation to interrupt here, so we make a dummy cancel token
370 62 : &cancel,
371 62 : )
372 195 : .await
373 62 : .ok_or_else(|| anyhow::Error::new(TimeoutOrCancel::Cancel))
374 62 : .and_then(|x| x)
375 62 : .context("request_data_upload")?;
376 :
377 62 : Ok(buffer.writer())
378 62 : }
379 :
380 : #[cfg(test)]
381 : mod tests {
382 : use std::{net::Ipv4Addr, num::NonZeroUsize, sync::Arc};
383 :
384 : use camino::Utf8Path;
385 : use clap::Parser;
386 : use futures::{Stream, StreamExt};
387 : use itertools::Itertools;
388 : use parquet::{
389 : basic::{Compression, ZstdLevel},
390 : file::{
391 : properties::{WriterProperties, DEFAULT_PAGE_SIZE},
392 : reader::FileReader,
393 : serialized_reader::SerializedFileReader,
394 : },
395 : };
396 : use rand::{rngs::StdRng, Rng, SeedableRng};
397 : use remote_storage::{
398 : GenericRemoteStorage, RemoteStorageConfig, RemoteStorageKind, S3Config,
399 : DEFAULT_MAX_KEYS_PER_LIST_RESPONSE, DEFAULT_REMOTE_STORAGE_S3_CONCURRENCY_LIMIT,
400 : };
401 : use tokio::{sync::mpsc, time};
402 : use walkdir::WalkDir;
403 :
404 : use super::{worker_inner, ParquetConfig, ParquetUploadArgs, RequestData};
405 :
406 4 : #[derive(Parser)]
407 : struct ProxyCliArgs {
408 : #[clap(flatten)]
409 : parquet_upload: ParquetUploadArgs,
410 : }
411 :
412 : #[test]
413 2 : fn default_parser() {
414 2 : let ProxyCliArgs { parquet_upload } = ProxyCliArgs::parse_from(["proxy"]);
415 2 : assert_eq!(parquet_upload.parquet_upload_remote_storage, None);
416 2 : assert_eq!(parquet_upload.parquet_upload_row_group_size, 8192);
417 2 : assert_eq!(parquet_upload.parquet_upload_page_size, DEFAULT_PAGE_SIZE);
418 2 : assert_eq!(parquet_upload.parquet_upload_size, 100_000_000);
419 2 : assert_eq!(
420 2 : parquet_upload.parquet_upload_maximum_duration,
421 2 : time::Duration::from_secs(20 * 60)
422 2 : );
423 2 : assert_eq!(
424 2 : parquet_upload.parquet_upload_compression,
425 2 : Compression::UNCOMPRESSED
426 2 : );
427 2 : }
428 :
429 : #[test]
430 2 : fn full_parser() {
431 2 : let ProxyCliArgs { parquet_upload } = ProxyCliArgs::parse_from([
432 2 : "proxy",
433 2 : "--parquet-upload-remote-storage",
434 2 : "{bucket_name='default',prefix_in_bucket='proxy/',bucket_region='us-east-1',endpoint='http://minio:9000'}",
435 2 : "--parquet-upload-row-group-size",
436 2 : "100",
437 2 : "--parquet-upload-page-size",
438 2 : "10000",
439 2 : "--parquet-upload-size",
440 2 : "10000000",
441 2 : "--parquet-upload-maximum-duration",
442 2 : "10m",
443 2 : "--parquet-upload-compression",
444 2 : "zstd(5)",
445 2 : ]);
446 2 : assert_eq!(
447 2 : parquet_upload.parquet_upload_remote_storage,
448 2 : Some(RemoteStorageConfig {
449 2 : storage: RemoteStorageKind::AwsS3(S3Config {
450 2 : bucket_name: "default".into(),
451 2 : bucket_region: "us-east-1".into(),
452 2 : prefix_in_bucket: Some("proxy/".into()),
453 2 : endpoint: Some("http://minio:9000".into()),
454 2 : concurrency_limit: NonZeroUsize::new(
455 2 : DEFAULT_REMOTE_STORAGE_S3_CONCURRENCY_LIMIT
456 2 : )
457 2 : .unwrap(),
458 2 : max_keys_per_list_response: DEFAULT_MAX_KEYS_PER_LIST_RESPONSE,
459 2 : upload_storage_class: None,
460 2 : }),
461 2 : timeout: RemoteStorageConfig::DEFAULT_TIMEOUT,
462 2 : })
463 2 : );
464 2 : assert_eq!(parquet_upload.parquet_upload_row_group_size, 100);
465 2 : assert_eq!(parquet_upload.parquet_upload_page_size, 10000);
466 2 : assert_eq!(parquet_upload.parquet_upload_size, 10_000_000);
467 2 : assert_eq!(
468 2 : parquet_upload.parquet_upload_maximum_duration,
469 2 : time::Duration::from_secs(10 * 60)
470 2 : );
471 2 : assert_eq!(
472 2 : parquet_upload.parquet_upload_compression,
473 2 : Compression::ZSTD(ZstdLevel::try_new(5).unwrap())
474 2 : );
475 2 : }
476 :
477 418000 : fn generate_request_data(rng: &mut impl Rng) -> RequestData {
478 418000 : RequestData {
479 418000 : session_id: uuid::Builder::from_random_bytes(rng.gen()).into_uuid(),
480 418000 : peer_addr: Ipv4Addr::from(rng.gen::<[u8; 4]>()).to_string(),
481 418000 : timestamp: chrono::DateTime::from_timestamp_millis(
482 418000 : rng.gen_range(1703862754..1803862754),
483 418000 : )
484 418000 : .unwrap()
485 418000 : .naive_utc(),
486 418000 : application_name: Some("test".to_owned()),
487 418000 : username: Some(hex::encode(rng.gen::<[u8; 4]>())),
488 418000 : endpoint_id: Some(hex::encode(rng.gen::<[u8; 16]>())),
489 418000 : database: Some(hex::encode(rng.gen::<[u8; 16]>())),
490 418000 : project: Some(hex::encode(rng.gen::<[u8; 16]>())),
491 418000 : branch: Some(hex::encode(rng.gen::<[u8; 16]>())),
492 418000 : auth_method: None,
493 418000 : protocol: ["tcp", "ws", "http"][rng.gen_range(0..3)],
494 418000 : region: "us-east-1",
495 418000 : error: None,
496 418000 : success: rng.gen(),
497 418000 : cold_start_info: "no",
498 418000 : duration_us: rng.gen_range(0..30_000_000),
499 418000 : disconnect_timestamp: None,
500 418000 : }
501 418000 : }
502 :
503 14 : fn random_stream(len: usize) -> impl Stream<Item = RequestData> + Unpin {
504 14 : let mut rng = StdRng::from_seed([0x39; 32]);
505 14 : futures::stream::iter(
506 418000 : std::iter::repeat_with(move || generate_request_data(&mut rng)).take(len),
507 14 : )
508 14 : }
509 :
510 10 : async fn run_test(
511 10 : tmpdir: &Utf8Path,
512 10 : config: ParquetConfig,
513 10 : rx: impl Stream<Item = RequestData>,
514 10 : ) -> Vec<(u64, usize, i64)> {
515 10 : let remote_storage_config = RemoteStorageConfig {
516 10 : storage: RemoteStorageKind::LocalFs(tmpdir.to_path_buf()),
517 10 : timeout: std::time::Duration::from_secs(120),
518 10 : };
519 10 : let storage = GenericRemoteStorage::from_config(&remote_storage_config).unwrap();
520 10 :
521 609 : worker_inner(storage, rx, config).await.unwrap();
522 10 :
523 10 : let mut files = WalkDir::new(tmpdir.as_std_path())
524 10 : .into_iter()
525 112 : .filter_map(|entry| entry.ok())
526 112 : .filter(|entry| entry.file_type().is_file())
527 62 : .map(|entry| entry.path().to_path_buf())
528 10 : .collect_vec();
529 10 : files.sort();
530 10 :
531 10 : files
532 10 : .into_iter()
533 62 : .map(|path| std::fs::File::open(tmpdir.as_std_path().join(path)).unwrap())
534 62 : .map(|file| {
535 62 : (
536 62 : file.metadata().unwrap(),
537 62 : SerializedFileReader::new(file).unwrap().metadata().clone(),
538 62 : )
539 62 : })
540 62 : .map(|(file_meta, parquet_meta)| {
541 62 : (
542 62 : file_meta.len(),
543 62 : parquet_meta.num_row_groups(),
544 62 : parquet_meta.file_metadata().num_rows(),
545 62 : )
546 62 : })
547 10 : .collect()
548 10 : }
549 :
550 : #[tokio::test]
551 2 : async fn verify_parquet_no_compression() {
552 2 : let tmpdir = camino_tempfile::tempdir().unwrap();
553 2 :
554 2 : let config = ParquetConfig {
555 2 : propeties: Arc::new(WriterProperties::new()),
556 2 : rows_per_group: 2_000,
557 2 : file_size: 1_000_000,
558 2 : max_duration: time::Duration::from_secs(20 * 60),
559 2 : test_remote_failures: 0,
560 2 : };
561 2 :
562 2 : let rx = random_stream(50_000);
563 124 : let file_stats = run_test(tmpdir.path(), config, rx).await;
564 2 :
565 2 : assert_eq!(
566 2 : file_stats,
567 2 : [
568 2 : (1315314, 3, 6000),
569 2 : (1315307, 3, 6000),
570 2 : (1315367, 3, 6000),
571 2 : (1315324, 3, 6000),
572 2 : (1315454, 3, 6000),
573 2 : (1315296, 3, 6000),
574 2 : (1315088, 3, 6000),
575 2 : (1315324, 3, 6000),
576 2 : (438713, 1, 2000)
577 2 : ]
578 2 : );
579 2 :
580 2 : tmpdir.close().unwrap();
581 2 : }
582 :
583 : #[tokio::test]
584 2 : async fn verify_parquet_min_compression() {
585 2 : let tmpdir = camino_tempfile::tempdir().unwrap();
586 2 :
587 2 : let config = ParquetConfig {
588 2 : propeties: Arc::new(
589 2 : WriterProperties::builder()
590 2 : .set_compression(parquet::basic::Compression::ZSTD(ZstdLevel::default()))
591 2 : .build(),
592 2 : ),
593 2 : rows_per_group: 2_000,
594 2 : file_size: 1_000_000,
595 2 : max_duration: time::Duration::from_secs(20 * 60),
596 2 : test_remote_failures: 0,
597 2 : };
598 2 :
599 2 : let rx = random_stream(50_000);
600 92 : let file_stats = run_test(tmpdir.path(), config, rx).await;
601 2 :
602 2 : // with compression, there are fewer files with more rows per file
603 2 : assert_eq!(
604 2 : file_stats,
605 2 : [
606 2 : (1222212, 5, 10000),
607 2 : (1228362, 5, 10000),
608 2 : (1230156, 5, 10000),
609 2 : (1229518, 5, 10000),
610 2 : (1220796, 5, 10000)
611 2 : ]
612 2 : );
613 2 :
614 2 : tmpdir.close().unwrap();
615 2 : }
616 :
617 : #[tokio::test]
618 2 : async fn verify_parquet_strong_compression() {
619 2 : let tmpdir = camino_tempfile::tempdir().unwrap();
620 2 :
621 2 : let config = ParquetConfig {
622 2 : propeties: Arc::new(
623 2 : WriterProperties::builder()
624 2 : .set_compression(parquet::basic::Compression::ZSTD(
625 2 : ZstdLevel::try_new(10).unwrap(),
626 2 : ))
627 2 : .build(),
628 2 : ),
629 2 : rows_per_group: 2_000,
630 2 : file_size: 1_000_000,
631 2 : max_duration: time::Duration::from_secs(20 * 60),
632 2 : test_remote_failures: 0,
633 2 : };
634 2 :
635 2 : let rx = random_stream(50_000);
636 92 : let file_stats = run_test(tmpdir.path(), config, rx).await;
637 2 :
638 2 : // with strong compression, the files are smaller
639 2 : assert_eq!(
640 2 : file_stats,
641 2 : [
642 2 : (1207859, 5, 10000),
643 2 : (1207590, 5, 10000),
644 2 : (1207883, 5, 10000),
645 2 : (1207871, 5, 10000),
646 2 : (1208126, 5, 10000)
647 2 : ]
648 2 : );
649 2 :
650 2 : tmpdir.close().unwrap();
651 2 : }
652 :
653 : #[tokio::test]
654 2 : async fn verify_parquet_unreliable_upload() {
655 2 : let tmpdir = camino_tempfile::tempdir().unwrap();
656 2 :
657 2 : let config = ParquetConfig {
658 2 : propeties: Arc::new(WriterProperties::new()),
659 2 : rows_per_group: 2_000,
660 2 : file_size: 1_000_000,
661 2 : max_duration: time::Duration::from_secs(20 * 60),
662 2 : test_remote_failures: 2,
663 2 : };
664 2 :
665 2 : let rx = random_stream(50_000);
666 123 : let file_stats = run_test(tmpdir.path(), config, rx).await;
667 2 :
668 2 : assert_eq!(
669 2 : file_stats,
670 2 : [
671 2 : (1315314, 3, 6000),
672 2 : (1315307, 3, 6000),
673 2 : (1315367, 3, 6000),
674 2 : (1315324, 3, 6000),
675 2 : (1315454, 3, 6000),
676 2 : (1315296, 3, 6000),
677 2 : (1315088, 3, 6000),
678 2 : (1315324, 3, 6000),
679 2 : (438713, 1, 2000)
680 2 : ]
681 2 : );
682 2 :
683 2 : tmpdir.close().unwrap();
684 2 : }
685 :
686 : #[tokio::test(start_paused = true)]
687 2 : async fn verify_parquet_regular_upload() {
688 2 : let tmpdir = camino_tempfile::tempdir().unwrap();
689 2 :
690 2 : let config = ParquetConfig {
691 2 : propeties: Arc::new(WriterProperties::new()),
692 2 : rows_per_group: 2_000,
693 2 : file_size: 1_000_000,
694 2 : max_duration: time::Duration::from_secs(60),
695 2 : test_remote_failures: 2,
696 2 : };
697 2 :
698 2 : let (tx, mut rx) = mpsc::unbounded_channel();
699 2 :
700 2 : tokio::spawn(async move {
701 8 : for _ in 0..3 {
702 6 : let mut s = random_stream(3000);
703 18006 : while let Some(r) = s.next().await {
704 18000 : tx.send(r).unwrap();
705 18000 : }
706 6 : time::sleep(time::Duration::from_secs(70)).await
707 2 : }
708 2 : });
709 2 :
710 18142 : let rx = futures::stream::poll_fn(move |cx| rx.poll_recv(cx));
711 178 : let file_stats = run_test(tmpdir.path(), config, rx).await;
712 2 :
713 2 : // files are smaller than the size threshold, but they took too long to fill so were flushed early
714 2 : assert_eq!(
715 2 : file_stats,
716 2 : [(659462, 2, 3001), (659176, 2, 3000), (658972, 2, 2999)]
717 2 : );
718 2 :
719 2 : tmpdir.close().unwrap();
720 2 : }
721 : }
|