Skip to content

Commit

Permalink
Use sort ordering on timestamp array (#443)
Browse files Browse the repository at this point in the history
Write timestamp sorted metadata to parquet and provide 
external sort information to datafusion. This way the 
SortExec can be avoided in execution plan with most 
queries which use order by p_timestamp.

For example, the query
"explain select p_timestamp from {{stream_name}} 
order by p_timestamp asc"

In physical plan it is visible that SortExec is eliminated as 
output_ordering is pushed to ParquetExec node

"plan": "SortPreservingMergeExec: [p_timestamp@0 ASC NULLS LAST]
  ParquetExec: file_groups={4 groups: [.....]}, projection=[p_timestamp], 
output_ordering=[p_timestamp@0 ASC NULLS LAST]",

Note that this is still not the most optimised version of this query as 
SortPreservingExec is not really needed here. The issue here 
is that the datafusion is not aware that the partitions / files are 
non overlapping when considering timestamp

Also if the target partition limit is crossed then datafusion again 
adds SortExec to physical plan.

Fixes #430
  • Loading branch information
trueleo authored Jun 25, 2023
1 parent f9fb6c5 commit 3e5548d
Show file tree
Hide file tree
Showing 8 changed files with 66 additions and 106 deletions.
20 changes: 4 additions & 16 deletions server/src/event/format.rs
Original file line number Diff line number Diff line change
Expand Up @@ -20,9 +20,8 @@
use std::{collections::HashMap, sync::Arc};

use anyhow::{anyhow, Error as AnyError};
use arrow_array::{RecordBatch, StringArray, TimestampMillisecondArray};
use arrow_array::{RecordBatch, StringArray};
use arrow_schema::{DataType, Field, Schema, TimeUnit};
use chrono::Utc;

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

Expand Down Expand Up @@ -94,25 +93,14 @@ pub trait EventFormat: Sized {
let tags_arr = StringArray::from_iter_values(std::iter::repeat(&tags).take(rb.num_rows()));
let metadata_arr =
StringArray::from_iter_values(std::iter::repeat(&metadata).take(rb.num_rows()));
let timestamp_array = get_timestamp_array(rb.num_rows());

// modify the record batch to add fields to respective indexes
let rb = utils::arrow::replace_columns(
Arc::clone(&schema),
rb,
&[0, tags_index, metadata_index],
&[
Arc::new(timestamp_array),
Arc::new(tags_arr),
Arc::new(metadata_arr),
],
&rb,
&[tags_index, metadata_index],
&[Arc::new(tags_arr), Arc::new(metadata_arr)],
);

Ok((rb, is_first))
}
}

fn get_timestamp_array(size: usize) -> TimestampMillisecondArray {
let time = Utc::now();
TimestampMillisecondArray::from_value(time.timestamp_millis(), size)
}
20 changes: 17 additions & 3 deletions server/src/event/writer/file_writer.rs
Original file line number Diff line number Diff line change
Expand Up @@ -17,14 +17,17 @@
*
*/

use arrow_array::RecordBatch;
use arrow_array::{RecordBatch, TimestampMillisecondArray};
use arrow_ipc::writer::StreamWriter;
use chrono::Utc;
use derive_more::{Deref, DerefMut};
use std::collections::HashMap;
use std::fs::{File, OpenOptions};
use std::path::PathBuf;
use std::sync::Arc;

use crate::storage::staging::StorageDir;
use crate::utils;

use super::errors::StreamWriterError;

Expand All @@ -44,17 +47,24 @@ impl FileWriter {
schema_key: &str,
record: &RecordBatch,
) -> Result<(), StreamWriterError> {
let record = utils::arrow::replace_columns(
record.schema(),
record,
&[0],
&[Arc::new(get_timestamp_array(record.num_rows()))],
);

match self.get_mut(schema_key) {
Some(writer) => {
writer
.writer
.write(record)
.write(&record)
.map_err(StreamWriterError::Writer)?;
}
// entry is not present thus we create it
None => {
// this requires mutable borrow of the map so we drop this read lock and wait for write lock
let (path, writer) = init_new_stream_writer_file(stream_name, schema_key, record)?;
let (path, writer) = init_new_stream_writer_file(stream_name, schema_key, &record)?;
self.insert(
schema_key.to_owned(),
ArrowWriter {
Expand All @@ -75,6 +85,10 @@ impl FileWriter {
}
}

fn get_timestamp_array(size: usize) -> TimestampMillisecondArray {
TimestampMillisecondArray::from_value(Utc::now().timestamp_millis(), size)
}

fn init_new_stream_writer_file(
stream_name: &str,
schema_key: &str,
Expand Down
29 changes: 26 additions & 3 deletions server/src/query.rs
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,8 @@ use chrono::TimeZone;
use chrono::{DateTime, Utc};
use datafusion::arrow::datatypes::Schema;
use datafusion::arrow::record_batch::RecordBatch;
use datafusion::datasource::file_format::parquet::ParquetFormat;
use datafusion::datasource::listing::{ListingOptions, ListingTable, ListingTableConfig};
use datafusion::execution::context::SessionState;
use datafusion::execution::disk_manager::DiskManagerConfig;
use datafusion::execution::runtime_env::RuntimeEnv;
Expand All @@ -32,6 +34,7 @@ use std::path::Path;
use std::sync::Arc;
use sysinfo::{System, SystemExt};

use crate::event::DEFAULT_TIMESTAMP_KEY;
use crate::option::CONFIG;
use crate::storage::ObjectStorageError;
use crate::storage::{ObjectStorage, OBJECT_STORE_DATA_GRANULARITY};
Expand Down Expand Up @@ -121,10 +124,30 @@ impl Query {
storage: Arc<dyn ObjectStorage + Send>,
) -> Result<(Vec<RecordBatch>, Vec<String>), ExecuteError> {
let ctx = self.create_session_context();
let prefixes = self.get_prefixes();
let Some(table) = storage.query_table(prefixes, Arc::clone(&self.schema))? else { return Ok((Vec::new(), Vec::new())) };
let prefixes = storage.query_prefixes(self.get_prefixes());

ctx.register_table(&*self.stream_name, Arc::new(table))
if prefixes.is_empty() {
return Ok((Vec::new(), Vec::new()));
}

let file_format = ParquetFormat::default().with_enable_pruning(Some(true));
let listing_options = ListingOptions {
file_extension: ".parquet".to_string(),
file_sort_order: vec![vec![col(DEFAULT_TIMESTAMP_KEY).sort(true, false)]],
infinite_source: false,
format: Arc::new(file_format),
table_partition_cols: vec![],
collect_stat: true,
target_partitions: 32,
};

let config = ListingTableConfig::new_with_multi_paths(prefixes)
.with_listing_options(listing_options)
.with_schema(self.schema.clone());

let table = Arc::new(ListingTable::try_new(config)?);

ctx.register_table(&*self.stream_name, table)
.map_err(ObjectStorageError::DataFusionError)?;
// execute the query and collect results
let df = ctx.sql(self.query.as_str()).await?;
Expand Down
41 changes: 4 additions & 37 deletions server/src/storage/localfs.rs
Original file line number Diff line number Diff line change
Expand Up @@ -24,15 +24,7 @@ use std::{

use async_trait::async_trait;
use bytes::Bytes;
use datafusion::arrow::datatypes::Schema;
use datafusion::{
datasource::{
file_format::parquet::ParquetFormat,
listing::{ListingOptions, ListingTable, ListingTableConfig, ListingTableUrl},
},
error::DataFusionError,
execution::runtime_env::RuntimeConfig,
};
use datafusion::{datasource::listing::ListingTableUrl, execution::runtime_env::RuntimeConfig};
use fs_extra::file::{move_file, CopyOptions};
use futures::{stream::FuturesUnordered, TryStreamExt};
use relative_path::RelativePath;
Expand Down Expand Up @@ -202,39 +194,14 @@ impl ObjectStorage for LocalFS {
Ok(())
}

fn query_table(
&self,
prefixes: Vec<String>,
schema: Arc<Schema>,
) -> Result<Option<ListingTable>, DataFusionError> {
let prefixes: Vec<ListingTableUrl> = prefixes
fn query_prefixes(&self, prefixes: Vec<String>) -> Vec<ListingTableUrl> {
prefixes
.into_iter()
.filter_map(|prefix| {
let path = self.root.join(prefix);
ListingTableUrl::parse(path.to_str().unwrap()).ok()
})
.collect();

if prefixes.is_empty() {
return Ok(None);
}

let file_format = ParquetFormat::default().with_enable_pruning(Some(true));
let listing_options = ListingOptions {
file_extension: ".parquet".to_string(),
file_sort_order: Vec::new(),
infinite_source: false,
format: Arc::new(file_format),
table_partition_cols: vec![],
collect_stat: true,
target_partitions: 1,
};

let config = ListingTableConfig::new_with_multi_paths(prefixes)
.with_listing_options(listing_options)
.with_schema(schema);

Ok(Some(ListingTable::try_new(config)?))
.collect()
}
}

Expand Down
12 changes: 3 additions & 9 deletions server/src/storage/object_storage.rs
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@ use super::{
retention::Retention, staging::convert_disk_files_to_parquet, LogStream, ObjectStorageError,
ObjectStoreFormat, Permisssion, StorageDir, StorageMetadata,
};

use crate::{
alerts::Alerts,
metadata::STREAM_INFO,
Expand All @@ -32,10 +33,7 @@ use actix_web_prometheus::PrometheusMetrics;
use arrow_schema::Schema;
use async_trait::async_trait;
use bytes::Bytes;
use datafusion::{
datasource::listing::ListingTable, error::DataFusionError,
execution::runtime_env::RuntimeConfig,
};
use datafusion::{datasource::listing::ListingTableUrl, execution::runtime_env::RuntimeConfig};
use relative_path::RelativePath;
use relative_path::RelativePathBuf;
use serde_json::Value;
Expand Down Expand Up @@ -69,11 +67,7 @@ pub trait ObjectStorage: Sync + 'static {
async fn list_streams(&self) -> Result<Vec<LogStream>, ObjectStorageError>;
async fn list_dates(&self, stream_name: &str) -> Result<Vec<String>, ObjectStorageError>;
async fn upload_file(&self, key: &str, path: &Path) -> Result<(), ObjectStorageError>;
fn query_table(
&self,
prefixes: Vec<String>,
schema: Arc<Schema>,
) -> Result<Option<ListingTable>, DataFusionError>;
fn query_prefixes(&self, prefixes: Vec<String>) -> Vec<ListingTableUrl>;

async fn put_schema(
&self,
Expand Down
40 changes: 4 additions & 36 deletions server/src/storage/s3.rs
Original file line number Diff line number Diff line change
Expand Up @@ -18,16 +18,10 @@

use async_trait::async_trait;
use bytes::Bytes;
use datafusion::arrow::datatypes::Schema;

use datafusion::datasource::file_format::parquet::ParquetFormat;
use datafusion::datasource::listing::{
ListingOptions, ListingTable, ListingTableConfig, ListingTableUrl,
};
use datafusion::datasource::listing::ListingTableUrl;
use datafusion::datasource::object_store::{
DefaultObjectStoreRegistry, ObjectStoreRegistry, ObjectStoreUrl,
};
use datafusion::error::DataFusionError;
use datafusion::execution::runtime_env::RuntimeConfig;
use futures::stream::FuturesUnordered;
use futures::{StreamExt, TryStreamExt};
Expand Down Expand Up @@ -453,40 +447,14 @@ impl ObjectStorage for S3 {
Ok(())
}

fn query_table(
&self,
prefixes: Vec<String>,
schema: Arc<Schema>,
) -> Result<Option<ListingTable>, DataFusionError> {
// Get all prefix paths and convert them into futures which yeilds ListingTableUrl
let prefixes: Vec<ListingTableUrl> = prefixes
fn query_prefixes(&self, prefixes: Vec<String>) -> Vec<ListingTableUrl> {
prefixes
.into_iter()
.map(|prefix| {
let path = format!("s3://{}/{}", &self.bucket, prefix);
ListingTableUrl::parse(path).unwrap()
})
.collect();

if prefixes.is_empty() {
return Ok(None);
}

let file_format = ParquetFormat::default().with_enable_pruning(Some(true));
let listing_options = ListingOptions {
file_extension: ".parquet".to_string(),
file_sort_order: Vec::default(),
infinite_source: false,
format: Arc::new(file_format),
table_partition_cols: vec![],
collect_stat: true,
target_partitions: 1,
};

let config = ListingTableConfig::new_with_multi_paths(prefixes)
.with_listing_options(listing_options)
.with_schema(schema);

Ok(Some(ListingTable::try_new(config)?))
.collect()
}
}

Expand Down
6 changes: 6 additions & 0 deletions server/src/storage/staging.rs
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,7 @@ use parquet::{
basic::Encoding,
errors::ParquetError,
file::properties::{WriterProperties, WriterPropertiesBuilder},
format::SortingColumn,
schema::types::ColumnPath,
};

Expand Down Expand Up @@ -234,6 +235,11 @@ fn parquet_writer_props() -> WriterPropertiesBuilder {
ColumnPath::new(vec![DEFAULT_TIMESTAMP_KEY.to_string()]),
Encoding::DELTA_BINARY_PACKED,
)
.set_sorting_columns(Some(vec![SortingColumn {
column_idx: 0,
descending: false,
nulls_first: false,
}]))
}

#[derive(Debug, thiserror::Error)]
Expand Down
4 changes: 2 additions & 2 deletions server/src/utils/arrow.rs
Original file line number Diff line number Diff line change
Expand Up @@ -31,7 +31,7 @@ pub use merged_reader::MergedRecordReader;

pub fn replace_columns(
schema: Arc<Schema>,
batch: RecordBatch,
batch: &RecordBatch,
indexes: &[usize],
arrays: &[Arc<dyn Array + 'static>],
) -> RecordBatch {
Expand Down Expand Up @@ -73,7 +73,7 @@ mod tests {

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

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

assert_eq!(new_rb.schema(), schema_ref);
assert_eq!(new_rb.num_columns(), 3);
Expand Down

0 comments on commit 3e5548d

Please sign in to comment.