Skip to content

Commit f7726a4

Browse files
committed
Use sort ordering on timestamp array
1 parent f9fb6c5 commit f7726a4

File tree

6 files changed

+40
-27
lines changed

6 files changed

+40
-27
lines changed

server/src/event/format.rs

Lines changed: 4 additions & 16 deletions
Original file line numberDiff line numberDiff line change
@@ -20,9 +20,8 @@
2020
use std::{collections::HashMap, sync::Arc};
2121

2222
use anyhow::{anyhow, Error as AnyError};
23-
use arrow_array::{RecordBatch, StringArray, TimestampMillisecondArray};
23+
use arrow_array::{RecordBatch, StringArray};
2424
use arrow_schema::{DataType, Field, Schema, TimeUnit};
25-
use chrono::Utc;
2625

2726
use crate::utils::{self, arrow::get_field};
2827

@@ -94,25 +93,14 @@ pub trait EventFormat: Sized {
9493
let tags_arr = StringArray::from_iter_values(std::iter::repeat(&tags).take(rb.num_rows()));
9594
let metadata_arr =
9695
StringArray::from_iter_values(std::iter::repeat(&metadata).take(rb.num_rows()));
97-
let timestamp_array = get_timestamp_array(rb.num_rows());
98-
9996
// modify the record batch to add fields to respective indexes
10097
let rb = utils::arrow::replace_columns(
10198
Arc::clone(&schema),
102-
rb,
103-
&[0, tags_index, metadata_index],
104-
&[
105-
Arc::new(timestamp_array),
106-
Arc::new(tags_arr),
107-
Arc::new(metadata_arr),
108-
],
99+
&rb,
100+
&[tags_index, metadata_index],
101+
&[Arc::new(tags_arr), Arc::new(metadata_arr)],
109102
);
110103

111104
Ok((rb, is_first))
112105
}
113106
}
114-
115-
fn get_timestamp_array(size: usize) -> TimestampMillisecondArray {
116-
let time = Utc::now();
117-
TimestampMillisecondArray::from_value(time.timestamp_millis(), size)
118-
}

server/src/event/writer/file_writer.rs

Lines changed: 17 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -17,14 +17,17 @@
1717
*
1818
*/
1919

20-
use arrow_array::RecordBatch;
20+
use arrow_array::{RecordBatch, TimestampMillisecondArray};
2121
use arrow_ipc::writer::StreamWriter;
22+
use chrono::Utc;
2223
use derive_more::{Deref, DerefMut};
2324
use std::collections::HashMap;
2425
use std::fs::{File, OpenOptions};
2526
use std::path::PathBuf;
27+
use std::sync::Arc;
2628

2729
use crate::storage::staging::StorageDir;
30+
use crate::utils;
2831

2932
use super::errors::StreamWriterError;
3033

@@ -44,17 +47,24 @@ impl FileWriter {
4447
schema_key: &str,
4548
record: &RecordBatch,
4649
) -> Result<(), StreamWriterError> {
50+
let record = utils::arrow::replace_columns(
51+
record.schema(),
52+
record,
53+
&[0],
54+
&[Arc::new(get_timestamp_array(record.num_rows()))],
55+
);
56+
4757
match self.get_mut(schema_key) {
4858
Some(writer) => {
4959
writer
5060
.writer
51-
.write(record)
61+
.write(&record)
5262
.map_err(StreamWriterError::Writer)?;
5363
}
5464
// entry is not present thus we create it
5565
None => {
5666
// this requires mutable borrow of the map so we drop this read lock and wait for write lock
57-
let (path, writer) = init_new_stream_writer_file(stream_name, schema_key, record)?;
67+
let (path, writer) = init_new_stream_writer_file(stream_name, schema_key, &record)?;
5868
self.insert(
5969
schema_key.to_owned(),
6070
ArrowWriter {
@@ -75,6 +85,10 @@ impl FileWriter {
7585
}
7686
}
7787

88+
fn get_timestamp_array(size: usize) -> TimestampMillisecondArray {
89+
TimestampMillisecondArray::from_value(Utc::now().timestamp_millis(), size)
90+
}
91+
7892
fn init_new_stream_writer_file(
7993
stream_name: &str,
8094
schema_key: &str,

server/src/storage/localfs.rs

Lines changed: 7 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -24,7 +24,7 @@ use std::{
2424

2525
use async_trait::async_trait;
2626
use bytes::Bytes;
27-
use datafusion::arrow::datatypes::Schema;
27+
use datafusion::{arrow::datatypes::Schema, prelude::col};
2828
use datafusion::{
2929
datasource::{
3030
file_format::parquet::ParquetFormat,
@@ -39,7 +39,10 @@ use relative_path::RelativePath;
3939
use tokio::fs::{self, DirEntry};
4040
use tokio_stream::wrappers::ReadDirStream;
4141

42-
use crate::metrics::storage::{localfs::REQUEST_RESPONSE_TIME, StorageMetrics};
42+
use crate::{
43+
event::DEFAULT_TIMESTAMP_KEY,
44+
metrics::storage::{localfs::REQUEST_RESPONSE_TIME, StorageMetrics},
45+
};
4346
use crate::{option::validation, utils::validate_path_is_writeable};
4447

4548
use super::{object_storage, LogStream, ObjectStorage, ObjectStorageError, ObjectStorageProvider};
@@ -222,12 +225,12 @@ impl ObjectStorage for LocalFS {
222225
let file_format = ParquetFormat::default().with_enable_pruning(Some(true));
223226
let listing_options = ListingOptions {
224227
file_extension: ".parquet".to_string(),
225-
file_sort_order: Vec::new(),
228+
file_sort_order: vec![vec![col(DEFAULT_TIMESTAMP_KEY).sort(true, false)]],
226229
infinite_source: false,
227230
format: Arc::new(file_format),
228231
table_partition_cols: vec![],
229232
collect_stat: true,
230-
target_partitions: 1,
233+
target_partitions: 32,
231234
};
232235

233236
let config = ListingTableConfig::new_with_multi_paths(prefixes)

server/src/storage/s3.rs

Lines changed: 4 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -29,6 +29,7 @@ use datafusion::datasource::object_store::{
2929
};
3030
use datafusion::error::DataFusionError;
3131
use datafusion::execution::runtime_env::RuntimeConfig;
32+
use datafusion::prelude::col;
3233
use futures::stream::FuturesUnordered;
3334
use futures::{StreamExt, TryStreamExt};
3435
use object_store::aws::{AmazonS3, AmazonS3Builder, Checksum};
@@ -44,6 +45,7 @@ use std::path::Path as StdPath;
4445
use std::sync::Arc;
4546
use std::time::{Duration, Instant};
4647

48+
use crate::event::DEFAULT_TIMESTAMP_KEY;
4749
use crate::metrics::storage::{s3::REQUEST_RESPONSE_TIME, StorageMetrics};
4850
use crate::storage::{LogStream, ObjectStorage, ObjectStorageError};
4951

@@ -474,12 +476,12 @@ impl ObjectStorage for S3 {
474476
let file_format = ParquetFormat::default().with_enable_pruning(Some(true));
475477
let listing_options = ListingOptions {
476478
file_extension: ".parquet".to_string(),
477-
file_sort_order: Vec::default(),
479+
file_sort_order: vec![vec![col(DEFAULT_TIMESTAMP_KEY).sort(true, false)]],
478480
infinite_source: false,
479481
format: Arc::new(file_format),
480482
table_partition_cols: vec![],
481483
collect_stat: true,
482-
target_partitions: 1,
484+
target_partitions: 32,
483485
};
484486

485487
let config = ListingTableConfig::new_with_multi_paths(prefixes)

server/src/storage/staging.rs

Lines changed: 6 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -32,6 +32,7 @@ use parquet::{
3232
basic::Encoding,
3333
errors::ParquetError,
3434
file::properties::{WriterProperties, WriterPropertiesBuilder},
35+
format::SortingColumn,
3536
schema::types::ColumnPath,
3637
};
3738

@@ -234,6 +235,11 @@ fn parquet_writer_props() -> WriterPropertiesBuilder {
234235
ColumnPath::new(vec![DEFAULT_TIMESTAMP_KEY.to_string()]),
235236
Encoding::DELTA_BINARY_PACKED,
236237
)
238+
.set_sorting_columns(Some(vec![SortingColumn {
239+
column_idx: 0,
240+
descending: false,
241+
nulls_first: false,
242+
}]))
237243
}
238244

239245
#[derive(Debug, thiserror::Error)]

server/src/utils/arrow.rs

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -31,7 +31,7 @@ pub use merged_reader::MergedRecordReader;
3131

3232
pub fn replace_columns(
3333
schema: Arc<Schema>,
34-
batch: RecordBatch,
34+
batch: &RecordBatch,
3535
indexes: &[usize],
3636
arrays: &[Arc<dyn Array + 'static>],
3737
) -> RecordBatch {
@@ -73,7 +73,7 @@ mod tests {
7373

7474
let arr: Arc<dyn Array + 'static> = Arc::new(Int32Array::from_value(0, 3));
7575

76-
let new_rb = replace_columns(schema_ref.clone(), rb, &[2], &[arr]);
76+
let new_rb = replace_columns(schema_ref.clone(), &rb, &[2], &[arr]);
7777

7878
assert_eq!(new_rb.schema(), schema_ref);
7979
assert_eq!(new_rb.num_columns(), 3);

0 commit comments

Comments
 (0)