From 5cb8dbe1d15ec200299b88293364e468dbe1fd6d Mon Sep 17 00:00:00 2001 From: m09526 Date: Mon, 11 Aug 2025 16:20:17 +0000 Subject: [PATCH 01/28] WIP query code --- rust/compactor/src/bin/main.rs | 3 +- rust/sleeper_core/src/datafusion.rs | 1 + rust/sleeper_core/src/datafusion/compact.rs | 4 +- .../src/datafusion/leaf_partition_query.rs | 123 ++++++++++++++++++ rust/sleeper_core/src/lib.rs | 4 +- 5 files changed, 130 insertions(+), 5 deletions(-) create mode 100644 rust/sleeper_core/src/datafusion/leaf_partition_query.rs diff --git a/rust/compactor/src/bin/main.rs b/rust/compactor/src/bin/main.rs index 45bca3dbf90..f8a951e47b6 100644 --- a/rust/compactor/src/bin/main.rs +++ b/rust/compactor/src/bin/main.rs @@ -149,7 +149,7 @@ async fn main() -> color_eyre::Result<()> { dict_enc_values: true, }; - let details = CommonConfig { + let mut details = CommonConfig { aws_config: None, input_files: input_urls, input_files_sorted: true, @@ -162,6 +162,7 @@ async fn main() -> color_eyre::Result<()> { }, iterator_config: args.iterator_config, }; + details.sanitise_java_s3_urls(); let result = run_compaction(&details).await; match result { diff --git a/rust/sleeper_core/src/datafusion.rs b/rust/sleeper_core/src/datafusion.rs index d06a770107c..cc30d1f9ebf 100644 --- a/rust/sleeper_core/src/datafusion.rs +++ b/rust/sleeper_core/src/datafusion.rs @@ -47,6 +47,7 @@ use std::{collections::HashMap, sync::Arc}; mod compact; mod config; mod filter_aggregation_config; +mod leaf_partition_query; mod metrics; mod region; pub mod sketch; diff --git a/rust/sleeper_core/src/datafusion/compact.rs b/rust/sleeper_core/src/datafusion/compact.rs index 407668d562c..e4ef65e7907 100644 --- a/rust/sleeper_core/src/datafusion/compact.rs +++ b/rust/sleeper_core/src/datafusion/compact.rs @@ -74,9 +74,9 @@ pub async fn compact( Ok(CompactionResult::from(&stats)) } -/// Creates the dataframe for a compaction. +/// Creates the [`DataFrame`] for a compaction. /// -/// This applies necessary filtering, sorting and sketch creation +/// This applies necessary loading, filtering, sorting, aggregation and sketch creation /// steps to the plan. /// /// # Errors diff --git a/rust/sleeper_core/src/datafusion/leaf_partition_query.rs b/rust/sleeper_core/src/datafusion/leaf_partition_query.rs new file mode 100644 index 00000000000..96e5c479919 --- /dev/null +++ b/rust/sleeper_core/src/datafusion/leaf_partition_query.rs @@ -0,0 +1,123 @@ +//! Contains the implementation for performing Sleeper leaf queries +//! using Apache `DataFusion`. +/* +* Copyright 2022-2025 Crown Copyright +* +* Licensed under the Apache License, Version 2.0 (the "License"); +* you may not use this file except in compliance with the License. +* You may obtain a copy of the License at +* +* http://www.apache.org/licenses/LICENSE-2.0 +* +* Unless required by applicable law or agreed to in writing, software +* distributed under the License is distributed on an "AS IS" BASIS, +* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +* See the License for the specific language governing permissions and +* limitations under the License. +*/ +use crate::{ + CommonConfig, OperationOutput, SleeperPartitionRegion, + datafusion::{ParquetWriterConfigurer, SleeperOperations}, +}; +#[cfg(doc)] +use arrow::record_batch::RecordBatch; +use datafusion::{ + error::DataFusionError, + prelude::{DataFrame, SessionConfig, SessionContext}, +}; +use log::info; +use objectstore_ext::s3::ObjectStoreFactory; +use std::fmt::{Display, Formatter}; + +/// All information needed for a Sleeper leaf partition query. +#[derive(Debug, Default)] +pub struct LeafPartitionQueryConfig<'a> { + /// Basic information + pub common: CommonConfig<'a>, + /// Query ranges + pub ranges: Vec>, + /// Should sketches be produced? + pub write_quantile_sketch: bool, + /// Should logical/physical plan explanation be logged? + pub explain_plans: bool, +} + +impl Display for LeafPartitionQueryConfig<'_> { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + write!( + f, + "Query config: {}, ranges: {:?} write quantile sketches: {}", + self.common, self.ranges, self.write_quantile_sketch + ) + } +} + +/// Contains the query output result. +#[derive(Debug, Default)] +pub enum LeafQueryOutput { + /// Results are returned via an iterator of Arrow [`RecordBatch`]es. + #[default] + ArrowBatches, + /// Results have been written to a Parquet file. + ParquetFile, +} + +/// Executes a Sleeper leaf partition query. +/// +/// The object store factory must be able to produce an [`object_store::ObjectStore`] capable of reading +/// from the input URLs and writing to the output URL. +pub async fn query( + store_factory: &ObjectStoreFactory, + config: &LeafPartitionQueryConfig<'_>, +) -> Result { + let ops = SleeperOperations::new(&config.common); + info!("DataFusion compaction: {ops}"); + + todo!(); +} + +/// Creates the [`DataFrame`] for a leaf partition query. +/// +/// This reads the Parquet and configures the frame's plan +/// to sort, filter and aggregate as necessary +/// +/// # Errors +/// Each step of query may produce an error. Any are reported back to the caller. +async fn build_query_dataframe<'a>( + ops: &'a SleeperOperations<'a>, + store_factory: &ObjectStoreFactory, +) -> Result { + let sf = prepare_session_config(ops, store_factory).await?; + let ctx = ops.apply_to_context(SessionContext::new_with_config(sf), store_factory)?; + let mut frame = ops.create_initial_partitioned_read(&ctx).await?; + frame = ops.apply_user_filters(frame)?; + frame = ops.apply_general_sort(frame)?; + frame = ops.apply_aggregations(frame)?; + let sketcher = ops.create_sketcher(frame.schema()); + frame = sketcher.apply_sketch(frame)?; + + Ok(frame) +} + +/// Create the [`SessionConfig`] for a query. +async fn prepare_session_config<'a>( + ops: &'a SleeperOperations<'a>, + store_factory: &ObjectStoreFactory, +) -> Result { + let sf = ops + .apply_config(SessionConfig::new(), store_factory) + .await?; + Ok( + if let OperationOutput::File { + output_file, + opts: parquet_options, + } = &ops.config.output + { + // Create Parquet configuration object based on requested output + let configurer = ParquetWriterConfigurer { parquet_options }; + configurer.apply_parquet_config(sf) + } else { + sf + }, + ) +} diff --git a/rust/sleeper_core/src/lib.rs b/rust/sleeper_core/src/lib.rs index 133f280d474..82c0f8eb791 100644 --- a/rust/sleeper_core/src/lib.rs +++ b/rust/sleeper_core/src/lib.rs @@ -27,7 +27,7 @@ use aws_credential_types::Credentials; use color_eyre::eyre::{Result, bail}; use object_store::aws::AmazonS3Builder; use objectstore_ext::s3::{ObjectStoreFactory, config_for_s3_module, default_creds_store}; -use std::fmt::Formatter; +use std::fmt::{Display, Formatter}; use url::Url; mod datafusion; @@ -163,7 +163,7 @@ impl CommonConfig<'_> { } } -impl std::fmt::Display for CommonConfig<'_> { +impl Display for CommonConfig<'_> { fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { write!( f, From 76ec854d916fbe49fb449b6b4a1dd27584a08e4b Mon Sep 17 00:00:00 2001 From: m09526 Date: Tue, 12 Aug 2025 15:02:59 +0000 Subject: [PATCH 02/28] Query building --- rust/aggregator_udfs/src/nonnull.rs | 10 +- rust/objectstore_ext/src/s3.rs | 1 + rust/sleeper_core/src/datafusion.rs | 8 +- rust/sleeper_core/src/datafusion/compact.rs | 6 +- rust/sleeper_core/src/datafusion/config.rs | 2 +- .../datafusion/filter_aggregation_config.rs | 2 +- .../src/datafusion/leaf_partition_query.rs | 158 +++++++++++++----- rust/sleeper_core/src/datafusion/region.rs | 5 +- rust/sleeper_core/src/datafusion/sketch.rs | 4 +- rust/sleeper_core/src/datafusion/util.rs | 4 +- 10 files changed, 133 insertions(+), 67 deletions(-) diff --git a/rust/aggregator_udfs/src/nonnull.rs b/rust/aggregator_udfs/src/nonnull.rs index 22938ea6259..c56ad2f8865 100644 --- a/rust/aggregator_udfs/src/nonnull.rs +++ b/rust/aggregator_udfs/src/nonnull.rs @@ -28,14 +28,13 @@ use datafusion::{ sum::sum_udaf, }, logical_expr::{ - Accumulator, AggregateUDF, AggregateUDFImpl, Documentation, EmitTo, GroupsAccumulator, - ReversedUDAF, SetMonotonicity, Signature, StatisticsArgs, + Accumulator, AggregateUDF, AggregateUDFImpl, Documentation, EmitTo, Expr, + GroupsAccumulator, ReversedUDAF, SetMonotonicity, Signature, StatisticsArgs, expr::{AggregateFunction, AggregateFunctionParams}, function::{AccumulatorArgs, AggregateFunctionSimplification, StateFieldsArgs}, simplify::SimplifyInfo, utils::AggregateOrderSensitivity, }, - prelude::Expr, scalar::ScalarValue, }; use std::{ @@ -455,8 +454,8 @@ mod tests { sum::{sum, sum_udaf}, }, logical_expr::{ - Accumulator, AggregateUDFImpl, Documentation, EmitTo, GroupsAccumulator, ReversedUDAF, - SetMonotonicity, Signature, StatisticsArgs, Volatility, + Accumulator, AggregateUDFImpl, Documentation, EmitTo, Expr, GroupsAccumulator, + ReversedUDAF, SetMonotonicity, Signature, StatisticsArgs, Volatility, expr::{AggregateFunction, AggregateFunctionParams, WindowFunctionParams}, function::{AccumulatorArgs, AggregateFunctionSimplification, StateFieldsArgs}, lit, @@ -464,7 +463,6 @@ mod tests { utils::AggregateOrderSensitivity, }, physical_expr::LexOrdering, - prelude::Expr, scalar::ScalarValue, }; use mockall::predicate::*; diff --git a/rust/objectstore_ext/src/s3.rs b/rust/objectstore_ext/src/s3.rs index 542004e7387..1a83a73e0cd 100644 --- a/rust/objectstore_ext/src/s3.rs +++ b/rust/objectstore_ext/src/s3.rs @@ -107,6 +107,7 @@ fn extract_bucket(src: &Url) -> color_eyre::Result { /// Creates [`object_store::ObjectStore`] implementations from a URL and loads credentials into the S3 /// object store. +#[derive(Debug)] pub struct ObjectStoreFactory { s3_config: Option, store_map: RefCell>>, diff --git a/rust/sleeper_core/src/datafusion.rs b/rust/sleeper_core/src/datafusion.rs index cc30d1f9ebf..2660c9985c4 100644 --- a/rust/sleeper_core/src/datafusion.rs +++ b/rust/sleeper_core/src/datafusion.rs @@ -32,13 +32,13 @@ use aggregator_udfs::nonnull::register_non_nullable_aggregate_udfs; use arrow::compute::SortOptions; use datafusion::{ common::{DFSchema, plan_err}, + dataframe::DataFrame, datasource::file_format::{format_as_file_type, parquet::ParquetFormatFactory}, error::DataFusionError, execution::{config::SessionConfig, context::SessionContext, options::ParquetReadOptions}, - logical_expr::{LogicalPlanBuilder, SortExpr}, + logical_expr::{Expr, LogicalPlanBuilder, SortExpr, col}, physical_expr::{LexOrdering, PhysicalSortExpr}, physical_plan::{ExecutionPlan, expressions::Column}, - prelude::*, }; use log::{info, warn}; use objectstore_ext::s3::ObjectStoreFactory; @@ -107,7 +107,7 @@ impl<'a> SleeperOperations<'a> { } // Configure a [`SessionContext`]. - pub fn apply_to_context( + pub fn configure_context( &self, mut ctx: SessionContext, store_factory: &ObjectStoreFactory, @@ -156,7 +156,7 @@ impl<'a> SleeperOperations<'a> { .await?; // Do we have partition bounds? Ok( - if let Some(expr) = Into::>::into(&self.config.region) { + if let Some(expr) = Option::::from(&self.config.region) { frame.filter(expr)? } else { frame diff --git a/rust/sleeper_core/src/datafusion/compact.rs b/rust/sleeper_core/src/datafusion/compact.rs index e4ef65e7907..c9e40fb3303 100644 --- a/rust/sleeper_core/src/datafusion/compact.rs +++ b/rust/sleeper_core/src/datafusion/compact.rs @@ -26,9 +26,11 @@ use crate::{ }; use datafusion::{ common::plan_err, + dataframe::DataFrame, error::DataFusionError, + execution::config::SessionConfig, + execution::context::SessionContext, physical_plan::{collect, displayable}, - prelude::{DataFrame, SessionConfig, SessionContext}, }; use log::info; use objectstore_ext::s3::ObjectStoreFactory; @@ -90,7 +92,7 @@ async fn build_compaction_dataframe<'a>( .apply_config(SessionConfig::new(), store_factory) .await?; let sf = configurer.apply_parquet_config(sf); - let ctx = ops.apply_to_context(SessionContext::new_with_config(sf), store_factory)?; + let ctx = ops.configure_context(SessionContext::new_with_config(sf), store_factory)?; let mut frame = ops.create_initial_partitioned_read(&ctx).await?; frame = ops.apply_user_filters(frame)?; frame = ops.apply_general_sort(frame)?; diff --git a/rust/sleeper_core/src/datafusion/config.rs b/rust/sleeper_core/src/datafusion/config.rs index 7d4b14b3f30..bf85b3c8af7 100644 --- a/rust/sleeper_core/src/datafusion/config.rs +++ b/rust/sleeper_core/src/datafusion/config.rs @@ -20,7 +20,7 @@ use datafusion::{ common::DFSchema, config::TableParquetOptions, parquet::basic::{BrotliLevel, GzipLevel, ZstdLevel}, - prelude::SessionConfig, + execution::config::SessionConfig, }; use log::error; diff --git a/rust/sleeper_core/src/datafusion/filter_aggregation_config.rs b/rust/sleeper_core/src/datafusion/filter_aggregation_config.rs index 6c967736d3c..83b8cdbf487 100644 --- a/rust/sleeper_core/src/datafusion/filter_aggregation_config.rs +++ b/rust/sleeper_core/src/datafusion/filter_aggregation_config.rs @@ -19,9 +19,9 @@ use aggregator_udfs::{ }; use datafusion::{ common::{Column, DFSchema, HashSet, plan_datafusion_err, plan_err}, + dataframe::DataFrame, error::{DataFusionError, Result}, logical_expr::{AggregateUDF, Expr, ExprSchemable, ScalarUDF, col}, - prelude::DataFrame, }; use filter_udfs::ageoff::AgeOff; use regex::Regex; diff --git a/rust/sleeper_core/src/datafusion/leaf_partition_query.rs b/rust/sleeper_core/src/datafusion/leaf_partition_query.rs index 96e5c479919..e9f68f0e078 100644 --- a/rust/sleeper_core/src/datafusion/leaf_partition_query.rs +++ b/rust/sleeper_core/src/datafusion/leaf_partition_query.rs @@ -17,13 +17,17 @@ */ use crate::{ CommonConfig, OperationOutput, SleeperPartitionRegion, - datafusion::{ParquetWriterConfigurer, SleeperOperations}, + datafusion::{ + ParquetWriterConfigurer, SleeperOperations, sketch::Sketcher, util::explain_plan, + }, }; #[cfg(doc)] use arrow::record_batch::RecordBatch; +use datafusion::{common::plan_err, logical_expr::Expr}; use datafusion::{ + dataframe::DataFrame, error::DataFusionError, - prelude::{DataFrame, SessionConfig, SessionContext}, + execution::{config::SessionConfig, context::SessionContext}, }; use log::info; use objectstore_ext::s3::ObjectStoreFactory; @@ -52,64 +56,128 @@ impl Display for LeafPartitionQueryConfig<'_> { } } -/// Contains the query output result. -#[derive(Debug, Default)] -pub enum LeafQueryOutput { - /// Results are returned via an iterator of Arrow [`RecordBatch`]es. - #[default] - ArrowBatches, - /// Results have been written to a Parquet file. - ParquetFile, +/// Manages and executes a Sleeper leaf partition query. +#[derive(Debug)] +pub struct LeafPartitionQuery<'a> { + /// The configuration information for the leaf query + config: &'a LeafPartitionQueryConfig<'a>, + /// Used to create object store implementations + store_factory: &'a ObjectStoreFactory, } -/// Executes a Sleeper leaf partition query. -/// -/// The object store factory must be able to produce an [`object_store::ObjectStore`] capable of reading -/// from the input URLs and writing to the output URL. -pub async fn query( - store_factory: &ObjectStoreFactory, - config: &LeafPartitionQueryConfig<'_>, -) -> Result { - let ops = SleeperOperations::new(&config.common); - info!("DataFusion compaction: {ops}"); +impl<'a> LeafPartitionQuery<'a> { + pub fn new( + config: &'a LeafPartitionQueryConfig<'a>, + store_factory: &'a ObjectStoreFactory, + ) -> LeafPartitionQuery<'a> { + Self { + config, + store_factory, + } + } - todo!(); -} + /// Executes a Sleeper leaf partition query. + /// + /// The object store factory must be able to produce an [`object_store::ObjectStore`] capable of reading + /// from the input URLs and writing to the output URL (if writing results to a file). + pub async fn run_query(&self) -> Result<(), DataFusionError> { + let ops = SleeperOperations::new(&self.config.common); + info!("DataFusion compaction: {ops}"); + // Create query frame an sketches if it has been enabled + let (sketcher, frame) = self.build_query_dataframe(&ops).await?; + + if self.config.explain_plans { + explain_plan(&frame).await?; + } + todo!(); + } + + /// Adds a quantile sketch to a query plan if sketch generation is enabled. + /// + /// # Errors + /// If sketch output is requested, then file output must be chosen in the query config. + fn maybe_add_sketch_output( + &self, + ops: &'a SleeperOperations<'a>, + frame: DataFrame, + ) -> Result<(Option>, DataFrame), DataFusionError> { + if self.config.write_quantile_sketch { + match self.config.common.output { + OperationOutput::File { + output_file: _, + opts: _, + } => { + let sketcher = ops.create_sketcher(frame.schema()); + let frame = sketcher.apply_sketch(frame)?; + Ok((Some(sketcher), frame)) + } + OperationOutput::ArrowRecordBatch => plan_err!( + "Quantile sketch output cannot be enabled if file output not selected" + ), + } + } else { + Ok((None, frame)) + } + } -/// Creates the [`DataFrame`] for a leaf partition query. -/// -/// This reads the Parquet and configures the frame's plan -/// to sort, filter and aggregate as necessary -/// -/// # Errors -/// Each step of query may produce an error. Any are reported back to the caller. -async fn build_query_dataframe<'a>( - ops: &'a SleeperOperations<'a>, - store_factory: &ObjectStoreFactory, -) -> Result { - let sf = prepare_session_config(ops, store_factory).await?; - let ctx = ops.apply_to_context(SessionContext::new_with_config(sf), store_factory)?; - let mut frame = ops.create_initial_partitioned_read(&ctx).await?; - frame = ops.apply_user_filters(frame)?; - frame = ops.apply_general_sort(frame)?; - frame = ops.apply_aggregations(frame)?; - let sketcher = ops.create_sketcher(frame.schema()); - frame = sketcher.apply_sketch(frame)?; + /// Creates the [`DataFrame`] for a leaf partition query. + /// + /// This reads the Parquet and configures the frame's plan + /// to sort, filter and aggregate as necessary + /// + /// # Errors + /// Each step of query may produce an error. Any are reported back to the caller. + async fn build_query_dataframe( + &self, + ops: &'a SleeperOperations<'a>, + ) -> Result<(Option>, DataFrame), DataFusionError> { + let sf = prepare_session_config(ops, self.store_factory).await?; + let ctx = ops.configure_context(SessionContext::new_with_config(sf), self.store_factory)?; + let mut frame = ops.create_initial_partitioned_read(&ctx).await?; + frame = self.apply_query_regions(frame)?; + frame = ops.apply_user_filters(frame)?; + frame = ops.apply_general_sort(frame)?; + frame = ops.apply_aggregations(frame)?; + self.maybe_add_sketch_output(ops, frame) + } +} - Ok(frame) +impl LeafPartitionQuery<'_> { + /// Apply the query regions to the frame. + /// + /// The list of query regions are created and then OR'd together and + /// added to the [`DataFrame`], this will ultimately be AND'd with the + /// initial Sleeper partition region. + pub fn apply_query_regions(&self, frame: DataFrame) -> Result { + let mut query_expr: Option = None; + for region in &self.config.ranges { + if let Some(expr) = Option::::from(region) { + query_expr = match query_expr { + Some(original) => Some(original.or(expr)), + None => Some(expr), + } + } + } + // If we have any filters apply to frame (will AND with any previous filter) + Ok(if let Some(expr) = query_expr { + frame.filter(expr)? + } else { + frame + }) + } } /// Create the [`SessionConfig`] for a query. async fn prepare_session_config<'a>( - ops: &'a SleeperOperations<'a>, - store_factory: &ObjectStoreFactory, + ops: &SleeperOperations<'a>, + store_factory: &'a ObjectStoreFactory, ) -> Result { let sf = ops .apply_config(SessionConfig::new(), store_factory) .await?; Ok( if let OperationOutput::File { - output_file, + output_file: _, opts: parquet_options, } = &ops.config.output { diff --git a/rust/sleeper_core/src/datafusion/region.rs b/rust/sleeper_core/src/datafusion/region.rs index dc66a8ce25a..3bb9bf3ea07 100644 --- a/rust/sleeper_core/src/datafusion/region.rs +++ b/rust/sleeper_core/src/datafusion/region.rs @@ -15,10 +15,7 @@ * limitations under the License. */ use crate::{ColRange, PartitionBound}; -use datafusion::{ - logical_expr::{col, lit}, - prelude::Expr, -}; +use datafusion::logical_expr::{Expr, col, lit}; use log::error; use std::collections::HashMap; diff --git a/rust/sleeper_core/src/datafusion/sketch.rs b/rust/sleeper_core/src/datafusion/sketch.rs index b67f4f156fc..095877e92f8 100644 --- a/rust/sleeper_core/src/datafusion/sketch.rs +++ b/rust/sleeper_core/src/datafusion/sketch.rs @@ -22,10 +22,10 @@ use color_eyre::eyre::eyre; use cxx::{Exception, UniquePtr}; use datafusion::{ common::DFSchema, + dataframe::DataFrame, error::DataFusionError, logical_expr::{ScalarUDF, col}, parquet::data_type::AsBytes, - prelude::DataFrame, }; use log::info; use num_format::{Locale, ToFormattedString}; @@ -349,7 +349,7 @@ impl<'a> Sketcher<'a> { let column_bind = frame.schema().columns(); let col_names = column_bind .iter() - .map(datafusion::prelude::Column::name) + .map(datafusion::common::Column::name) .collect::>(); let row_key_exprs = self.row_keys.iter().map(col).collect::>(); diff --git a/rust/sleeper_core/src/datafusion/util.rs b/rust/sleeper_core/src/datafusion/util.rs index 52d8e918505..5da6c4af4d2 100644 --- a/rust/sleeper_core/src/datafusion/util.rs +++ b/rust/sleeper_core/src/datafusion/util.rs @@ -21,15 +21,15 @@ use datafusion::{ tree_node::{Transformed, TreeNode, TreeNodeRecursion}, }, config::ExecutionOptions, + dataframe::DataFrame, error::DataFusionError, - execution::SessionStateBuilder, + execution::{SessionStateBuilder, context::SessionContext}, physical_expr::LexOrdering, physical_plan::{ ExecutionPlan, accept, coalesce_partitions::CoalescePartitionsExec, sorts::{sort::SortExec, sort_preserving_merge::SortPreservingMergeExec}, }, - prelude::{DataFrame, SessionContext}, }; use log::info; use num_format::{Locale, ToFormattedString}; From 572284cc5cd66d6c01acd2de3073c21a4d3ed2f9 Mon Sep 17 00:00:00 2001 From: m09526 Date: Tue, 12 Aug 2025 16:00:00 +0000 Subject: [PATCH 03/28] WIP query output --- rust/sleeper_core/src/datafusion.rs | 1 + rust/sleeper_core/src/datafusion/metrics.rs | 1 + rust/sleeper_core/src/datafusion/output.rs | 61 +++++++++++++++++++++ rust/sleeper_core/src/lib.rs | 15 +++++ 4 files changed, 78 insertions(+) create mode 100644 rust/sleeper_core/src/datafusion/output.rs diff --git a/rust/sleeper_core/src/datafusion.rs b/rust/sleeper_core/src/datafusion.rs index 2660c9985c4..9c290544813 100644 --- a/rust/sleeper_core/src/datafusion.rs +++ b/rust/sleeper_core/src/datafusion.rs @@ -49,6 +49,7 @@ mod config; mod filter_aggregation_config; mod leaf_partition_query; mod metrics; +pub mod output; mod region; pub mod sketch; mod sketch_udf; diff --git a/rust/sleeper_core/src/datafusion/metrics.rs b/rust/sleeper_core/src/datafusion/metrics.rs index 37548e554eb..98c03d26459 100644 --- a/rust/sleeper_core/src/datafusion/metrics.rs +++ b/rust/sleeper_core/src/datafusion/metrics.rs @@ -45,6 +45,7 @@ pub const FILE_METRICS: [&str; 10] = [ ]; /// Simple struct used for storing the collected statistics from an execution plan. +#[derive(Debug)] pub struct RowCounts { pub rows_read: usize, pub rows_written: usize, diff --git a/rust/sleeper_core/src/datafusion/output.rs b/rust/sleeper_core/src/datafusion/output.rs new file mode 100644 index 00000000000..adbf5417193 --- /dev/null +++ b/rust/sleeper_core/src/datafusion/output.rs @@ -0,0 +1,61 @@ +//! Structs and trait to handle output of Sleeper data processing. +/* +* Copyright 2022-2025 Crown Copyright +* +* Licensed under the Apache License, Version 2.0 (the "License"); +* you may not use this file except in compliance with the License. +* You may obtain a copy of the License at +* +* http://www.apache.org/licenses/LICENSE-2.0 +* +* Unless required by applicable law or agreed to in writing, software +* distributed under the License is distributed on an "AS IS" BASIS, +* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +* See the License for the specific language governing permissions and +* limitations under the License. +*/ +use crate::datafusion::{SleeperOperations, metrics::RowCounts}; +use datafusion::{ + error::DataFusionError, execution::SendableRecordBatchStream, prelude::DataFrame, +}; +use std::fmt::{Debug, Formatter}; + +pub enum CompletedOutput { + ArrowRecordBatch(SendableRecordBatchStream), + File(RowCounts), +} + +impl Debug for CompletedOutput { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + match self { + Self::ArrowRecordBatch(_) => write!(f, "CompletedOutput::ArrowRecordBatch"), + Self::File(_) => write!(f, "CompletedOutput::File"), + } + } +} + +pub trait Completer { + fn complete( + ops: &SleeperOperations<'_>, + frame: DataFrame, + ) -> Result; +} + +// #[derive(Debug)] +// pub struct RecordBatchCompleter {} + +// impl Completer for RecordBatchCompleter {} + +#[derive(Debug)] +pub struct FileOutputCompleter {} + +impl FileOutputCompleter {} + +impl Completer for FileOutputCompleter { + fn complete( + ops: &SleeperOperations<'_>, + frame: DataFrame, + ) -> Result { + todo!() + } +} diff --git a/rust/sleeper_core/src/lib.rs b/rust/sleeper_core/src/lib.rs index 82c0f8eb791..56c8de14ef6 100644 --- a/rust/sleeper_core/src/lib.rs +++ b/rust/sleeper_core/src/lib.rs @@ -37,6 +37,7 @@ pub use datafusion::{ sketch::{DataSketchVariant, deserialise_sketches}, }; + /// Type safe variant for Sleeper partition boundary #[derive(Debug, Copy, Clone)] pub enum PartitionBound<'a> { @@ -196,6 +197,20 @@ pub enum OperationOutput { }, } +// impl OperationOutput { +// /// Create a [`Completer`] for this type of output. +// pub fn finisher(&self) -> Arc { +// Arc::new(match self { +// Self::ArrowRecordBatch => { +// unimplemented!() +// } +// Self::File { output_file, opts } => { +// unimplemented!() +// } +// }) +// } +// } + #[derive(Debug)] pub struct AwsConfig { pub region: String, From c7669b2afd6ab2642ad97a84df240c0edb65eb77 Mon Sep 17 00:00:00 2001 From: m09526 Date: Wed, 13 Aug 2025 09:28:29 +0000 Subject: [PATCH 04/28] Fixed --- rust/compactor/src/bin/main.rs | 21 ++- rust/sleeper_core/src/datafusion.rs | 3 +- rust/sleeper_core/src/datafusion/config.rs | 2 +- rust/sleeper_core/src/datafusion/output.rs | 35 ++++- rust/sleeper_core/src/lib.rs | 153 ++++++++++++--------- rust/sleeper_df/src/lib.rs | 17 ++- 6 files changed, 144 insertions(+), 87 deletions(-) diff --git a/rust/compactor/src/bin/main.rs b/rust/compactor/src/bin/main.rs index f8a951e47b6..95ffcf266b5 100644 --- a/rust/compactor/src/bin/main.rs +++ b/rust/compactor/src/bin/main.rs @@ -149,20 +149,19 @@ async fn main() -> color_eyre::Result<()> { dict_enc_values: true, }; - let mut details = CommonConfig { - aws_config: None, - input_files: input_urls, - input_files_sorted: true, - row_key_cols: args.row_keys, - sort_key_cols: args.sort_keys, - region: SleeperPartitionRegion::new(map), - output: OperationOutput::File { + let details = CommonConfig::try_new( + None, + input_urls, + true, + args.row_keys, + args.sort_keys, + SleeperPartitionRegion::new(map), + OperationOutput::File { output_file, opts: parquet_options, }, - iterator_config: args.iterator_config, - }; - details.sanitise_java_s3_urls(); + args.iterator_config, + )?; let result = run_compaction(&details).await; match result { diff --git a/rust/sleeper_core/src/datafusion.rs b/rust/sleeper_core/src/datafusion.rs index 9c290544813..5ccef704ab1 100644 --- a/rust/sleeper_core/src/datafusion.rs +++ b/rust/sleeper_core/src/datafusion.rs @@ -18,7 +18,7 @@ * limitations under the License. */ use crate::{ - CommonConfig, OperationOutput, + CommonConfig, datafusion::{ filter_aggregation_config::{FilterAggregationConfig, validate_aggregations}, sketch::Sketcher, @@ -57,6 +57,7 @@ mod util; pub use compact::compact; pub use config::ParquetWriterConfigurer; +pub use output::OperationOutput; pub use region::SleeperPartitionRegion; /// Drives common operations in processing of `DataFusion` for Sleeper. diff --git a/rust/sleeper_core/src/datafusion/config.rs b/rust/sleeper_core/src/datafusion/config.rs index bf85b3c8af7..10f618e07e8 100644 --- a/rust/sleeper_core/src/datafusion/config.rs +++ b/rust/sleeper_core/src/datafusion/config.rs @@ -19,8 +19,8 @@ use crate::{CommonConfig, SleeperParquetOptions}; use datafusion::{ common::DFSchema, config::TableParquetOptions, - parquet::basic::{BrotliLevel, GzipLevel, ZstdLevel}, execution::config::SessionConfig, + parquet::basic::{BrotliLevel, GzipLevel, ZstdLevel}, }; use log::error; diff --git a/rust/sleeper_core/src/datafusion/output.rs b/rust/sleeper_core/src/datafusion/output.rs index adbf5417193..a41328d8c4a 100644 --- a/rust/sleeper_core/src/datafusion/output.rs +++ b/rust/sleeper_core/src/datafusion/output.rs @@ -14,11 +14,44 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -use crate::datafusion::{SleeperOperations, metrics::RowCounts}; +use crate::{ + SleeperParquetOptions, + datafusion::{SleeperOperations, metrics::RowCounts}, +}; use datafusion::{ error::DataFusionError, execution::SendableRecordBatchStream, prelude::DataFrame, }; use std::fmt::{Debug, Formatter}; +use url::Url; + +/// Defines how operation output should be given. +#[derive(Debug, Default)] +pub enum OperationOutput { + /// `DataFusion` results will be returned as a stream of Arrow [`RecordBatch`]es. + #[default] + ArrowRecordBatch, + /// `DataFusion` results will be written to a file with given Parquet options. + File { + /// Output file Url + output_file: Url, + /// Parquet output options + opts: SleeperParquetOptions, + }, +} + +// impl OperationOutput { +// /// Create a [`Completer`] for this type of output. +// pub fn finisher(&self) -> Arc { +// Arc::new(match self { +// Self::ArrowRecordBatch => { +// unimplemented!() +// } +// Self::File { output_file, opts } => { +// unimplemented!() +// } +// }) +// } +// } pub enum CompletedOutput { ArrowRecordBatch(SendableRecordBatchStream), diff --git a/rust/sleeper_core/src/lib.rs b/rust/sleeper_core/src/lib.rs index 56c8de14ef6..a44fc554acc 100644 --- a/rust/sleeper_core/src/lib.rs +++ b/rust/sleeper_core/src/lib.rs @@ -33,11 +33,10 @@ use url::Url; mod datafusion; pub use datafusion::{ - SleeperPartitionRegion, + OperationOutput, SleeperPartitionRegion, sketch::{DataSketchVariant, deserialise_sketches}, }; - /// Type safe variant for Sleeper partition boundary #[derive(Debug, Copy, Clone)] pub enum PartitionBound<'a> { @@ -90,7 +89,7 @@ impl Default for SleeperParquetOptions { /// Common items necessary to perform any `DataFusion` related /// work for Sleeper. -#[derive(Debug, Default)] +#[derive(Debug)] pub struct CommonConfig<'a> { /// Aws credentials configuration pub aws_config: Option, @@ -110,45 +109,101 @@ pub struct CommonConfig<'a> { pub iterator_config: Option, } -impl CommonConfig<'_> { - /// Convert all Java "s3a" URLs in input and output to "s3." - pub fn sanitise_java_s3_urls(&mut self) { - self.input_files.iter_mut().for_each(|t| { - if t.scheme() == "s3a" { - let _ = t.set_scheme("s3"); - } - }); - - if let OperationOutput::File { - output_file, - opts: _, - } = &mut self.output - && output_file.scheme() == "s3a" - { - let _ = output_file.set_scheme("s3"); +impl Default for CommonConfig<'_> { + fn default() -> Self { + Self { + aws_config: Option::default(), + input_files: Vec::default(), + input_files_sorted: true, + row_key_cols: Vec::default(), + sort_key_cols: Vec::default(), + region: SleeperPartitionRegion::default(), + output: OperationOutput::default(), + iterator_config: Option::default(), } } +} - /// Checks for simple configuration errors +impl<'a> CommonConfig<'a> { + /// Creates a new configuration object. /// /// # Errors - /// It is an error for input paths to be empty or for a length - /// mismatch between row key columns length and number of ranges in - /// partition region. - pub fn validate(&self) -> Result<()> { - if self.input_files.is_empty() { - bail!("No input paths supplied"); - } - if self.row_key_cols.len() != self.region.len() { - bail!( - "Length mismatch between row keys {} and partition region bounds {}", - self.row_key_cols.len(), - self.region.len() - ); + /// The configuration must validate. Input files mustn't be empty + /// and the number of row key columns must match the number of region + /// dimensions. + #[allow(clippy::too_many_arguments)] + pub fn try_new( + aws_config: Option, + input_files: Vec, + input_files_sorted: bool, + row_key_cols: Vec, + sort_key_cols: Vec, + region: SleeperPartitionRegion<'a>, + output: OperationOutput, + iterator_config: Option, + ) -> Result { + validate(&input_files, &row_key_cols, ®ion)?; + // Convert Java s3a schema to s3 + let (input_files, output) = normalise_s3a_urls(input_files, output); + Ok(Self { + aws_config, + input_files, + input_files_sorted, + row_key_cols, + sort_key_cols, + region, + output, + iterator_config, + }) + } +} + +/// Change all input and output URLS from s3a to s3 scheme. +fn normalise_s3a_urls( + mut input_files: Vec, + mut output: OperationOutput, +) -> (Vec, OperationOutput) { + for t in &mut input_files { + if t.scheme() == "s3a" { + let _ = t.set_scheme("s3"); } - Ok(()) } + if let OperationOutput::File { + output_file, + opts: _, + } = &mut output + && output_file.scheme() == "s3a" + { + let _ = output_file.set_scheme("s3"); + } + (input_files, output) +} + +/// Performs validity checks on parameters. +/// +/// # Errors +/// There must be at least one input file. +/// The length of `row_key_cols` must match the number of region dimensions. +fn validate( + input_files: &[Url], + row_key_cols: &[String], + region: &SleeperPartitionRegion<'_>, +) -> Result<()> { + if input_files.is_empty() { + bail!("No input paths supplied"); + } + if row_key_cols.len() != region.len() { + bail!( + "Length mismatch between row keys {} and partition region bounds {}", + row_key_cols.len(), + region.len() + ); + } + Ok(()) +} + +impl CommonConfig<'_> { /// Get iterator for row and sort key columns in order pub fn sorting_columns_iter(&self) -> impl Iterator { self.row_key_cols @@ -182,35 +237,6 @@ impl Display for CommonConfig<'_> { } } -/// Defines how operation output should be given. -#[derive(Debug, Default)] -pub enum OperationOutput { - /// `DataFusion` results will be returned as a stream of Arrow [`RecordBatch`]es. - #[default] - ArrowRecordBatch, - /// `DataFusion` results will be written to a file with given Parquet options. - File { - /// Output file Url - output_file: Url, - /// Parquet output options - opts: SleeperParquetOptions, - }, -} - -// impl OperationOutput { -// /// Create a [`Completer`] for this type of output. -// pub fn finisher(&self) -> Arc { -// Arc::new(match self { -// Self::ArrowRecordBatch => { -// unimplemented!() -// } -// Self::File { output_file, opts } => { -// unimplemented!() -// } -// }) -// } -// } - #[derive(Debug)] pub struct AwsConfig { pub region: String, @@ -268,7 +294,6 @@ pub struct CompactionResult { /// There must be at least one input file. /// pub async fn run_compaction(config: &CommonConfig<'_>) -> Result { - config.validate()?; let store_factory = create_object_store_factory(config.aws_config.as_ref()).await; crate::datafusion::compact(&store_factory, config) diff --git a/rust/sleeper_df/src/lib.rs b/rust/sleeper_df/src/lib.rs index 3613950d895..93ba0f5a037 100644 --- a/rust/sleeper_df/src/lib.rs +++ b/rust/sleeper_df/src/lib.rs @@ -153,27 +153,27 @@ impl<'a> TryFrom<&'a FFICompactionParams> for CommonConfig<'a> { dict_enc_values: params.dict_enc_values, }; - Ok(Self { - aws_config: unpack_aws_config(params)?, - input_files: unpack_string_array(params.input_files, params.input_files_len)? + Self::try_new( + unpack_aws_config(params)?, + unpack_string_array(params.input_files, params.input_files_len)? .into_iter() .map(Url::parse) .collect::, _>>()?, - input_files_sorted: true, + true, row_key_cols, - sort_key_cols: unpack_string_array(params.sort_key_cols, params.sort_key_cols_len)? + unpack_string_array(params.sort_key_cols, params.sort_key_cols_len)? .into_iter() .map(String::from) .collect(), region, - output: OperationOutput::File { + OperationOutput::File { output_file: unsafe { CStr::from_ptr(params.output_file) } .to_str() .map(Url::parse)??, opts, }, iterator_config, - }) + ) } } @@ -317,7 +317,7 @@ pub extern "C" fn merge_sorted_files( } }; - let mut details = match TryInto::::try_into(params) { + let details = match TryInto::::try_into(params) { Ok(d) => d, Err(e) => { error!("Couldn't convert compaction input data {e}"); @@ -326,7 +326,6 @@ pub extern "C" fn merge_sorted_files( }; // Run compaction - details.sanitise_java_s3_urls(); let result = rt.block_on(run_compaction(&details)); match result { Ok(res) => { From bb0a2af9ecdc4d2440248dbcda99e41d34876909 Mon Sep 17 00:00:00 2001 From: m09526 Date: Wed, 13 Aug 2025 12:28:29 +0000 Subject: [PATCH 05/28] OutputOptions compiles --- rust/Cargo.lock | 1 + rust/compactor/src/bin/main.rs | 4 +- rust/sleeper_core/Cargo.toml | 1 + rust/sleeper_core/src/datafusion.rs | 10 +- rust/sleeper_core/src/datafusion/compact.rs | 16 ++- .../src/datafusion/leaf_partition_query.rs | 8 +- rust/sleeper_core/src/datafusion/output.rs | 124 +++++++++++++----- rust/sleeper_core/src/lib.rs | 18 +-- rust/sleeper_core/tests/compaction_test.rs | 12 +- rust/sleeper_df/src/lib.rs | 4 +- 10 files changed, 131 insertions(+), 67 deletions(-) diff --git a/rust/Cargo.lock b/rust/Cargo.lock index 3017732559d..a159e4d10bf 100644 --- a/rust/Cargo.lock +++ b/rust/Cargo.lock @@ -4237,6 +4237,7 @@ version = "0.33.0-SNAPSHOT" dependencies = [ "aggregator_udfs", "arrow", + "async-trait", "aws-config", "aws-credential-types", "aws-types", diff --git a/rust/compactor/src/bin/main.rs b/rust/compactor/src/bin/main.rs index 95ffcf266b5..0350163601c 100644 --- a/rust/compactor/src/bin/main.rs +++ b/rust/compactor/src/bin/main.rs @@ -20,7 +20,7 @@ use human_panic::setup_panic; use log::info; use num_format::{Locale, ToFormattedString}; use sleeper_core::{ - ColRange, CommonConfig, OperationOutput, PartitionBound, SleeperParquetOptions, + ColRange, CommonConfig, CompletionOptions, PartitionBound, SleeperParquetOptions, SleeperPartitionRegion, run_compaction, }; use std::{collections::HashMap, io::Write, path::Path}; @@ -156,7 +156,7 @@ async fn main() -> color_eyre::Result<()> { args.row_keys, args.sort_keys, SleeperPartitionRegion::new(map), - OperationOutput::File { + CompletionOptions::File { output_file, opts: parquet_options, }, diff --git a/rust/sleeper_core/Cargo.toml b/rust/sleeper_core/Cargo.toml index 91bd9c1813d..cceaad8d8c5 100644 --- a/rust/sleeper_core/Cargo.toml +++ b/rust/sleeper_core/Cargo.toml @@ -27,6 +27,7 @@ publish = { workspace = true } [dependencies] arrow = { workspace = true, features = ["prettyprint"] } aggregator_udfs = { path = "../aggregator_udfs" } +async-trait = { workspace = true } aws-config = { workspace = true } aws-credential-types = { workspace = true, features = ["hardcoded-credentials"] } aws-types = { workspace = true } diff --git a/rust/sleeper_core/src/datafusion.rs b/rust/sleeper_core/src/datafusion.rs index 5ccef704ab1..af8d02ab702 100644 --- a/rust/sleeper_core/src/datafusion.rs +++ b/rust/sleeper_core/src/datafusion.rs @@ -57,7 +57,7 @@ mod util; pub use compact::compact; pub use config::ParquetWriterConfigurer; -pub use output::OperationOutput; +pub use output::CompletionOptions; pub use region::SleeperPartitionRegion; /// Drives common operations in processing of `DataFusion` for Sleeper. @@ -94,7 +94,7 @@ impl<'a> SleeperOperations<'a> { // together in wrong order. cfg.options_mut().optimizer.repartition_aggregations = false; // Set upload size if outputting to a file - if let OperationOutput::File { + if let CompletionOptions::File { output_file: _, opts: _, } = self.config.output @@ -120,8 +120,8 @@ impl<'a> SleeperOperations<'a> { store_factory, &self.config.input_files, match &self.config.output { - OperationOutput::ArrowRecordBatch => None, - OperationOutput::File { + CompletionOptions::ArrowRecordBatch => None, + CompletionOptions::File { output_file, opts: _, } => Some(output_file), @@ -270,7 +270,7 @@ impl<'a> SleeperOperations<'a> { frame: DataFrame, configurer: &ParquetWriterConfigurer<'_>, ) -> Result { - let OperationOutput::File { + let CompletionOptions::File { output_file, opts: _, } = &self.config.output diff --git a/rust/sleeper_core/src/datafusion/compact.rs b/rust/sleeper_core/src/datafusion/compact.rs index c9e40fb3303..4cba884e065 100644 --- a/rust/sleeper_core/src/datafusion/compact.rs +++ b/rust/sleeper_core/src/datafusion/compact.rs @@ -16,10 +16,11 @@ * limitations under the License. */ use crate::{ - CommonConfig, CompactionResult, OperationOutput, + CommonConfig, CompactionResult, CompletionOptions, datafusion::{ ParquetWriterConfigurer, SleeperOperations, metrics::RowCounts, + output::Completer, sketch::{Sketcher, output_sketch}, util::{collect_stats, explain_plan}, }, @@ -49,7 +50,7 @@ pub async fn compact( info!("DataFusion compaction: {ops}"); // Retrieve Parquet output options - let OperationOutput::File { + let CompletionOptions::File { output_file, opts: parquet_options, } = &config.output @@ -57,11 +58,12 @@ pub async fn compact( return plan_err!("Sleeper compactions must output to a file"); }; - // Create Parquet configuration object based on requested output let configurer = ParquetWriterConfigurer { parquet_options }; + let completer = config.output.finisher(&ops); // Make compaction DataFrame - let (sketcher, frame) = build_compaction_dataframe(&ops, &configurer, store_factory).await?; + let (sketcher, frame) = + build_compaction_dataframe(&ops, &completer, &configurer, store_factory).await?; // Explain logical plan explain_plan(&frame).await?; @@ -85,7 +87,8 @@ pub async fn compact( /// Each step of compaction may produce an error. Any are reported back to the caller. async fn build_compaction_dataframe<'a>( ops: &'a SleeperOperations<'a>, - configurer: &'a ParquetWriterConfigurer<'a>, + completer: &Arc + 'a>, + configurer: &ParquetWriterConfigurer<'a>, store_factory: &ObjectStoreFactory, ) -> Result<(Sketcher<'a>, DataFrame), DataFusionError> { let sf = ops @@ -99,7 +102,8 @@ async fn build_compaction_dataframe<'a>( frame = ops.apply_aggregations(frame)?; let sketcher = ops.create_sketcher(frame.schema()); frame = sketcher.apply_sketch(frame)?; - frame = ops.plan_with_parquet_output(frame, configurer)?; + frame = completer.complete_frame(frame)?; + // frame = ops.plan_with_parquet_output(frame, configurer)?; Ok((sketcher, frame)) } diff --git a/rust/sleeper_core/src/datafusion/leaf_partition_query.rs b/rust/sleeper_core/src/datafusion/leaf_partition_query.rs index e9f68f0e078..5c8804837f5 100644 --- a/rust/sleeper_core/src/datafusion/leaf_partition_query.rs +++ b/rust/sleeper_core/src/datafusion/leaf_partition_query.rs @@ -16,7 +16,7 @@ * limitations under the License. */ use crate::{ - CommonConfig, OperationOutput, SleeperPartitionRegion, + CommonConfig, CompletionOptions, SleeperPartitionRegion, datafusion::{ ParquetWriterConfigurer, SleeperOperations, sketch::Sketcher, util::explain_plan, }, @@ -103,7 +103,7 @@ impl<'a> LeafPartitionQuery<'a> { ) -> Result<(Option>, DataFrame), DataFusionError> { if self.config.write_quantile_sketch { match self.config.common.output { - OperationOutput::File { + CompletionOptions::File { output_file: _, opts: _, } => { @@ -111,7 +111,7 @@ impl<'a> LeafPartitionQuery<'a> { let frame = sketcher.apply_sketch(frame)?; Ok((Some(sketcher), frame)) } - OperationOutput::ArrowRecordBatch => plan_err!( + CompletionOptions::ArrowRecordBatch => plan_err!( "Quantile sketch output cannot be enabled if file output not selected" ), } @@ -176,7 +176,7 @@ async fn prepare_session_config<'a>( .apply_config(SessionConfig::new(), store_factory) .await?; Ok( - if let OperationOutput::File { + if let CompletionOptions::File { output_file: _, opts: parquet_options, } = &ops.config.output diff --git a/rust/sleeper_core/src/datafusion/output.rs b/rust/sleeper_core/src/datafusion/output.rs index a41328d8c4a..37397d6b205 100644 --- a/rust/sleeper_core/src/datafusion/output.rs +++ b/rust/sleeper_core/src/datafusion/output.rs @@ -16,17 +16,29 @@ */ use crate::{ SleeperParquetOptions, - datafusion::{SleeperOperations, metrics::RowCounts}, + datafusion::{ + ParquetWriterConfigurer, SleeperOperations, metrics::RowCounts, util::collect_stats, + }, }; +#[cfg(doc)] +use arrow::record_batch::RecordBatch; +use async_trait::async_trait; use datafusion::{ - error::DataFusionError, execution::SendableRecordBatchStream, prelude::DataFrame, + common::plan_err, + error::DataFusionError, + execution::{SendableRecordBatchStream, TaskContext}, + physical_plan::{ExecutionPlan, collect}, + prelude::DataFrame, +}; +use std::{ + fmt::{Debug, Formatter}, + sync::Arc, }; -use std::fmt::{Debug, Formatter}; use url::Url; /// Defines how operation output should be given. #[derive(Debug, Default)] -pub enum OperationOutput { +pub enum CompletionOptions { /// `DataFusion` results will be returned as a stream of Arrow [`RecordBatch`]es. #[default] ArrowRecordBatch, @@ -39,22 +51,31 @@ pub enum OperationOutput { }, } -// impl OperationOutput { -// /// Create a [`Completer`] for this type of output. -// pub fn finisher(&self) -> Arc { -// Arc::new(match self { -// Self::ArrowRecordBatch => { -// unimplemented!() -// } -// Self::File { output_file, opts } => { -// unimplemented!() -// } -// }) -// } -// } +impl CompletionOptions { + /// Create a [`Completer`] for this type of output. + pub fn finisher<'a>(&self, ops: &'a SleeperOperations<'a>) -> Arc + 'a> { + Arc::new(match self { + Self::ArrowRecordBatch => { + unimplemented!() + } + Self::File { + output_file: _, + opts: _, + } => FileOutputCompleter::new(ops), + }) + } +} +/// The result of executing a [`DataFrame`] with a [`Completer`]. +/// +/// A completer will return a different variant depending on the +/// [`CompletionOptions`] the plan was configured with. pub enum CompletedOutput { + /// Results of plan are returned as a asynchronous stream + /// of Arrow [`RecordBatch`]es. ArrowRecordBatch(SendableRecordBatchStream), + /// Results of plan have been written to a file(s) and + /// the row counts are returned. File(RowCounts), } @@ -67,28 +88,65 @@ impl Debug for CompletedOutput { } } -pub trait Completer { - fn complete( - ops: &SleeperOperations<'_>, - frame: DataFrame, +/// A `Completer` object governs how the final stages of a Sleeper operation +/// is finished. Sometimes we want to output the results to file(s) and sometimes +/// we need a stream of processed records back. +#[async_trait] +pub trait Completer<'a> { + /// Modify the given [`DataFrame`] as necessary for the desired output. + /// + /// # Errors + /// An error will occur if the frame cannot be modified or if this [`Completer`] + /// is not suitable for use with the configured `ops`. + fn complete_frame(&self, frame: DataFrame) -> Result; + /// Runs the plan. + /// + /// # Errors + /// If any part of the conversion to physical plan or execution fails, then the error + /// is returned. + async fn execute_frame( + &self, + physical_plan: Arc, + task_ctx: Arc, ) -> Result; } -// #[derive(Debug)] -// pub struct RecordBatchCompleter {} - -// impl Completer for RecordBatchCompleter {} - +/// Writes output of frames to a Parquet file. #[derive(Debug)] -pub struct FileOutputCompleter {} +pub struct FileOutputCompleter<'a> { + ops: &'a SleeperOperations<'a>, +} + +impl<'a> FileOutputCompleter<'a> { + pub fn new(ops: &'a SleeperOperations<'a>) -> Self { + Self { ops } + } +} -impl FileOutputCompleter {} +#[async_trait] +impl<'a> Completer<'a> for FileOutputCompleter<'a> { + fn complete_frame(&self, frame: DataFrame) -> Result { + match &self.ops.config.output { + CompletionOptions::File { + output_file: _, + opts: parquet_options, + } => { + let configurer = ParquetWriterConfigurer { parquet_options }; + self.ops.plan_with_parquet_output(frame, &configurer) + } + CompletionOptions::ArrowRecordBatch => { + plan_err!("Can't use FileOutputCompleter with CompletionOptions::ArrowRecordBatch") + } + } + } -impl Completer for FileOutputCompleter { - fn complete( - ops: &SleeperOperations<'_>, - frame: DataFrame, + async fn execute_frame( + &self, + physical_plan: Arc, + task_ctx: Arc, ) -> Result { - todo!() + collect(physical_plan.clone(), task_ctx).await?; + let stats = collect_stats(&self.ops.config.input_files, &physical_plan)?; + Ok(CompletedOutput::File(stats)) } } diff --git a/rust/sleeper_core/src/lib.rs b/rust/sleeper_core/src/lib.rs index a44fc554acc..a10963673b3 100644 --- a/rust/sleeper_core/src/lib.rs +++ b/rust/sleeper_core/src/lib.rs @@ -33,7 +33,7 @@ use url::Url; mod datafusion; pub use datafusion::{ - OperationOutput, SleeperPartitionRegion, + CompletionOptions, SleeperPartitionRegion, sketch::{DataSketchVariant, deserialise_sketches}, }; @@ -104,7 +104,7 @@ pub struct CommonConfig<'a> { /// Ranges for each column to filter input files pub region: SleeperPartitionRegion<'a>, /// How output from operation should be returned - pub output: OperationOutput, + pub output: CompletionOptions, /// Iterator config. Filters, aggregators, etc. pub iterator_config: Option, } @@ -118,7 +118,7 @@ impl Default for CommonConfig<'_> { row_key_cols: Vec::default(), sort_key_cols: Vec::default(), region: SleeperPartitionRegion::default(), - output: OperationOutput::default(), + output: CompletionOptions::default(), iterator_config: Option::default(), } } @@ -139,7 +139,7 @@ impl<'a> CommonConfig<'a> { row_key_cols: Vec, sort_key_cols: Vec, region: SleeperPartitionRegion<'a>, - output: OperationOutput, + output: CompletionOptions, iterator_config: Option, ) -> Result { validate(&input_files, &row_key_cols, ®ion)?; @@ -161,15 +161,15 @@ impl<'a> CommonConfig<'a> { /// Change all input and output URLS from s3a to s3 scheme. fn normalise_s3a_urls( mut input_files: Vec, - mut output: OperationOutput, -) -> (Vec, OperationOutput) { + mut output: CompletionOptions, +) -> (Vec, CompletionOptions) { for t in &mut input_files { if t.scheme() == "s3a" { let _ = t.set_scheme("s3"); } } - if let OperationOutput::File { + if let CompletionOptions::File { output_file, opts: _, } = &mut output @@ -228,8 +228,8 @@ impl Display for CommonConfig<'_> { self.region )?; match &self.output { - OperationOutput::ArrowRecordBatch => write!(f, " output is Arrow RecordBatches"), - OperationOutput::File { + CompletionOptions::ArrowRecordBatch => write!(f, " output is Arrow RecordBatches"), + CompletionOptions::File { output_file, opts: _, } => write!(f, "output file {output_file:?}"), diff --git a/rust/sleeper_core/tests/compaction_test.rs b/rust/sleeper_core/tests/compaction_test.rs index be3b61ee5de..5cd38e94b4c 100644 --- a/rust/sleeper_core/tests/compaction_test.rs +++ b/rust/sleeper_core/tests/compaction_test.rs @@ -19,7 +19,7 @@ use arrow::datatypes::{DataType, Field, Schema}; use color_eyre::eyre::Error; use compaction_helpers::*; use sleeper_core::{ - CommonConfig, OperationOutput, SleeperParquetOptions, SleeperPartitionRegion, run_compaction, + CommonConfig, CompletionOptions, SleeperParquetOptions, SleeperPartitionRegion, run_compaction, }; use std::{collections::HashMap, sync::Arc}; use tempfile::tempdir; @@ -41,7 +41,7 @@ async fn should_merge_two_files() -> Result<(), Error> { input_files_sorted: true, row_key_cols: row_key_cols(["key"]), region: SleeperPartitionRegion::new(single_int_range("key", 0, 5)), - output: OperationOutput::File { + output: CompletionOptions::File { output_file: output.clone(), opts: SleeperParquetOptions::default(), }, @@ -74,7 +74,7 @@ async fn should_merge_files_with_overlapping_data() -> Result<(), Error> { input_files_sorted: true, row_key_cols: row_key_cols(["key"]), region: SleeperPartitionRegion::new(single_int_range("key", 0, 5)), - output: OperationOutput::File { + output: CompletionOptions::File { output_file: output.clone(), opts: SleeperParquetOptions::default(), }, @@ -107,7 +107,7 @@ async fn should_exclude_data_not_in_region() -> Result<(), Error> { input_files_sorted: true, row_key_cols: row_key_cols(["key"]), region: SleeperPartitionRegion::new(single_int_range("key", 2, 4)), - output: OperationOutput::File { + output: CompletionOptions::File { output_file: output.clone(), opts: SleeperParquetOptions::default(), }, @@ -149,7 +149,7 @@ async fn should_exclude_data_not_in_multidimensional_region() -> Result<(), Erro region_entry("key1", int_range(2, 4)), region_entry("key2", int_range(13, 23)), ])), - output: OperationOutput::File { + output: CompletionOptions::File { output_file: output.clone(), opts: SleeperParquetOptions::default(), }, @@ -194,7 +194,7 @@ async fn should_compact_with_second_column_row_key() -> Result<(), Error> { "key2", int_range(11, 25), )])), - output: OperationOutput::File { + output: CompletionOptions::File { output_file: output.clone(), opts: SleeperParquetOptions::default(), }, diff --git a/rust/sleeper_df/src/lib.rs b/rust/sleeper_df/src/lib.rs index 93ba0f5a037..f675d844a9a 100644 --- a/rust/sleeper_df/src/lib.rs +++ b/rust/sleeper_df/src/lib.rs @@ -19,7 +19,7 @@ use color_eyre::eyre::{bail, eyre}; use libc::{EFAULT, EINVAL, EIO, size_t}; use log::{LevelFilter, error, warn}; use sleeper_core::{ - AwsConfig, ColRange, CommonConfig, OperationOutput, PartitionBound, SleeperParquetOptions, + AwsConfig, ColRange, CommonConfig, CompletionOptions, PartitionBound, SleeperParquetOptions, SleeperPartitionRegion, run_compaction, }; use std::{ @@ -166,7 +166,7 @@ impl<'a> TryFrom<&'a FFICompactionParams> for CommonConfig<'a> { .map(String::from) .collect(), region, - OperationOutput::File { + CompletionOptions::File { output_file: unsafe { CStr::from_ptr(params.output_file) } .to_str() .map(Url::parse)??, From 80f66eefdcaadb23f02ea23a3a8887bb81693a50 Mon Sep 17 00:00:00 2001 From: m09526 Date: Wed, 13 Aug 2025 12:39:06 +0000 Subject: [PATCH 06/28] CLippy --- rust/sleeper_core/src/datafusion/output.rs | 43 +++++++++++++++++++++- 1 file changed, 41 insertions(+), 2 deletions(-) diff --git a/rust/sleeper_core/src/datafusion/output.rs b/rust/sleeper_core/src/datafusion/output.rs index 37397d6b205..57fe391cc5d 100644 --- a/rust/sleeper_core/src/datafusion/output.rs +++ b/rust/sleeper_core/src/datafusion/output.rs @@ -27,7 +27,7 @@ use datafusion::{ common::plan_err, error::DataFusionError, execution::{SendableRecordBatchStream, TaskContext}, - physical_plan::{ExecutionPlan, collect}, + physical_plan::{ExecutionPlan, collect, execute_stream}, prelude::DataFrame, }; use std::{ @@ -53,6 +53,7 @@ pub enum CompletionOptions { impl CompletionOptions { /// Create a [`Completer`] for this type of output. + #[must_use] pub fn finisher<'a>(&self, ops: &'a SleeperOperations<'a>) -> Arc + 'a> { Arc::new(match self { Self::ArrowRecordBatch => { @@ -113,7 +114,7 @@ pub trait Completer<'a> { /// Writes output of frames to a Parquet file. #[derive(Debug)] -pub struct FileOutputCompleter<'a> { +struct FileOutputCompleter<'a> { ops: &'a SleeperOperations<'a>, } @@ -150,3 +151,41 @@ impl<'a> Completer<'a> for FileOutputCompleter<'a> { Ok(CompletedOutput::File(stats)) } } + +/// Returns completed output as Arrow [`RecordBatch`]es. +#[derive(Debug)] +struct ArrowOutputCompleter<'a> { + ops: &'a SleeperOperations<'a>, +} + +impl<'a> ArrowOutputCompleter<'a> { + pub fn new(ops: &'a SleeperOperations<'a>) -> Self { + Self { ops } + } +} + +#[async_trait] +impl<'a> Completer<'a> for ArrowOutputCompleter<'a> { + fn complete_frame(&self, frame: DataFrame) -> Result { + match &self.ops.config.output { + CompletionOptions::File { + output_file: _, + opts: _, + } => { + plan_err!("Can't use ArrowOutputCompleter with CompletionOptions::File") + } + CompletionOptions::ArrowRecordBatch => Ok(frame), + } + } + + async fn execute_frame( + &self, + physical_plan: Arc, + task_ctx: Arc, + ) -> Result { + Ok(CompletedOutput::ArrowRecordBatch(execute_stream( + physical_plan, + task_ctx, + )?)) + } +} From 9410a8637b526390f5f6d94fcda11eb583646328 Mon Sep 17 00:00:00 2001 From: m09526 Date: Wed, 13 Aug 2025 12:40:48 +0000 Subject: [PATCH 07/28] Remove unneccessary lifetime --- rust/sleeper_core/src/datafusion/compact.rs | 2 +- rust/sleeper_core/src/datafusion/output.rs | 8 ++++---- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/rust/sleeper_core/src/datafusion/compact.rs b/rust/sleeper_core/src/datafusion/compact.rs index 4cba884e065..3e973788201 100644 --- a/rust/sleeper_core/src/datafusion/compact.rs +++ b/rust/sleeper_core/src/datafusion/compact.rs @@ -87,7 +87,7 @@ pub async fn compact( /// Each step of compaction may produce an error. Any are reported back to the caller. async fn build_compaction_dataframe<'a>( ops: &'a SleeperOperations<'a>, - completer: &Arc + 'a>, + completer: &Arc, configurer: &ParquetWriterConfigurer<'a>, store_factory: &ObjectStoreFactory, ) -> Result<(Sketcher<'a>, DataFrame), DataFusionError> { diff --git a/rust/sleeper_core/src/datafusion/output.rs b/rust/sleeper_core/src/datafusion/output.rs index 57fe391cc5d..b4ebab9e8ae 100644 --- a/rust/sleeper_core/src/datafusion/output.rs +++ b/rust/sleeper_core/src/datafusion/output.rs @@ -54,7 +54,7 @@ pub enum CompletionOptions { impl CompletionOptions { /// Create a [`Completer`] for this type of output. #[must_use] - pub fn finisher<'a>(&self, ops: &'a SleeperOperations<'a>) -> Arc + 'a> { + pub fn finisher<'a>(&self, ops: &'a SleeperOperations<'a>) -> Arc { Arc::new(match self { Self::ArrowRecordBatch => { unimplemented!() @@ -93,7 +93,7 @@ impl Debug for CompletedOutput { /// is finished. Sometimes we want to output the results to file(s) and sometimes /// we need a stream of processed records back. #[async_trait] -pub trait Completer<'a> { +pub trait Completer { /// Modify the given [`DataFrame`] as necessary for the desired output. /// /// # Errors @@ -125,7 +125,7 @@ impl<'a> FileOutputCompleter<'a> { } #[async_trait] -impl<'a> Completer<'a> for FileOutputCompleter<'a> { +impl<'a> Completer for FileOutputCompleter<'a> { fn complete_frame(&self, frame: DataFrame) -> Result { match &self.ops.config.output { CompletionOptions::File { @@ -165,7 +165,7 @@ impl<'a> ArrowOutputCompleter<'a> { } #[async_trait] -impl<'a> Completer<'a> for ArrowOutputCompleter<'a> { +impl<'a> Completer for ArrowOutputCompleter<'a> { fn complete_frame(&self, frame: DataFrame) -> Result { match &self.ops.config.output { CompletionOptions::File { From 68dc9ca0ef354367f2bd182c4327e7b640ec5faa Mon Sep 17 00:00:00 2001 From: m09526 Date: Wed, 13 Aug 2025 13:54:28 +0000 Subject: [PATCH 08/28] More common code --- rust/sleeper_core/src/datafusion.rs | 14 ++++++---- rust/sleeper_core/src/datafusion/compact.rs | 26 +++++++++--------- .../src/datafusion/leaf_partition_query.rs | 27 +++---------------- 3 files changed, 24 insertions(+), 43 deletions(-) diff --git a/rust/sleeper_core/src/datafusion.rs b/rust/sleeper_core/src/datafusion.rs index af8d02ab702..1589ec1bce7 100644 --- a/rust/sleeper_core/src/datafusion.rs +++ b/rust/sleeper_core/src/datafusion.rs @@ -73,9 +73,7 @@ impl<'a> SleeperOperations<'a> { Self { config } } - /// Create the `DataFusion` session configuration for a given session. - /// - /// This sets as many parameters as possible from the given input data. + /// Sets as many parameters as possible from the given input data. /// pub async fn apply_config( &self, @@ -96,14 +94,20 @@ impl<'a> SleeperOperations<'a> { // Set upload size if outputting to a file if let CompletionOptions::File { output_file: _, - opts: _, - } = self.config.output + opts: parquet_options, + } = &self.config.output { let total_input_size = retrieve_input_size(&self.config.input_files, store_factory) .await .inspect_err(|e| warn!("Error getting total input data size {e}"))?; cfg.options_mut().execution.objectstore_writer_buffer_size = calculate_upload_size(total_input_size)?; + + // Create Parquet configuration object based on requested output + let configurer = ParquetWriterConfigurer { + parquet_options: &parquet_options, + }; + cfg = configurer.apply_parquet_config(cfg); } Ok(cfg) } diff --git a/rust/sleeper_core/src/datafusion/compact.rs b/rust/sleeper_core/src/datafusion/compact.rs index 3e973788201..e7fdb72ee5d 100644 --- a/rust/sleeper_core/src/datafusion/compact.rs +++ b/rust/sleeper_core/src/datafusion/compact.rs @@ -20,7 +20,7 @@ use crate::{ datafusion::{ ParquetWriterConfigurer, SleeperOperations, metrics::RowCounts, - output::Completer, + output::{CompletedOutput, Completer}, sketch::{Sketcher, output_sketch}, util::{collect_stats, explain_plan}, }, @@ -58,18 +58,15 @@ pub async fn compact( return plan_err!("Sleeper compactions must output to a file"); }; - let configurer = ParquetWriterConfigurer { parquet_options }; - let completer = config.output.finisher(&ops); - // Make compaction DataFrame - let (sketcher, frame) = - build_compaction_dataframe(&ops, &completer, &configurer, store_factory).await?; + let completer = config.output.finisher(&ops); + let (sketcher, frame) = build_compaction_dataframe(&ops, &completer, store_factory).await?; // Explain logical plan explain_plan(&frame).await?; // Run plan - let stats = execute_compaction_plan(&ops, frame).await?; + let stats = execute_compaction_plan(&ops, &completer, frame).await?; // Write the frame out and collect stats output_sketch(store_factory, output_file, sketcher.sketch()).await?; @@ -88,13 +85,11 @@ pub async fn compact( async fn build_compaction_dataframe<'a>( ops: &'a SleeperOperations<'a>, completer: &Arc, - configurer: &ParquetWriterConfigurer<'a>, store_factory: &ObjectStoreFactory, ) -> Result<(Sketcher<'a>, DataFrame), DataFusionError> { let sf = ops .apply_config(SessionConfig::new(), store_factory) .await?; - let sf = configurer.apply_parquet_config(sf); let ctx = ops.configure_context(SessionContext::new_with_config(sf), store_factory)?; let mut frame = ops.create_initial_partitioned_read(&ctx).await?; frame = ops.apply_user_filters(frame)?; @@ -103,7 +98,6 @@ async fn build_compaction_dataframe<'a>( let sketcher = ops.create_sketcher(frame.schema()); frame = sketcher.apply_sketch(frame)?; frame = completer.complete_frame(frame)?; - // frame = ops.plan_with_parquet_output(frame, configurer)?; Ok((sketcher, frame)) } @@ -113,8 +107,9 @@ async fn build_compaction_dataframe<'a>( /// /// # Errors /// Any error that occurs during execution will be returned. -async fn execute_compaction_plan( +async fn execute_compaction_plan<'a>( ops: &SleeperOperations<'_>, + completer: &Arc, frame: DataFrame, ) -> Result { let task_ctx = Arc::new(frame.task_ctx()); @@ -123,7 +118,10 @@ async fn execute_compaction_plan( "Physical plan\n{}", displayable(&*physical_plan).indent(true) ); - collect(physical_plan.clone(), task_ctx).await?; - let stats = collect_stats(&ops.config.input_files, &physical_plan)?; - Ok(stats) + match completer.execute_frame(physical_plan, task_ctx).await? { + CompletedOutput::File(stats) => Ok(stats), + CompletedOutput::ArrowRecordBatch(_) => { + panic!("FileOutputCompleter did not return a CompletedOutput::File") + } + } } diff --git a/rust/sleeper_core/src/datafusion/leaf_partition_query.rs b/rust/sleeper_core/src/datafusion/leaf_partition_query.rs index 5c8804837f5..55bed60715a 100644 --- a/rust/sleeper_core/src/datafusion/leaf_partition_query.rs +++ b/rust/sleeper_core/src/datafusion/leaf_partition_query.rs @@ -131,7 +131,9 @@ impl<'a> LeafPartitionQuery<'a> { &self, ops: &'a SleeperOperations<'a>, ) -> Result<(Option>, DataFrame), DataFusionError> { - let sf = prepare_session_config(ops, self.store_factory).await?; + let sf = ops + .apply_config(SessionConfig::new(), &self.store_factory) + .await?; let ctx = ops.configure_context(SessionContext::new_with_config(sf), self.store_factory)?; let mut frame = ops.create_initial_partitioned_read(&ctx).await?; frame = self.apply_query_regions(frame)?; @@ -166,26 +168,3 @@ impl LeafPartitionQuery<'_> { }) } } - -/// Create the [`SessionConfig`] for a query. -async fn prepare_session_config<'a>( - ops: &SleeperOperations<'a>, - store_factory: &'a ObjectStoreFactory, -) -> Result { - let sf = ops - .apply_config(SessionConfig::new(), store_factory) - .await?; - Ok( - if let CompletionOptions::File { - output_file: _, - opts: parquet_options, - } = &ops.config.output - { - // Create Parquet configuration object based on requested output - let configurer = ParquetWriterConfigurer { parquet_options }; - configurer.apply_parquet_config(sf) - } else { - sf - }, - ) -} From 2fccf4bfe64555220b0f505c6e041a00ffec7ca9 Mon Sep 17 00:00:00 2001 From: m09526 Date: Wed, 13 Aug 2025 14:09:43 +0000 Subject: [PATCH 09/28] Clippy warnings --- rust/sleeper_core/src/datafusion.rs | 11 +++-- rust/sleeper_core/src/datafusion/compact.rs | 7 ++-- .../src/datafusion/leaf_partition_query.rs | 42 +++++++++++++++---- rust/sleeper_core/src/datafusion/output.rs | 4 +- 4 files changed, 48 insertions(+), 16 deletions(-) diff --git a/rust/sleeper_core/src/datafusion.rs b/rust/sleeper_core/src/datafusion.rs index 1589ec1bce7..adee3f8905c 100644 --- a/rust/sleeper_core/src/datafusion.rs +++ b/rust/sleeper_core/src/datafusion.rs @@ -21,6 +21,7 @@ use crate::{ CommonConfig, datafusion::{ filter_aggregation_config::{FilterAggregationConfig, validate_aggregations}, + output::Completer, sketch::Sketcher, util::{ calculate_upload_size, check_for_sort_exec, register_store, @@ -104,9 +105,7 @@ impl<'a> SleeperOperations<'a> { calculate_upload_size(total_input_size)?; // Create Parquet configuration object based on requested output - let configurer = ParquetWriterConfigurer { - parquet_options: &parquet_options, - }; + let configurer = ParquetWriterConfigurer { parquet_options }; cfg = configurer.apply_parquet_config(cfg); } Ok(cfg) @@ -346,6 +345,12 @@ impl<'a> SleeperOperations<'a> { .collect::, DataFusionError>>()?, )) } + + /// Create appropriate output completer. + #[must_use] + pub fn create_output_completer(&self) -> Arc { + self.config.output.finisher(self) + } } impl std::fmt::Display for SleeperOperations<'_> { diff --git a/rust/sleeper_core/src/datafusion/compact.rs b/rust/sleeper_core/src/datafusion/compact.rs index e7fdb72ee5d..6e634303bc0 100644 --- a/rust/sleeper_core/src/datafusion/compact.rs +++ b/rust/sleeper_core/src/datafusion/compact.rs @@ -18,11 +18,11 @@ use crate::{ CommonConfig, CompactionResult, CompletionOptions, datafusion::{ - ParquetWriterConfigurer, SleeperOperations, + SleeperOperations, metrics::RowCounts, output::{CompletedOutput, Completer}, sketch::{Sketcher, output_sketch}, - util::{collect_stats, explain_plan}, + util::explain_plan, }, }; use datafusion::{ @@ -31,7 +31,7 @@ use datafusion::{ error::DataFusionError, execution::config::SessionConfig, execution::context::SessionContext, - physical_plan::{collect, displayable}, + physical_plan::displayable, }; use log::info; use objectstore_ext::s3::ObjectStoreFactory; @@ -71,6 +71,7 @@ pub async fn compact( // Write the frame out and collect stats output_sketch(store_factory, output_file, sketcher.sketch()).await?; + // Dump input file metrics to logging console stats.log_metrics(); Ok(CompactionResult::from(&stats)) } diff --git a/rust/sleeper_core/src/datafusion/leaf_partition_query.rs b/rust/sleeper_core/src/datafusion/leaf_partition_query.rs index 55bed60715a..5a8bf2a5d5b 100644 --- a/rust/sleeper_core/src/datafusion/leaf_partition_query.rs +++ b/rust/sleeper_core/src/datafusion/leaf_partition_query.rs @@ -18,12 +18,16 @@ use crate::{ CommonConfig, CompletionOptions, SleeperPartitionRegion, datafusion::{ - ParquetWriterConfigurer, SleeperOperations, sketch::Sketcher, util::explain_plan, + SleeperOperations, output::CompletedOutput, sketch::Sketcher, + util::explain_plan, }, }; #[cfg(doc)] use arrow::record_batch::RecordBatch; -use datafusion::{common::plan_err, logical_expr::Expr}; +use datafusion::{ + common::plan_err, execution::SendableRecordBatchStream, logical_expr::Expr, + physical_plan::displayable, +}; use datafusion::{ dataframe::DataFrame, error::DataFusionError, @@ -31,7 +35,10 @@ use datafusion::{ }; use log::info; use objectstore_ext::s3::ObjectStoreFactory; -use std::fmt::{Display, Formatter}; +use std::{ + fmt::{Display, Formatter}, + sync::Arc, +}; /// All information needed for a Sleeper leaf partition query. #[derive(Debug, Default)] @@ -80,16 +87,35 @@ impl<'a> LeafPartitionQuery<'a> { /// /// The object store factory must be able to produce an [`object_store::ObjectStore`] capable of reading /// from the input URLs and writing to the output URL (if writing results to a file). - pub async fn run_query(&self) -> Result<(), DataFusionError> { + pub async fn run_query(&self) -> Result { let ops = SleeperOperations::new(&self.config.common); - info!("DataFusion compaction: {ops}"); - // Create query frame an sketches if it has been enabled + info!("DataFusion query: {ops}"); + // Create query frame and sketches if it has been enabled let (sketcher, frame) = self.build_query_dataframe(&ops).await?; if self.config.explain_plans { explain_plan(&frame).await?; } - todo!(); + + // Convert to physical plan + let completer = ops.create_output_completer(); + let frame = completer.complete_frame(frame)?; + let task_ctx = Arc::new(frame.task_ctx()); + let physical_plan = ops.to_physical_plan(frame).await?; + + if self.config.explain_plans { + info!( + "Physical plan\n{}", + displayable(&*physical_plan).indent(true) + ); + } + + match completer.execute_frame(physical_plan, task_ctx).await? { + CompletedOutput::ArrowRecordBatch(stream) => Ok(stream), + CompletedOutput::File(_) => { + panic!("ArrowOutputCompleter did not return a CompletedOutput::ArrowRecordBatch") + } + } } /// Adds a quantile sketch to a query plan if sketch generation is enabled. @@ -132,7 +158,7 @@ impl<'a> LeafPartitionQuery<'a> { ops: &'a SleeperOperations<'a>, ) -> Result<(Option>, DataFrame), DataFusionError> { let sf = ops - .apply_config(SessionConfig::new(), &self.store_factory) + .apply_config(SessionConfig::new(), self.store_factory) .await?; let ctx = ops.configure_context(SessionContext::new_with_config(sf), self.store_factory)?; let mut frame = ops.create_initial_partitioned_read(&ctx).await?; diff --git a/rust/sleeper_core/src/datafusion/output.rs b/rust/sleeper_core/src/datafusion/output.rs index b4ebab9e8ae..6bef187b84c 100644 --- a/rust/sleeper_core/src/datafusion/output.rs +++ b/rust/sleeper_core/src/datafusion/output.rs @@ -125,7 +125,7 @@ impl<'a> FileOutputCompleter<'a> { } #[async_trait] -impl<'a> Completer for FileOutputCompleter<'a> { +impl Completer for FileOutputCompleter<'_> { fn complete_frame(&self, frame: DataFrame) -> Result { match &self.ops.config.output { CompletionOptions::File { @@ -165,7 +165,7 @@ impl<'a> ArrowOutputCompleter<'a> { } #[async_trait] -impl<'a> Completer for ArrowOutputCompleter<'a> { +impl Completer for ArrowOutputCompleter<'_> { fn complete_frame(&self, frame: DataFrame) -> Result { match &self.ops.config.output { CompletionOptions::File { From 3dcd32153103e0ecf770452e7195dbfb0ecfd88a Mon Sep 17 00:00:00 2001 From: m09526 Date: Wed, 13 Aug 2025 14:20:02 +0000 Subject: [PATCH 10/28] Query sketch generation --- .../src/datafusion/leaf_partition_query.rs | 31 +++++++++++++++---- 1 file changed, 25 insertions(+), 6 deletions(-) diff --git a/rust/sleeper_core/src/datafusion/leaf_partition_query.rs b/rust/sleeper_core/src/datafusion/leaf_partition_query.rs index 5a8bf2a5d5b..cd04a0aa2b1 100644 --- a/rust/sleeper_core/src/datafusion/leaf_partition_query.rs +++ b/rust/sleeper_core/src/datafusion/leaf_partition_query.rs @@ -18,7 +18,9 @@ use crate::{ CommonConfig, CompletionOptions, SleeperPartitionRegion, datafusion::{ - SleeperOperations, output::CompletedOutput, sketch::Sketcher, + SleeperOperations, + output::CompletedOutput, + sketch::{Sketcher, output_sketch}, util::explain_plan, }, }; @@ -87,7 +89,7 @@ impl<'a> LeafPartitionQuery<'a> { /// /// The object store factory must be able to produce an [`object_store::ObjectStore`] capable of reading /// from the input URLs and writing to the output URL (if writing results to a file). - pub async fn run_query(&self) -> Result { + pub async fn run_query(&self) -> Result { let ops = SleeperOperations::new(&self.config.common); info!("DataFusion query: {ops}"); // Create query frame and sketches if it has been enabled @@ -110,12 +112,29 @@ impl<'a> LeafPartitionQuery<'a> { ); } - match completer.execute_frame(physical_plan, task_ctx).await? { - CompletedOutput::ArrowRecordBatch(stream) => Ok(stream), - CompletedOutput::File(_) => { - panic!("ArrowOutputCompleter did not return a CompletedOutput::ArrowRecordBatch") + // Run query + let result = completer.execute_frame(physical_plan, task_ctx).await?; + + // Do we have some sketch output to write? + if let Some(sketch_func) = sketcher + && self.config.write_quantile_sketch + { + match &self.config.common.output { + CompletionOptions::File { + output_file, + opts: _, + } => { + output_sketch(self.store_factory, output_file, sketch_func.sketch()).await?; + } + CompletionOptions::ArrowRecordBatch => { + return plan_err!( + "Quantile sketch output cannot be enabled if file output not selected" + ); + } } } + + Ok(result) } /// Adds a quantile sketch to a query plan if sketch generation is enabled. From 6ef8df18b33fdc327f2960c171a67964de0af1cb Mon Sep 17 00:00:00 2001 From: m09526 Date: Wed, 13 Aug 2025 15:12:01 +0000 Subject: [PATCH 11/28] Fix --- rust/sleeper_core/src/datafusion.rs | 3 ++- rust/sleeper_core/src/datafusion/compact.rs | 19 ++++++++----------- .../src/datafusion/leaf_partition_query.rs | 4 ++-- rust/sleeper_core/src/datafusion/output.rs | 12 +++++------- 4 files changed, 17 insertions(+), 21 deletions(-) diff --git a/rust/sleeper_core/src/datafusion.rs b/rust/sleeper_core/src/datafusion.rs index adee3f8905c..fdc9dfc503d 100644 --- a/rust/sleeper_core/src/datafusion.rs +++ b/rust/sleeper_core/src/datafusion.rs @@ -58,6 +58,7 @@ mod util; pub use compact::compact; pub use config::ParquetWriterConfigurer; +pub use leaf_partition_query::{LeafPartitionQuery, LeafPartitionQueryConfig}; pub use output::CompletionOptions; pub use region::SleeperPartitionRegion; @@ -348,7 +349,7 @@ impl<'a> SleeperOperations<'a> { /// Create appropriate output completer. #[must_use] - pub fn create_output_completer(&self) -> Arc { + pub fn create_output_completer(&self) -> Box { self.config.output.finisher(self) } } diff --git a/rust/sleeper_core/src/datafusion/compact.rs b/rust/sleeper_core/src/datafusion/compact.rs index 6e634303bc0..b7d66b92aeb 100644 --- a/rust/sleeper_core/src/datafusion/compact.rs +++ b/rust/sleeper_core/src/datafusion/compact.rs @@ -16,9 +16,9 @@ * limitations under the License. */ use crate::{ - CommonConfig, CompactionResult, CompletionOptions, + CommonConfig, CompactionResult, datafusion::{ - SleeperOperations, + CompletionOptions, SleeperOperations, metrics::RowCounts, output::{CompletedOutput, Completer}, sketch::{Sketcher, output_sketch}, @@ -26,11 +26,8 @@ use crate::{ }, }; use datafusion::{ - common::plan_err, - dataframe::DataFrame, - error::DataFusionError, - execution::config::SessionConfig, - execution::context::SessionContext, + common::plan_err, dataframe::DataFrame, error::DataFusionError, + execution::config::SessionConfig, execution::context::SessionContext, physical_plan::displayable, }; use log::info; @@ -52,7 +49,7 @@ pub async fn compact( // Retrieve Parquet output options let CompletionOptions::File { output_file, - opts: parquet_options, + opts: _, } = &config.output else { return plan_err!("Sleeper compactions must output to a file"); @@ -69,7 +66,7 @@ pub async fn compact( let stats = execute_compaction_plan(&ops, &completer, frame).await?; // Write the frame out and collect stats - output_sketch(store_factory, output_file, sketcher.sketch()).await?; + output_sketch(store_factory, &output_file, sketcher.sketch()).await?; // Dump input file metrics to logging console stats.log_metrics(); @@ -85,7 +82,7 @@ pub async fn compact( /// Each step of compaction may produce an error. Any are reported back to the caller. async fn build_compaction_dataframe<'a>( ops: &'a SleeperOperations<'a>, - completer: &Arc, + completer: &Box, store_factory: &ObjectStoreFactory, ) -> Result<(Sketcher<'a>, DataFrame), DataFusionError> { let sf = ops @@ -110,7 +107,7 @@ async fn build_compaction_dataframe<'a>( /// Any error that occurs during execution will be returned. async fn execute_compaction_plan<'a>( ops: &SleeperOperations<'_>, - completer: &Arc, + completer: &Box, frame: DataFrame, ) -> Result { let task_ctx = Arc::new(frame.task_ctx()); diff --git a/rust/sleeper_core/src/datafusion/leaf_partition_query.rs b/rust/sleeper_core/src/datafusion/leaf_partition_query.rs index cd04a0aa2b1..545630629d1 100644 --- a/rust/sleeper_core/src/datafusion/leaf_partition_query.rs +++ b/rust/sleeper_core/src/datafusion/leaf_partition_query.rs @@ -16,9 +16,9 @@ * limitations under the License. */ use crate::{ - CommonConfig, CompletionOptions, SleeperPartitionRegion, + CommonConfig, SleeperPartitionRegion, datafusion::{ - SleeperOperations, + CompletionOptions, SleeperOperations, output::CompletedOutput, sketch::{Sketcher, output_sketch}, util::explain_plan, diff --git a/rust/sleeper_core/src/datafusion/output.rs b/rust/sleeper_core/src/datafusion/output.rs index 6bef187b84c..4666fb4f901 100644 --- a/rust/sleeper_core/src/datafusion/output.rs +++ b/rust/sleeper_core/src/datafusion/output.rs @@ -54,16 +54,14 @@ pub enum CompletionOptions { impl CompletionOptions { /// Create a [`Completer`] for this type of output. #[must_use] - pub fn finisher<'a>(&self, ops: &'a SleeperOperations<'a>) -> Arc { - Arc::new(match self { - Self::ArrowRecordBatch => { - unimplemented!() - } + pub fn finisher<'a>(&self, ops: &'a SleeperOperations<'a>) -> Box { + match self { + Self::ArrowRecordBatch => Box::new(ArrowOutputCompleter::new(ops)), Self::File { output_file: _, opts: _, - } => FileOutputCompleter::new(ops), - }) + } => Box::new(FileOutputCompleter::new(ops)), + } } } From bf63e7bf2e201633be62bf9109a55417442ab009 Mon Sep 17 00:00:00 2001 From: m09526 Date: Wed, 13 Aug 2025 15:19:39 +0000 Subject: [PATCH 12/28] tests passing --- .../src/datafusion/leaf_partition_query.rs | 5 +---- rust/sleeper_core/src/lib.rs | 17 ++++++++++++++--- 2 files changed, 15 insertions(+), 7 deletions(-) diff --git a/rust/sleeper_core/src/datafusion/leaf_partition_query.rs b/rust/sleeper_core/src/datafusion/leaf_partition_query.rs index 545630629d1..ee629198456 100644 --- a/rust/sleeper_core/src/datafusion/leaf_partition_query.rs +++ b/rust/sleeper_core/src/datafusion/leaf_partition_query.rs @@ -26,10 +26,7 @@ use crate::{ }; #[cfg(doc)] use arrow::record_batch::RecordBatch; -use datafusion::{ - common::plan_err, execution::SendableRecordBatchStream, logical_expr::Expr, - physical_plan::displayable, -}; +use datafusion::{common::plan_err, logical_expr::Expr, physical_plan::displayable}; use datafusion::{ dataframe::DataFrame, error::DataFusionError, diff --git a/rust/sleeper_core/src/lib.rs b/rust/sleeper_core/src/lib.rs index a10963673b3..589faaa7e56 100644 --- a/rust/sleeper_core/src/lib.rs +++ b/rust/sleeper_core/src/lib.rs @@ -32,8 +32,10 @@ use url::Url; mod datafusion; +use crate::datafusion::LeafPartitionQuery; +pub use crate::datafusion::output::CompletedOutput; pub use datafusion::{ - CompletionOptions, SleeperPartitionRegion, + CompletionOptions, LeafPartitionQueryConfig, SleeperPartitionRegion, sketch::{DataSketchVariant, deserialise_sketches}, }; @@ -271,11 +273,11 @@ pub struct CompactionResult { /// # use aws_types::region::Region; /// # use std::collections::HashMap; /// # use crate::sleeper_core::{run_compaction, CommonConfig, PartitionBound, ColRange, -/// # OperationOutput, SleeperParquetOptions}; +/// # CompletionOptions, SleeperParquetOptions}; /// let mut compaction_input = CommonConfig::default(); /// compaction_input.input_files_sorted = true; /// compaction_input.input_files = vec![Url::parse("file:///path/to/file1.parquet").unwrap()]; -/// compaction_input.output = OperationOutput::File{ output_file: Url::parse("file:///path/to/output").unwrap(), opts: SleeperParquetOptions::default() }; +/// compaction_input.output = CompletionOptions::File{ output_file: Url::parse("file:///path/to/output").unwrap(), opts: SleeperParquetOptions::default() }; /// compaction_input.row_key_cols = vec!["key".into()]; /// let mut region : HashMap> = HashMap::new(); /// region.insert("key".into(), ColRange { @@ -301,6 +303,15 @@ pub async fn run_compaction(config: &CommonConfig<'_>) -> Result) -> Result { + let store_factory = create_object_store_factory(config.common.aws_config.as_ref()).await; + + LeafPartitionQuery::new(config, &store_factory) + .run_query() + .await + .map_err(Into::into) +} + async fn create_object_store_factory( aws_config_override: Option<&AwsConfig>, ) -> ObjectStoreFactory { From a63ef5e31f92e84d6b69a5097bd0a5a5c161e28d Mon Sep 17 00:00:00 2001 From: m09526 Date: Wed, 13 Aug 2025 15:30:03 +0000 Subject: [PATCH 13/28] Doc tests pass --- .../src/datafusion/leaf_partition_query.rs | 6 +++ rust/sleeper_core/src/lib.rs | 54 +++++++++++++++++-- 2 files changed, 55 insertions(+), 5 deletions(-) diff --git a/rust/sleeper_core/src/datafusion/leaf_partition_query.rs b/rust/sleeper_core/src/datafusion/leaf_partition_query.rs index ee629198456..e9afd668503 100644 --- a/rust/sleeper_core/src/datafusion/leaf_partition_query.rs +++ b/rust/sleeper_core/src/datafusion/leaf_partition_query.rs @@ -86,7 +86,13 @@ impl<'a> LeafPartitionQuery<'a> { /// /// The object store factory must be able to produce an [`object_store::ObjectStore`] capable of reading /// from the input URLs and writing to the output URL (if writing results to a file). + /// + /// # Errors + /// There must be at least one query region specified. pub async fn run_query(&self) -> Result { + if self.config.ranges.is_empty() { + return plan_err!("No query regions specified"); + } let ops = SleeperOperations::new(&self.config.common); info!("DataFusion query: {ops}"); // Create query frame and sketches if it has been enabled diff --git a/rust/sleeper_core/src/lib.rs b/rust/sleeper_core/src/lib.rs index 589faaa7e56..4cbba879412 100644 --- a/rust/sleeper_core/src/lib.rs +++ b/rust/sleeper_core/src/lib.rs @@ -260,10 +260,7 @@ pub struct CompactionResult { pub rows_written: usize, } -/// Merges the given Parquet files and reads the schema from the first. -/// -/// This function reads the schema from the first file, then calls -/// `merge_sorted_files_with_schema(...)`. +/// Compacts the given Parquet files and reads the schema from the first. /// /// The `aws_creds` are optional if you are not attempting to read/write files from S3. /// @@ -273,7 +270,7 @@ pub struct CompactionResult { /// # use aws_types::region::Region; /// # use std::collections::HashMap; /// # use crate::sleeper_core::{run_compaction, CommonConfig, PartitionBound, ColRange, -/// # CompletionOptions, SleeperParquetOptions}; +/// # CompletionOptions, SleeperParquetOptions, SleeperPartitionRegion}; /// let mut compaction_input = CommonConfig::default(); /// compaction_input.input_files_sorted = true; /// compaction_input.input_files = vec![Url::parse("file:///path/to/file1.parquet").unwrap()]; @@ -286,6 +283,7 @@ pub struct CompactionResult { /// upper: PartitionBound::String("h"), /// upper_inclusive: true, /// }); +/// compaction_input.region = SleeperPartitionRegion::new(region); /// /// # tokio_test::block_on(async { /// let result = run_compaction(&compaction_input).await; @@ -303,6 +301,52 @@ pub async fn run_compaction(config: &CommonConfig<'_>) -> Result> = HashMap::new(); +/// region.insert("key".into(), ColRange { +/// lower : PartitionBound::String("a"), +/// lower_inclusive: true, +/// upper: PartitionBound::String("h"), +/// upper_inclusive: true, +/// }); +/// common.region = SleeperPartitionRegion::new(region); +/// +/// let mut leaf_config = LeafPartitionQueryConfig::default(); +/// leaf_config.common = common; +/// let mut query_region : HashMap> = HashMap::new(); +/// query_region.insert("key".into(), ColRange { +/// lower : PartitionBound::String("a"), +/// lower_inclusive: true, +/// upper: PartitionBound::String("h"), +/// upper_inclusive: true, +/// }); +/// leaf_config.ranges = vec![SleeperPartitionRegion::new(query_region)]; +/// +/// # tokio_test::block_on(async { +/// let result = run_query(&leaf_config).await; +/// # }) +/// ``` +/// +/// # Errors +/// There must be at least one input file. +/// There must be at least one query region specified. +/// pub async fn run_query(config: &LeafPartitionQueryConfig<'_>) -> Result { let store_factory = create_object_store_factory(config.common.aws_config.as_ref()).await; From e6a2660617a918648dfc8e0e7d730fe0c41932bf Mon Sep 17 00:00:00 2001 From: m09526 Date: Wed, 13 Aug 2025 15:31:40 +0000 Subject: [PATCH 14/28] Clippy warnings --- rust/sleeper_core/src/datafusion/compact.rs | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/rust/sleeper_core/src/datafusion/compact.rs b/rust/sleeper_core/src/datafusion/compact.rs index b7d66b92aeb..a054826726b 100644 --- a/rust/sleeper_core/src/datafusion/compact.rs +++ b/rust/sleeper_core/src/datafusion/compact.rs @@ -57,16 +57,17 @@ pub async fn compact( // Make compaction DataFrame let completer = config.output.finisher(&ops); - let (sketcher, frame) = build_compaction_dataframe(&ops, &completer, store_factory).await?; + let (sketcher, frame) = + build_compaction_dataframe(&ops, completer.as_ref(), store_factory).await?; // Explain logical plan explain_plan(&frame).await?; // Run plan - let stats = execute_compaction_plan(&ops, &completer, frame).await?; + let stats = execute_compaction_plan(&ops, completer.as_ref(), frame).await?; // Write the frame out and collect stats - output_sketch(store_factory, &output_file, sketcher.sketch()).await?; + output_sketch(store_factory, output_file, sketcher.sketch()).await?; // Dump input file metrics to logging console stats.log_metrics(); @@ -82,7 +83,7 @@ pub async fn compact( /// Each step of compaction may produce an error. Any are reported back to the caller. async fn build_compaction_dataframe<'a>( ops: &'a SleeperOperations<'a>, - completer: &Box, + completer: &(dyn Completer + 'a), store_factory: &ObjectStoreFactory, ) -> Result<(Sketcher<'a>, DataFrame), DataFusionError> { let sf = ops @@ -107,7 +108,7 @@ async fn build_compaction_dataframe<'a>( /// Any error that occurs during execution will be returned. async fn execute_compaction_plan<'a>( ops: &SleeperOperations<'_>, - completer: &Box, + completer: &(dyn Completer + 'a), frame: DataFrame, ) -> Result { let task_ctx = Arc::new(frame.task_ctx()); From e09cd53fdab89221b0574d312be03acb998bc8bb Mon Sep 17 00:00:00 2001 From: m09526 Date: Wed, 13 Aug 2025 16:19:01 +0000 Subject: [PATCH 15/28] CLI for query --- rust/Cargo.lock | 38 ++-- rust/Cargo.toml | 2 +- rust/{compactor => apps}/Cargo.toml | 7 +- .../bin/main.rs => apps/src/bin/compact.rs} | 110 +++------- rust/apps/src/bin/query.rs | 189 ++++++++++++++++++ rust/apps/src/lib.rs | 74 +++++++ .../tests/compactor_cli.rs | 4 +- 7 files changed, 315 insertions(+), 109 deletions(-) rename rust/{compactor => apps}/Cargo.toml (88%) rename rust/{compactor/src/bin/main.rs => apps/src/bin/compact.rs} (62%) create mode 100644 rust/apps/src/bin/query.rs create mode 100644 rust/apps/src/lib.rs rename rust/{compactor => apps}/tests/compactor_cli.rs (94%) diff --git a/rust/Cargo.lock b/rust/Cargo.lock index eaef9097229..ecbe2e0cbac 100644 --- a/rust/Cargo.lock +++ b/rust/Cargo.lock @@ -137,6 +137,26 @@ dependencies = [ "windows-sys 0.59.0", ] +[[package]] +name = "apps" +version = "0.33.0-SNAPSHOT" +dependencies = [ + "arrow", + "assert_cmd", + "chrono", + "clap", + "color-eyre", + "env_logger", + "futures", + "human-panic", + "log", + "num-format", + "predicates", + "sleeper_core", + "tokio", + "url", +] + [[package]] name = "arrayref" version = "0.3.9" @@ -1212,24 +1232,6 @@ dependencies = [ "unicode-width", ] -[[package]] -name = "compactor" -version = "0.33.0-SNAPSHOT" -dependencies = [ - "assert_cmd", - "chrono", - "clap", - "color-eyre", - "env_logger", - "human-panic", - "log", - "num-format", - "predicates", - "sleeper_core", - "tokio", - "url", -] - [[package]] name = "const-random" version = "0.1.18" diff --git a/rust/Cargo.toml b/rust/Cargo.toml index d6e48e54cb0..26b5077db15 100644 --- a/rust/Cargo.toml +++ b/rust/Cargo.toml @@ -12,7 +12,7 @@ # See the License for the specific language governing permissions and # limitations under the License. [workspace] -members = [ "aggregator_udfs", "filter_udfs", "compactor", "objectstore_ext", "rust_sketch", "sleeper_core", "sleeper_df" ] +members = [ "aggregator_udfs", "filter_udfs", "apps", "objectstore_ext", "rust_sketch", "sleeper_core", "sleeper_df" ] resolver = "2" [workspace.package] diff --git a/rust/compactor/Cargo.toml b/rust/apps/Cargo.toml similarity index 88% rename from rust/compactor/Cargo.toml rename to rust/apps/Cargo.toml index 9992c6d29b1..2ea405dcdfb 100644 --- a/rust/compactor/Cargo.toml +++ b/rust/apps/Cargo.toml @@ -12,8 +12,9 @@ # See the License for the specific language governing permissions and # limitations under the License. [package] -name = "compactor" -description = "Binary crate for running the data compactor from the command line." +name = "apps" +description = "Binary crate for running the Sleeper apps from the command line." +default-run = "compact" keywords = ["sleeper", "compaction"] categories = ["science"] version = { workspace = true } @@ -25,11 +26,13 @@ rust-version = { workspace = true } publish = { workspace = true } [dependencies] +arrow = { workspace = true } chrono = { workspace = true } clap = { workspace = true, features = ["derive"] } color-eyre = { workspace = true } sleeper_core = { path = "../sleeper_core" } env_logger = { workspace = true } +futures = { workspace = true } human-panic = { workspace = true } log = { workspace = true, features = ["release_max_level_debug"] } num-format = { workspace = true } diff --git a/rust/compactor/src/bin/main.rs b/rust/apps/src/bin/compact.rs similarity index 62% rename from rust/compactor/src/bin/main.rs rename to rust/apps/src/bin/compact.rs index 0350163601c..ca2fe57a976 100644 --- a/rust/compactor/src/bin/main.rs +++ b/rust/apps/src/bin/compact.rs @@ -1,18 +1,19 @@ /* - * Copyright 2022-2025 Crown Copyright - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ +* Copyright 2022-2025 Crown Copyright +* +* Licensed under the Apache License, Version 2.0 (the "License"); +* you may not use this file except in compliance with the License. +* You may obtain a copy of the License at +* +* http://www.apache.org/licenses/LICENSE-2.0 +* +* Unless required by applicable law or agreed to in writing, software +* distributed under the License is distributed on an "AS IS" BASIS, +* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +* See the License for the specific language governing permissions and +* limitations under the License. +*/ +use apps::path_absolute; use chrono::Local; use clap::Parser; use color_eyre::eyre::bail; @@ -23,10 +24,10 @@ use sleeper_core::{ ColRange, CommonConfig, CompletionOptions, PartitionBound, SleeperParquetOptions, SleeperPartitionRegion, run_compaction, }; -use std::{collections::HashMap, io::Write, path::Path}; +use std::{collections::HashMap, io::Write}; use url::Url; -/// Implements a Sleeper compaction algorithm in Rust. +/// Runs a Sleeper compaction algorithm. /// /// A sequence of Parquet files is read and compacted into a single output Parquet file. The input /// files must be individually sorted according to the row key columns and then the sort columns. A sketches file containing @@ -63,16 +64,6 @@ struct CmdLineArgs { iterator_config: Option, } -/// Converts a [`Path`] reference to an absolute path (if not already absolute) -/// and returns it as a String. -/// -/// # Panics -/// If the path can't be made absolute due to not being able to get the current -/// directory or the path is not valid. -fn path_absolute>(path: &T) -> String { - std::path::absolute(path).unwrap().to_str().unwrap().into() -} - #[tokio::main(flavor = "multi_thread")] async fn main() -> color_eyre::Result<()> { // Install coloured errors @@ -163,65 +154,12 @@ async fn main() -> color_eyre::Result<()> { args.iterator_config, )?; - let result = run_compaction(&details).await; - match result { - Ok(r) => { - info!( - "Compaction read {} rows and wrote {} rows", - r.rows_read.to_formatted_string(&Locale::en), - r.rows_written.to_formatted_string(&Locale::en) - ); - } - Err(e) => { - bail!(e); - } - } - Ok(()) -} - -#[cfg(test)] -mod path_test { - use crate::path_absolute; - - #[cfg(not(any(target_os = "windows", target_os = "macos")))] - fn cd_to_tmp() { - std::env::set_current_dir("/tmp").unwrap(); - } - - #[test] - #[cfg(not(any(target_os = "windows", target_os = "macos")))] - fn relative_path_converts() { - cd_to_tmp(); - assert_eq!("/tmp/foo/bar/baz.txt", path_absolute("foo/bar/baz.txt")); - } - - #[test] - #[cfg(not(any(target_os = "windows", target_os = "macos")))] - fn relative_path_converts_with_one_dot() { - cd_to_tmp(); - assert_eq!("/tmp/foo/bar/baz.txt", path_absolute("./foo/bar/baz.txt")); - } - - #[test] - #[cfg(not(any(target_os = "windows", target_os = "macos")))] - fn relative_path_converts_with_double_dot() { - cd_to_tmp(); - assert_eq!( - "/tmp/../foo/bar/baz.txt", - path_absolute("../foo/bar/baz.txt") - ); - } - - #[test] - #[cfg(not(any(target_os = "windows", target_os = "macos")))] - fn absolute_path_unchanged() { - cd_to_tmp(); - assert_eq!("/tmp/foo/bar", path_absolute("/tmp/foo/bar")); - } + let result = run_compaction(&details).await?; + info!( + "Compaction read {} rows and wrote {} rows", + result.rows_read.to_formatted_string(&Locale::en), + result.rows_written.to_formatted_string(&Locale::en) + ); - #[test] - #[should_panic(expected = "cannot make an empty path absolute")] - fn empty_path_panic() { - let _ = path_absolute(""); - } + Ok(()) } diff --git a/rust/apps/src/bin/query.rs b/rust/apps/src/bin/query.rs new file mode 100644 index 00000000000..ba1589bb957 --- /dev/null +++ b/rust/apps/src/bin/query.rs @@ -0,0 +1,189 @@ +/* +* Copyright 2022-2025 Crown Copyright +* +* Licensed under the Apache License, Version 2.0 (the "License"); +* you may not use this file except in compliance with the License. +* You may obtain a copy of the License at +* +* http://www.apache.org/licenses/LICENSE-2.0 +* +* Unless required by applicable law or agreed to in writing, software +* distributed under the License is distributed on an "AS IS" BASIS, +* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +* See the License for the specific language governing permissions and +* limitations under the License. +*/ +use apps::path_absolute; +use arrow::util::pretty::print_batches; +use chrono::Local; +use clap::Parser; +use color_eyre::eyre::bail; +use futures::StreamExt; +use human_panic::setup_panic; +use sleeper_core::{ + ColRange, CommonConfig, CompletedOutput, CompletionOptions, LeafPartitionQueryConfig, + PartitionBound, SleeperPartitionRegion, run_query, +}; +use std::{collections::HashMap, io::Write}; +use url::Url; + +/// Implements a Sleeper query algorithm. +/// +/// A sequence of Parquet files is read and merge sorted. The input +/// files must be individually sorted according to the row key columns and then the sort columns. A selection +/// of query results are written to standard output. +/// +#[derive(Parser, Debug)] +#[command(author, version)] +struct CmdLineArgs { + /// List of input Parquet files (must be sorted) as URLs + #[arg(num_args=1.., required=true)] + input: Vec, + /// Column names for a row key columns + #[arg(short = 'k', long, num_args=1.., required=true)] + row_keys: Vec, + /// Column names for sort key columns + #[arg(short = 's', long)] + sort_keys: Vec, + /// Partition region minimum keys (inclusive). Must be one per row key specified. + #[arg(short='m',long,required=true,num_args=1..)] + region_mins: Vec, + /// Partition region maximum keys (exclusive). Must be one per row key specified. + #[arg(short='n',long,required=true,num_args=1..)] + region_maxs: Vec, + /// Query region minimum keys (inclusive). Must be one per row key specified. + #[arg(short='q',long,required=true,num_args=1..)] + query_mins: Vec, + /// Query region maximum keys (exclusive). Must be one per row key specified. + #[arg(short='w',long,required=true,num_args=1..)] + query_maxs: Vec, + /// Row count to write to standard output + #[arg(short = 'c', long, required = false, default_value_t = 1000)] + row_count: usize, + /// Sleeper iterator configuration + #[arg(short = 'i', long, required = false, num_args = 1)] + iterator_config: Option, +} + +#[tokio::main(flavor = "multi_thread")] +#[allow(clippy::too_many_lines)] +async fn main() -> color_eyre::Result<()> { + // Install coloured errors + color_eyre::install().unwrap(); + + // Install human readable panics + setup_panic!(); + + // Install and configure environment logger + env_logger::builder() + .format(|buf, record| { + writeln!( + buf, + "{} [{}] {}:{} - {}", + Local::now().format("%Y-%m-%dT%H:%M:%S"), + record.level(), + record.file().unwrap_or("??"), + record.line().unwrap_or(0), + record.args() + ) + }) + .format_timestamp(Some(env_logger::TimestampPrecision::Millis)) + .filter_level(log::LevelFilter::Info) + .format_target(false) + .init(); + + let args = CmdLineArgs::parse(); + + // Check URL conversion + let input_urls = args + .input + .iter() + .map(|x| { + Url::parse(x).or_else(|_e| Url::parse(&("file://".to_owned() + &path_absolute(x)))) + }) + .collect::, _>>()?; + + if args.row_keys.len() != args.region_maxs.len() { + bail!("quantity of region maximums != quantity of row key columns"); + } + if args.row_keys.len() != args.region_mins.len() { + bail!("quantity of region minimums != quantity of row key columns"); + } + if args.row_keys.len() != args.query_maxs.len() { + bail!("quantity of query region maximums != quantity of row key columns"); + } + if args.row_keys.len() != args.query_mins.len() { + bail!("quantity of query region minimums != quantity of row key columns"); + } + let mut map = HashMap::new(); + for (key, bounds) in args + .row_keys + .iter() + .zip(args.region_mins.iter().zip(args.region_maxs.iter())) + { + map.insert( + key.into(), + ColRange { + lower: PartitionBound::String(bounds.0), + lower_inclusive: true, + upper: PartitionBound::String(bounds.1), + upper_inclusive: false, + }, + ); + } + + let mut query_map = HashMap::new(); + for (key, bounds) in args + .row_keys + .iter() + .zip(args.query_mins.iter().zip(args.query_maxs.iter())) + { + query_map.insert( + key.into(), + ColRange { + lower: PartitionBound::String(bounds.0), + lower_inclusive: true, + upper: PartitionBound::String(bounds.1), + upper_inclusive: false, + }, + ); + } + + let common = CommonConfig::try_new( + None, + input_urls, + true, + args.row_keys, + args.sort_keys, + SleeperPartitionRegion::new(map), + CompletionOptions::ArrowRecordBatch, + args.iterator_config, + )?; + + let query_config = LeafPartitionQueryConfig { + common, + ranges: vec![SleeperPartitionRegion::new(query_map)], + explain_plans: true, + write_quantile_sketch: false, + }; + + let result = run_query(&query_config).await?; + let CompletedOutput::ArrowRecordBatch(mut stream) = result else { + bail!("Expected ArrowRecordBatch output"); + }; + + let mut batches = Vec::new(); + let mut rows_needed = args.row_count; + while let Some(Ok(batch)) = stream.next().await + && rows_needed > 0 + { + let batch_size = batch.num_rows(); + let rows_from_batch = std::cmp::min(rows_needed, batch_size); + let partial = batch.slice(0, rows_from_batch); + batches.push(partial); + rows_needed -= rows_from_batch; + } + + print_batches(&batches)?; + Ok(()) +} diff --git a/rust/apps/src/lib.rs b/rust/apps/src/lib.rs new file mode 100644 index 00000000000..c2851acf9a0 --- /dev/null +++ b/rust/apps/src/lib.rs @@ -0,0 +1,74 @@ +/* + * Copyright 2022-2025 Crown Copyright + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +use std::path::Path; + +/// Converts a [`Path`] reference to an absolute path (if not already absolute) +/// and returns it as a String. +/// +/// # Panics +/// If the path can't be made absolute due to not being able to get the current +/// directory or the path is not valid. +pub fn path_absolute>(path: &T) -> String { + std::path::absolute(path).unwrap().to_str().unwrap().into() +} + +#[cfg(test)] +mod path_test { + use crate::path_absolute; + + #[cfg(not(any(target_os = "windows", target_os = "macos")))] + fn cd_to_tmp() { + std::env::set_current_dir("/tmp").unwrap(); + } + + #[test] + #[cfg(not(any(target_os = "windows", target_os = "macos")))] + fn relative_path_converts() { + cd_to_tmp(); + assert_eq!("/tmp/foo/bar/baz.txt", path_absolute("foo/bar/baz.txt")); + } + + #[test] + #[cfg(not(any(target_os = "windows", target_os = "macos")))] + fn relative_path_converts_with_one_dot() { + cd_to_tmp(); + assert_eq!("/tmp/foo/bar/baz.txt", path_absolute("./foo/bar/baz.txt")); + } + + #[test] + #[cfg(not(any(target_os = "windows", target_os = "macos")))] + fn relative_path_converts_with_double_dot() { + cd_to_tmp(); + assert_eq!( + "/tmp/../foo/bar/baz.txt", + path_absolute("../foo/bar/baz.txt") + ); + } + + #[test] + #[cfg(not(any(target_os = "windows", target_os = "macos")))] + fn absolute_path_unchanged() { + cd_to_tmp(); + assert_eq!("/tmp/foo/bar", path_absolute("/tmp/foo/bar")); + } + + #[test] + #[should_panic(expected = "cannot make an empty path absolute")] + fn empty_path_panic() { + let _ = path_absolute(""); + } +} diff --git a/rust/compactor/tests/compactor_cli.rs b/rust/apps/tests/compactor_cli.rs similarity index 94% rename from rust/compactor/tests/compactor_cli.rs rename to rust/apps/tests/compactor_cli.rs index b3d04ceff71..f9235544ede 100644 --- a/rust/compactor/tests/compactor_cli.rs +++ b/rust/apps/tests/compactor_cli.rs @@ -20,7 +20,7 @@ use std::process::Command; // Run programs #[test] fn invalid_region_maxs() -> Result<()> { - let mut cmd = Command::cargo_bin("main")?; + let mut cmd = Command::cargo_bin("compact")?; cmd.args([ "/tmp/output.parquet", "/tmp/input.parquet", @@ -43,7 +43,7 @@ fn invalid_region_maxs() -> Result<()> { #[test] fn invalid_region_mins() -> Result<()> { - let mut cmd = Command::cargo_bin("main")?; + let mut cmd = Command::cargo_bin("compact")?; cmd.args([ "/tmp/output.parquet", "/tmp/input.parquet", From 53eced8af05d5ad0dcf659e2994d46aeae86d0f2 Mon Sep 17 00:00:00 2001 From: m09526 Date: Tue, 2 Sep 2025 10:26:53 +0000 Subject: [PATCH 16/28] Compile issue fixed --- rust/sleeper_core/tests/compaction_test.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/rust/sleeper_core/tests/compaction_test.rs b/rust/sleeper_core/tests/compaction_test.rs index 6d295096595..afcd522b574 100644 --- a/rust/sleeper_core/tests/compaction_test.rs +++ b/rust/sleeper_core/tests/compaction_test.rs @@ -230,7 +230,7 @@ async fn should_merge_empty_files() -> Result<(), Error> { input_files_sorted: true, row_key_cols: row_key_cols(["key"]), region: SleeperPartitionRegion::new(single_int_range("key", 0, 5)), - output: OperationOutput::File { + output: CompletionOptions::File { output_file: output.clone(), opts: SleeperParquetOptions::default(), }, From edde88ff8a16eeb529e4dabab8497f45d7c88868 Mon Sep 17 00:00:00 2001 From: m09526 Date: Tue, 2 Sep 2025 13:58:12 +0000 Subject: [PATCH 17/28] SOme unit tests --- rust/sleeper_core/src/lib.rs | 162 +++++++++++++++++++++++++++++++++++ 1 file changed, 162 insertions(+) diff --git a/rust/sleeper_core/src/lib.rs b/rust/sleeper_core/src/lib.rs index 4cbba879412..2998f0e4f5f 100644 --- a/rust/sleeper_core/src/lib.rs +++ b/rust/sleeper_core/src/lib.rs @@ -377,3 +377,165 @@ pub fn to_s3_config(aws_config: &AwsConfig) -> AmazonS3Builder { .with_endpoint(&aws_config.endpoint) .with_allow_http(aws_config.allow_http) } + +#[cfg(test)] +mod tests { + use super::*; + use url::Url; + + #[test] + fn test_convert_s3a_scheme_in_input_files() { + // Given + let input_files = vec![Url::parse("s3a://bucket/key1").unwrap(), Url::parse("s3a://bucket/key2").unwrap()]; + let output = CompletionOptions::File { + output_file: Url::parse("https://example.com/output").unwrap(), + opts: SleeperParquetOptions::default(), + }; + + // When + let (new_files, new_output) = normalise_s3a_urls(input_files, output); + + // Then + for url in new_files { + assert_eq!(url.scheme(), "s3"); + } + if let CompletionOptions::File { output_file, .. } = new_output { + assert_eq!(output_file.scheme(), "https"); // unchanged + } else { + panic!("Output option changed unexpectedly") + } + } + + #[test] + fn test_no_change_for_non_s3a_urls() { + // Given + let input_files = vec![Url::parse("https://example.com/key").unwrap()]; + let output = CompletionOptions::File { + output_file: Url::parse("https://example.com/output").unwrap(), + opts: SleeperParquetOptions::default(), + }; + + // When + let (new_files, new_output) = normalise_s3a_urls(input_files, output); + + // Then + assert_eq!(new_files[0].scheme(), "https"); + if let CompletionOptions::File { output_file, .. } = new_output { + assert_eq!(output_file.scheme(), "https"); // unchanged + } else { + panic!("Output option changed unexpectedly") + } + } + + #[test] + fn test_convert_output_scheme_when_s3a() { + // Given + let input_files = vec![Url::parse("https://example.com/key").unwrap()]; + let output = CompletionOptions::File { + output_file: Url::parse("s3a://bucket/output").unwrap(), + opts: SleeperParquetOptions::default(), + }; + + // When + let (_, new_output) = normalise_s3a_urls(input_files, output); + + // Then + if let CompletionOptions::File { output_file, .. } = new_output { + assert_eq!(output_file.scheme(), "s3"); + } else { + panic!("Unexpected output option type") + } + } + + #[test] + fn test_empty_input_files() { + // Given + let input_files: Vec = vec![]; + let output = CompletionOptions::File { + output_file: Url::parse("https://example.com/output").unwrap(), + opts: SleeperParquetOptions::default(), + }; + + // When + let (new_files, _) = normalise_s3a_urls(input_files, output); + + // Then + assert!(new_files.is_empty()); + } + + #[test] + fn test_normalise_s3a_urls_arrow_record_batch() { + // Given + let input_files = vec![Url::parse("s3a://bucket/key1").unwrap(), Url::parse("s3a://bucket/key2").unwrap()]; + let output = CompletionOptions::ArrowRecordBatch; + + // When + let (new_files, new_output) = normalise_s3a_urls(input_files.clone(), output); + + // Then + for url in new_files { + assert_eq!(url.scheme(), "s3"); + } + + match new_output { + CompletionOptions::ArrowRecordBatch => {}, + _ => panic!("Output should be ArrowRecordBatch") + } + } + + #[test] + fn test_validate_no_input_files() { + // Given + let input_files: Vec = vec![]; + let row_key_cols = vec!["key".to_string()]; + let region = SleeperPartitionRegion::default(); + + // When + let result = validate(&input_files, &row_key_cols, ®ion); + + // Then + assert!(result.is_err()); + assert_eq!(result.err().unwrap().to_string(), "No input paths supplied"); + } + + #[test] + fn test_validate_row_key_mismatch() { + // Given + let input_files = vec![Url::parse("file:///path/to/file.parquet").unwrap()]; + let row_key_cols = vec!["key1".to_string(), "key2".to_string()]; + let mut region = SleeperPartitionRegion::default(); + region.region.insert("col".to_string(), ColRange { + lower: PartitionBound::String("a"), + lower_inclusive: true, + upper: PartitionBound::String("z"), + upper_inclusive: true, + }); + + // When + let result = validate(&input_files, &row_key_cols, ®ion); + + // Then + assert!(result.is_err()); + assert!(result.err().unwrap().to_string().contains("Length mismatch")); + } + + #[test] + fn test_validate_success() { + // Given + let input_files = vec![Url::parse("file:///path/to/file.parquet").unwrap()]; + let row_key_cols = vec!["key".to_string()]; + let mut region = SleeperPartitionRegion::default(); + region.region.insert("col".to_string(), ColRange { + lower: PartitionBound::String("a"), + lower_inclusive: true, + upper: PartitionBound::String("z"), + upper_inclusive: true, + }); + + // When + let result = validate(&input_files, &row_key_cols, ®ion); + + // Then + assert!(result.is_ok()); + } +} From 1f4544ae6718f6929d640e992905b851bdb03391 Mon Sep 17 00:00:00 2001 From: m09526 Date: Tue, 2 Sep 2025 14:49:11 +0000 Subject: [PATCH 18/28] Split util tests out --- rust/Cargo.lock | 13 ++++++++ rust/sleeper_core/Cargo.toml | 3 +- rust/sleeper_core/test_util/Cargo.toml | 33 +++++++++++++++++++ .../src/lib.rs} | 0 rust/sleeper_core/tests/compaction_test.rs | 4 +-- rust/sleeper_core/tests/query_test.rs | 15 +++++++++ 6 files changed, 64 insertions(+), 4 deletions(-) create mode 100644 rust/sleeper_core/test_util/Cargo.toml rename rust/sleeper_core/{tests/compaction_helpers.rs => test_util/src/lib.rs} (100%) create mode 100644 rust/sleeper_core/tests/query_test.rs diff --git a/rust/Cargo.lock b/rust/Cargo.lock index 7078862aa2c..e0fe1a05718 100644 --- a/rust/Cargo.lock +++ b/rust/Cargo.lock @@ -4264,6 +4264,7 @@ dependencies = [ "rust_sketch", "tempfile", "test-log", + "test_util", "tokio", "tokio-test", "url", @@ -4456,6 +4457,18 @@ dependencies = [ "syn", ] +[[package]] +name = "test_util" +version = "0.33.0-SNAPSHOT" +dependencies = [ + "arrow", + "color-eyre", + "datafusion", + "sleeper_core", + "tempfile", + "url", +] + [[package]] name = "testing_logger" version = "0.1.1" diff --git a/rust/sleeper_core/Cargo.toml b/rust/sleeper_core/Cargo.toml index cceaad8d8c5..53de212f861 100644 --- a/rust/sleeper_core/Cargo.toml +++ b/rust/sleeper_core/Cargo.toml @@ -50,4 +50,5 @@ url = { workspace = true } [dev-dependencies] tempfile = { workspace = true } test-log = { workspace = true } -tokio-test = { workspace = true } \ No newline at end of file +tokio-test = { workspace = true } +test_util = { path = "test_util" } \ No newline at end of file diff --git a/rust/sleeper_core/test_util/Cargo.toml b/rust/sleeper_core/test_util/Cargo.toml new file mode 100644 index 00000000000..8034152e6c3 --- /dev/null +++ b/rust/sleeper_core/test_util/Cargo.toml @@ -0,0 +1,33 @@ +# Copyright 2022-2025 Crown Copyright +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +[package] +name = "test_util" +description = "Test utilities for integration tests." +keywords = ["sleeper"] +categories = ["science"] +version = { workspace = true } +edition = { workspace = true } +repository = { workspace = true } +license = { workspace = true } +authors = { workspace = true } +rust-version = { workspace = true } +publish = false + +[dependencies] +arrow = { workspace = true } +color-eyre = { workspace = true } +datafusion = { workspace = true } +url = { workspace = true } +tempfile = { workspace = true } +sleeper_core = { path = "../" } diff --git a/rust/sleeper_core/tests/compaction_helpers.rs b/rust/sleeper_core/test_util/src/lib.rs similarity index 100% rename from rust/sleeper_core/tests/compaction_helpers.rs rename to rust/sleeper_core/test_util/src/lib.rs diff --git a/rust/sleeper_core/tests/compaction_test.rs b/rust/sleeper_core/tests/compaction_test.rs index afcd522b574..a48895cef39 100644 --- a/rust/sleeper_core/tests/compaction_test.rs +++ b/rust/sleeper_core/tests/compaction_test.rs @@ -13,11 +13,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -mod compaction_helpers; - use arrow::datatypes::{DataType, Field, Schema}; use color_eyre::eyre::Error; -use compaction_helpers::*; +use test_util::*; use sleeper_core::{ CommonConfig, CompletionOptions, SleeperParquetOptions, SleeperPartitionRegion, run_compaction, }; diff --git a/rust/sleeper_core/tests/query_test.rs b/rust/sleeper_core/tests/query_test.rs new file mode 100644 index 00000000000..3e58369840d --- /dev/null +++ b/rust/sleeper_core/tests/query_test.rs @@ -0,0 +1,15 @@ +/* + * Copyright 2022-2025 Crown Copyright + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ From 4c5897e767b553d71e61785b71c6ea30a4e951a9 Mon Sep 17 00:00:00 2001 From: m09526 Date: Tue, 2 Sep 2025 16:20:40 +0000 Subject: [PATCH 19/28] Unit tests written --- rust/Cargo.lock | 96 ++++--- rust/Cargo.toml | 10 +- rust/sleeper_core/src/lib.rs | 56 ++-- rust/sleeper_core/test_util/Cargo.toml | 1 + rust/sleeper_core/test_util/src/lib.rs | 56 +++- rust/sleeper_core/tests/query_test.rs | 383 +++++++++++++++++++++++++ 6 files changed, 523 insertions(+), 79 deletions(-) diff --git a/rust/Cargo.lock b/rust/Cargo.lock index e0fe1a05718..b455cd1ecc6 100644 --- a/rust/Cargo.lock +++ b/rust/Cargo.lock @@ -473,9 +473,9 @@ checksum = "ace50bade8e6234aa140d9a2f552bbee1db4d353f69b8217bc503490fc1a9f26" [[package]] name = "aws-config" -version = "1.8.5" +version = "1.8.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c478f5b10ce55c9a33f87ca3404ca92768b144fc1bfdede7c0121214a8283a25" +checksum = "8bc1b40fb26027769f16960d2f4a6bc20c4bb755d403e552c8c1a73af433c246" dependencies = [ "aws-credential-types", "aws-runtime", @@ -503,9 +503,9 @@ dependencies = [ [[package]] name = "aws-credential-types" -version = "1.2.5" +version = "1.2.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1541072f81945fa1251f8795ef6c92c4282d74d59f88498ae7d4bf00f0ebdad9" +checksum = "d025db5d9f52cbc413b167136afb3d8aeea708c0d8884783cf6253be5e22f6f2" dependencies = [ "aws-smithy-async", "aws-smithy-runtime-api", @@ -562,9 +562,9 @@ dependencies = [ [[package]] name = "aws-sdk-sso" -version = "1.80.0" +version = "1.83.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e822be5d4ed48fa7adc983de1b814dea33a5460c7e0e81b053b8d2ca3b14c354" +checksum = "643cd43af212d2a1c4dedff6f044d7e1961e5d9e7cfe773d70f31d9842413886" dependencies = [ "aws-credential-types", "aws-runtime", @@ -584,9 +584,9 @@ dependencies = [ [[package]] name = "aws-sdk-ssooidc" -version = "1.81.0" +version = "1.84.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "66aa7b30f1fac6e02ca26e3839fa78db3b94f6298a6e7a6208fb59071d93a87e" +checksum = "20ec4a95bd48e0db7a424356a161f8d87bd6a4f0af37204775f0da03d9e39fc3" dependencies = [ "aws-credential-types", "aws-runtime", @@ -606,9 +606,9 @@ dependencies = [ [[package]] name = "aws-sdk-sts" -version = "1.82.0" +version = "1.85.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2194426df72592f91df0cda790cb1e571aa87d66cecfea59a64031b58145abe3" +checksum = "410309ad0df4606bc721aff0d89c3407682845453247213a0ccc5ff8801ee107" dependencies = [ "aws-credential-types", "aws-runtime", @@ -682,9 +682,9 @@ dependencies = [ [[package]] name = "aws-smithy-http-client" -version = "1.0.6" +version = "1.1.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f108f1ca850f3feef3009bdcc977be201bca9a91058864d9de0684e64514bee0" +checksum = "147e8eea63a40315d704b97bf9bc9b8c1402ae94f89d5ad6f7550d963309da1b" dependencies = [ "aws-smithy-async", "aws-smithy-runtime-api", @@ -699,15 +699,16 @@ dependencies = [ "rustls-native-certs", "rustls-pki-types", "tokio", + "tokio-rustls", "tower", "tracing", ] [[package]] name = "aws-smithy-json" -version = "0.61.4" +version = "0.61.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a16e040799d29c17412943bdbf488fd75db04112d0c0d4b9290bacf5ae0014b9" +checksum = "eaa31b350998e703e9826b2104dd6f63be0508666e1aba88137af060e8944047" dependencies = [ "aws-smithy-types", ] @@ -733,9 +734,9 @@ dependencies = [ [[package]] name = "aws-smithy-runtime" -version = "1.8.6" +version = "1.9.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9e107ce0783019dbff59b3a244aa0c114e4a8c9d93498af9162608cd5474e796" +checksum = "d3946acbe1ead1301ba6862e712c7903ca9bb230bdf1fbd1b5ac54158ef2ab1f" dependencies = [ "aws-smithy-async", "aws-smithy-http", @@ -757,9 +758,9 @@ dependencies = [ [[package]] name = "aws-smithy-runtime-api" -version = "1.8.7" +version = "1.9.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "75d52251ed4b9776a3e8487b2a01ac915f73b2da3af8fc1e77e0fce697a550d4" +checksum = "07f5e0fc8a6b3f2303f331b94504bbf754d85488f402d6f1dd7a6080f99afe56" dependencies = [ "aws-smithy-async", "aws-smithy-types", @@ -1003,18 +1004,18 @@ dependencies = [ [[package]] name = "camino" -version = "1.1.9" +version = "1.1.12" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8b96ec4966b5813e2c0507c1f86115c8c5abaadc3980879c3424042a02fd1ad3" +checksum = "dd0b03af37dad7a14518b7691d81acb0f8222604ad3d1b02f6b4bed5188c0cd5" dependencies = [ "serde", ] [[package]] name = "cargo-platform" -version = "0.2.0" +version = "0.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "84982c6c0ae343635a3a4ee6dedef965513735c8b183caa7289fa6e27399ebd4" +checksum = "8abf5d501fd757c2d2ee78d0cc40f606e92e3a63544420316565556ed28485e2" dependencies = [ "serde", ] @@ -1037,9 +1038,9 @@ dependencies = [ [[package]] name = "cargo_metadata" -version = "0.21.0" +version = "0.22.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5cfca2aaa699835ba88faf58a06342a314a950d2b9686165e038286c30316868" +checksum = "0c3f56c207c76c07652489840ff98687dcf213de178ac0974660d6fefeaf5ec6" dependencies = [ "camino", "cargo-platform", @@ -1337,9 +1338,9 @@ dependencies = [ [[package]] name = "cxx" -version = "1.0.170" +version = "1.0.174" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "77377ef8d2a668ad5ad5c58a1d0b172836495dffd3b10e0488a41c9b288b3a0e" +checksum = "e5ba77f286ce5c44c7ba02de894b057bc0a605a210e3d81fa83b92d94586c0e1" dependencies = [ "cc", "cxxbridge-cmd", @@ -1351,9 +1352,9 @@ dependencies = [ [[package]] name = "cxx-build" -version = "1.0.170" +version = "1.0.174" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d3579771cb1e6b9cd41903887f3ea98bc2e0761f975bf55adcc0c1e62ab4521f" +checksum = "0c56fdf6fba27288d1fda3384062692e66dc40ca41bafd15f616dd4e8b0ac909" dependencies = [ "cc", "codespan-reporting", @@ -1366,9 +1367,9 @@ dependencies = [ [[package]] name = "cxxbridge-cmd" -version = "1.0.170" +version = "1.0.174" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "dea3ad43f199bbb9df0547f5db39896604b9554784d963d73a8c7f83fc6fecb1" +checksum = "19ade5eb6d6e6ef9c5631eff7e4f74e0e7109140e775f124d76904c0e5e6a202" dependencies = [ "clap", "codespan-reporting", @@ -1380,15 +1381,15 @@ dependencies = [ [[package]] name = "cxxbridge-flags" -version = "1.0.170" +version = "1.0.174" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e213bf54b95ba7bceebbbdaf923dfe32fdbd630c685fedf5c47132da64df6aa1" +checksum = "99f99fe2f3f76a2ba40c5431f854efe3725c19a89f4d59966bca3ec561be940e" [[package]] name = "cxxbridge-macro" -version = "1.0.170" +version = "1.0.174" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9caa03943851c364d0ff229bd11ba07f3144673cca9feb52b9ca09817c8a239e" +checksum = "2b6e5fa0545804d2d8d398a1e995203a1f2403a9f0651d50546462e61a28340e" dependencies = [ "indexmap", "proc-macro2", @@ -2373,9 +2374,9 @@ checksum = "a8d1add55171497b4705a648c6b583acafb01d58050a51727785f0b2c8e0a2b2" [[package]] name = "h2" -version = "0.4.10" +version = "0.4.12" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a9421a676d1b147b16b82c9225157dc629087ef8ec4d5e2960f9437a90dac0a5" +checksum = "f3c0b69cfcb4e1b9f1bf2f53f95f766e4661169728ec61cd3fe5a0166f2d1386" dependencies = [ "atomic-waker", "bytes", @@ -2576,19 +2577,23 @@ dependencies = [ [[package]] name = "hyper-util" -version = "0.1.11" +version = "0.1.16" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "497bbc33a26fdd4af9ed9c70d63f61cf56a938375fbb32df34db9b1cd6d643f2" +checksum = "8d9b05277c7e8da2c93a568989bb6207bef0112e8d17df7a6eda4a3cf143bc5e" dependencies = [ + "base64", "bytes", "futures-channel", + "futures-core", "futures-util", "http 1.3.1", "http-body 1.0.1", "hyper", + "ipnet", "libc", + "percent-encoding", "pin-project-lite", - "socket2 0.5.9", + "socket2 0.6.0", "tokio", "tower-service", "tracing", @@ -2987,9 +2992,9 @@ dependencies = [ [[package]] name = "link-cplusplus" -version = "1.0.10" +version = "1.0.11" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4a6f6da007f968f9def0d65a05b187e2960183de70c160204ecfccf0ee330212" +checksum = "8c349c75e1ab4a03bd6b33fe6cbd3c479c5dd443e44ad732664d72cb0e755475" dependencies = [ "cc", ] @@ -3964,9 +3969,9 @@ dependencies = [ [[package]] name = "rustls" -version = "0.23.27" +version = "0.23.31" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "730944ca083c1c233a75c09f199e973ca499344a2b7ba9e755c457e86fb4a321" +checksum = "c0ebcbd2f03de0fc1122ad9bb24b127a5a6cd51d72604a3f3c50ac459762b6cc" dependencies = [ "aws-lc-rs", "once_cell", @@ -4010,9 +4015,9 @@ dependencies = [ [[package]] name = "rustls-webpki" -version = "0.103.3" +version = "0.103.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e4a72fe2bcf7a6ac6fd7d0b9e5cb68aeb7d4c0a0271730218b3e92d43b4eb435" +checksum = "0a17884ae0c1b773f1ccd2bd4a8c72f16da897310a98b0e84bf349ad5ead92fc" dependencies = [ "aws-lc-rs", "ring", @@ -4464,6 +4469,7 @@ dependencies = [ "arrow", "color-eyre", "datafusion", + "futures", "sleeper_core", "tempfile", "url", diff --git a/rust/Cargo.toml b/rust/Cargo.toml index ccee03b3521..a4b671b4b78 100644 --- a/rust/Cargo.toml +++ b/rust/Cargo.toml @@ -28,16 +28,16 @@ authors = ["GCHQ"] arrow = { version = "55.1.0" } # Batch of reading from Parquet files assert_cmd = { version = "2.0.17" } # CLI interface testing async-trait = { version = "0.1.89" } -aws-config = { version = "1.8.5" } # Credential loading -aws-credential-types = { version = "1.2.5" } # Credential provider types +aws-config = { version = "1.8.6" } # Credential loading +aws-credential-types = { version = "1.2.6" } # Credential provider types aws-types = { version = "1.3.8" } # for Region bytes = { version = "1.10.1" } # Byte buffer for S3 uploading -cargo_metadata = { version = "0.21.0" } # Access cargo metadata programmtically +cargo_metadata = { version = "0.22.0" } # Access cargo metadata programmtically chrono = { version = "0.4.41" } # Log helper clap = { version = "4.5.46" } # Cmd line args processing color-eyre = { version = "0.6.5" } # Colourised version of `anyhow` -cxx = { version = "1.0.170" } # Exception handling for Rust -cxx-build = { version = "1.0.170" } +cxx = { version = "1.0.174" } # Exception handling for Rust +cxx-build = { version = "1.0.174" } datafusion = { version = "48.0.1" } env_logger = { version = "0.11.8" } # Standard logging to stderr futures = { version = "0.3.31" } # Async processing diff --git a/rust/sleeper_core/src/lib.rs b/rust/sleeper_core/src/lib.rs index 2998f0e4f5f..a9509ef2554 100644 --- a/rust/sleeper_core/src/lib.rs +++ b/rust/sleeper_core/src/lib.rs @@ -380,13 +380,18 @@ pub fn to_s3_config(aws_config: &AwsConfig) -> AmazonS3Builder { #[cfg(test)] mod tests { + use std::collections::HashMap; + use super::*; use url::Url; #[test] fn test_convert_s3a_scheme_in_input_files() { // Given - let input_files = vec![Url::parse("s3a://bucket/key1").unwrap(), Url::parse("s3a://bucket/key2").unwrap()]; + let input_files = vec![ + Url::parse("s3a://bucket/key1").unwrap(), + Url::parse("s3a://bucket/key2").unwrap(), + ]; let output = CompletionOptions::File { output_file: Url::parse("https://example.com/output").unwrap(), opts: SleeperParquetOptions::default(), @@ -466,7 +471,10 @@ mod tests { #[test] fn test_normalise_s3a_urls_arrow_record_batch() { // Given - let input_files = vec![Url::parse("s3a://bucket/key1").unwrap(), Url::parse("s3a://bucket/key2").unwrap()]; + let input_files = vec![ + Url::parse("s3a://bucket/key1").unwrap(), + Url::parse("s3a://bucket/key2").unwrap(), + ]; let output = CompletionOptions::ArrowRecordBatch; // When @@ -478,8 +486,8 @@ mod tests { } match new_output { - CompletionOptions::ArrowRecordBatch => {}, - _ => panic!("Output should be ArrowRecordBatch") + CompletionOptions::ArrowRecordBatch => {} + _ => panic!("Output should be ArrowRecordBatch"), } } @@ -503,20 +511,28 @@ mod tests { // Given let input_files = vec![Url::parse("file:///path/to/file.parquet").unwrap()]; let row_key_cols = vec!["key1".to_string(), "key2".to_string()]; - let mut region = SleeperPartitionRegion::default(); - region.region.insert("col".to_string(), ColRange { - lower: PartitionBound::String("a"), - lower_inclusive: true, - upper: PartitionBound::String("z"), - upper_inclusive: true, - }); + let region = SleeperPartitionRegion::new(HashMap::from([( + "col".to_string(), + ColRange { + lower: PartitionBound::String("a"), + lower_inclusive: true, + upper: PartitionBound::String("z"), + upper_inclusive: true, + }, + )])); // When let result = validate(&input_files, &row_key_cols, ®ion); // Then assert!(result.is_err()); - assert!(result.err().unwrap().to_string().contains("Length mismatch")); + assert!( + result + .err() + .unwrap() + .to_string() + .contains("Length mismatch") + ); } #[test] @@ -524,13 +540,15 @@ mod tests { // Given let input_files = vec![Url::parse("file:///path/to/file.parquet").unwrap()]; let row_key_cols = vec!["key".to_string()]; - let mut region = SleeperPartitionRegion::default(); - region.region.insert("col".to_string(), ColRange { - lower: PartitionBound::String("a"), - lower_inclusive: true, - upper: PartitionBound::String("z"), - upper_inclusive: true, - }); + let region = SleeperPartitionRegion::new(HashMap::from([( + "col".to_string(), + ColRange { + lower: PartitionBound::String("a"), + lower_inclusive: true, + upper: PartitionBound::String("z"), + upper_inclusive: true, + }, + )])); // When let result = validate(&input_files, &row_key_cols, ®ion); diff --git a/rust/sleeper_core/test_util/Cargo.toml b/rust/sleeper_core/test_util/Cargo.toml index 8034152e6c3..f0a7bb9e6d0 100644 --- a/rust/sleeper_core/test_util/Cargo.toml +++ b/rust/sleeper_core/test_util/Cargo.toml @@ -26,6 +26,7 @@ publish = false [dependencies] arrow = { workspace = true } +futures = { workspace = true } color-eyre = { workspace = true } datafusion = { workspace = true } url = { workspace = true } diff --git a/rust/sleeper_core/test_util/src/lib.rs b/rust/sleeper_core/test_util/src/lib.rs index 5562d1a250c..59e57a0b4ed 100644 --- a/rust/sleeper_core/test_util/src/lib.rs +++ b/rust/sleeper_core/test_util/src/lib.rs @@ -18,14 +18,20 @@ use arrow::{ datatypes::{DataType, Field, Schema}, }; use color_eyre::eyre::{Error, OptionExt, eyre}; -use datafusion::parquet::{ - arrow::{ - ArrowWriter, - arrow_reader::{ArrowReaderMetadata, ArrowReaderOptions, ParquetRecordBatchReaderBuilder}, +use datafusion::{ + execution::SendableRecordBatchStream, + parquet::{ + arrow::{ + ArrowWriter, + arrow_reader::{ + ArrowReaderMetadata, ArrowReaderOptions, ParquetRecordBatchReaderBuilder, + }, + }, + basic::{Compression, ZstdLevel}, + file::properties::WriterProperties, }, - basic::{Compression, ZstdLevel}, - file::properties::WriterProperties, }; +use futures::StreamExt; use sleeper_core::{ColRange, DataSketchVariant, PartitionBound, deserialise_sketches}; use std::{collections::HashMap, fs::File, sync::Arc}; use tempfile::TempDir; @@ -84,7 +90,7 @@ pub fn read_file_of_ints(path: &Url, field_name: &str) -> Result, Error let file = File::open(path.path())?; let mut data: Vec = Vec::new(); let metadata = ArrowReaderMetadata::load(&file, ArrowReaderOptions::default())?; - check_non_null_field(field_name, &DataType::Int32, &metadata)?; + check_non_null_field(field_name, &DataType::Int32, metadata.schema().as_ref())?; for result in ParquetRecordBatchReaderBuilder::new_with_metadata(file, metadata).build()? { data.extend(get_int_array(field_name, &result?)?.values()); } @@ -124,7 +130,7 @@ pub fn read_file_of_int_fields( let mut data: Vec<[i32; N]> = Vec::new(); let metadata = ArrowReaderMetadata::load(&file, ArrowReaderOptions::default())?; for field_name in field_names { - check_non_null_field(field_name, &DataType::Int32, &metadata)?; + check_non_null_field(field_name, &DataType::Int32, &metadata.schema())?; } for result in ParquetRecordBatchReaderBuilder::new_with_metadata(file, metadata).build()? { let batch = result?; @@ -134,6 +140,23 @@ pub fn read_file_of_int_fields( Ok(data) } +#[allow(clippy::missing_errors_doc)] +pub async fn read_batches_of_int_fields( + mut stream: SendableRecordBatchStream, + field_names: [&str; N], +) -> Result, Error> { + let schema = stream.schema(); + for field_name in field_names { + check_non_null_field(field_name, &DataType::Int32, schema.as_ref())?; + } + let mut data: Vec<[i32; N]> = Vec::new(); + while let Some(Ok(batch)) = stream.next().await { + let arrays: Vec<&Int32Array> = get_int_arrays(&batch, field_names)?; + data.extend((0..batch.num_rows()).map(|row_number| read_row(row_number, &arrays))); + } + Ok(data) +} + fn get_int_arrays<'b, const N: usize>( batch: &'b RecordBatch, field_names: [&str; N], @@ -164,9 +187,9 @@ fn get_int_array<'b>(field_name: &str, batch: &'b RecordBatch) -> Result<&'b Int fn check_non_null_field( field_name: &str, field_type: &DataType, - metadata: &ArrowReaderMetadata, + schema: &Schema, ) -> Result<(), Error> { - let field = metadata.schema().field_with_name(field_name)?; + let field = schema.field_with_name(field_name)?; if field.data_type() != field_type { Err(eyre!( "Expected field {} to be of type {}, found {}", @@ -203,3 +226,16 @@ pub fn int_range<'r>(min: i32, max: i32) -> ColRange<'r> { upper_inclusive: false, } } + +#[macro_export] +macro_rules! assert_error { + ($err_expr: expr, $err_type: path, $err_contents: expr) => { + let result = if let Err($err_type(err)) = $err_expr { + assert_eq!(err, $err_contents); + true + } else { + false + }; + assert!(result, "Expected different error type"); + }; +} diff --git a/rust/sleeper_core/tests/query_test.rs b/rust/sleeper_core/tests/query_test.rs index 3e58369840d..921f748400a 100644 --- a/rust/sleeper_core/tests/query_test.rs +++ b/rust/sleeper_core/tests/query_test.rs @@ -13,3 +13,386 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + +use color_eyre::eyre::{Error, bail}; +use sleeper_core::{ + CommonConfig, CompletedOutput, CompletionOptions, LeafPartitionQueryConfig, + SleeperParquetOptions, SleeperPartitionRegion, run_query, +}; +use tempfile::tempdir; +use test_util::*; + +#[tokio::test] +async fn should_return_subset_results_with_query_subset_of_partition() -> Result<(), Error> { + // Given + let dir = tempdir()?; + let file_1 = file(&dir, "file1.parquet"); + let file_2 = file(&dir, "file2.parquet"); + + write_file_of_ints(&file_1, "key", vec![1, 3])?; + write_file_of_ints(&file_2, "key", vec![2, 4])?; + + let input = CommonConfig { + input_files: Vec::from([file_1, file_2]), + input_files_sorted: true, + row_key_cols: row_key_cols(["key"]), + region: SleeperPartitionRegion::new(single_int_range("key", 0, 5)), + output: CompletionOptions::ArrowRecordBatch, + ..Default::default() + }; + + let query_config = LeafPartitionQueryConfig { + common: input, + explain_plans: false, + write_quantile_sketch: false, + ranges: vec![SleeperPartitionRegion::new(single_int_range("key", 2, 4))], + }; + + // When + let result = run_query(&query_config).await?; + + // Then + let CompletedOutput::ArrowRecordBatch(stream) = result else { + bail!("Expected arrow record batch stream output"); + }; + + assert_eq!( + read_batches_of_int_fields(stream, ["key"]).await?, + vec![[2], [3]] + ); + Ok(()) +} + +#[tokio::test] +async fn should_return_subset_results_with_query_subset_of_partition_unsorted_input() +-> Result<(), Error> { + // Given + let dir = tempdir()?; + let file_1 = file(&dir, "file1.parquet"); + let file_2 = file(&dir, "file2.parquet"); + + write_file_of_ints(&file_1, "key", vec![7, 3, 5, 1])?; + write_file_of_ints(&file_2, "key", vec![8, 6, 2, 4])?; + + let input = CommonConfig { + input_files: Vec::from([file_1, file_2]), + input_files_sorted: false, + row_key_cols: row_key_cols(["key"]), + region: SleeperPartitionRegion::new(single_int_range("key", 1, 7)), + output: CompletionOptions::ArrowRecordBatch, + ..Default::default() + }; + + let query_config = LeafPartitionQueryConfig { + common: input, + explain_plans: false, + write_quantile_sketch: false, + ranges: vec![SleeperPartitionRegion::new(single_int_range("key", 2, 6))], + }; + + // When + let result = run_query(&query_config).await?; + + // Then + let CompletedOutput::ArrowRecordBatch(stream) = result else { + bail!("Expected arrow record batch stream output"); + }; + + assert_eq!( + read_batches_of_int_fields(stream, ["key"]).await?, + vec![[2], [3], [4], [5]] + ); + Ok(()) +} + +#[tokio::test] +async fn should_return_subset_results_with_overlapping_query_and_partition_range() +-> Result<(), Error> { + // Given + let dir = tempdir()?; + let file_1 = file(&dir, "file1.parquet"); + let file_2 = file(&dir, "file2.parquet"); + + write_file_of_ints(&file_1, "key", vec![1, 3, 5, 7, 9])?; + write_file_of_ints(&file_2, "key", vec![2, 4, 6, 8, 10])?; + + let input = CommonConfig { + input_files: Vec::from([file_1, file_2]), + input_files_sorted: true, + row_key_cols: row_key_cols(["key"]), + region: SleeperPartitionRegion::new(single_int_range("key", 0, 6)), + output: CompletionOptions::ArrowRecordBatch, + ..Default::default() + }; + + let query_config = LeafPartitionQueryConfig { + common: input, + explain_plans: false, + write_quantile_sketch: false, + ranges: vec![SleeperPartitionRegion::new(single_int_range("key", 2, 9))], + }; + + // When + let result = run_query(&query_config).await?; + + // Then + let CompletedOutput::ArrowRecordBatch(stream) = result else { + bail!("Expected arrow record batch stream output"); + }; + + assert_eq!( + read_batches_of_int_fields(stream, ["key"]).await?, + vec![[2], [3], [4], [5]] + ); + Ok(()) +} + +#[tokio::test] +async fn should_return_zero_results_with_non_overlapping_query_and_partition_range() +-> Result<(), Error> { + // Given + let dir = tempdir()?; + let file_1 = file(&dir, "file1.parquet"); + let file_2 = file(&dir, "file2.parquet"); + + write_file_of_ints(&file_1, "key", vec![1, 3, 5, 7, 9])?; + write_file_of_ints(&file_2, "key", vec![2, 4, 6, 8, 10])?; + + let input = CommonConfig { + input_files: Vec::from([file_1, file_2]), + input_files_sorted: true, + row_key_cols: row_key_cols(["key"]), + region: SleeperPartitionRegion::new(single_int_range("key", 0, 3)), + output: CompletionOptions::ArrowRecordBatch, + ..Default::default() + }; + + let query_config = LeafPartitionQueryConfig { + common: input, + explain_plans: false, + write_quantile_sketch: false, + ranges: vec![SleeperPartitionRegion::new(single_int_range("key", 6, 9))], + }; + + // When + let result = run_query(&query_config).await?; + + // Then + let CompletedOutput::ArrowRecordBatch(stream) = result else { + bail!("Expected arrow record batch stream output"); + }; + + assert_eq!( + read_batches_of_int_fields(stream, ["key"]).await?, + Vec::<[i32; 1]>::new() + ); + Ok(()) +} + +#[tokio::test] +async fn should_return_results_from_two_overlapping_query_ranges() -> Result<(), Error> { + // Given + let dir = tempdir()?; + let file_1 = file(&dir, "file1.parquet"); + let file_2 = file(&dir, "file2.parquet"); + + write_file_of_ints(&file_1, "key", vec![1, 3, 5, 7, 9])?; + write_file_of_ints(&file_2, "key", vec![2, 4, 6, 8, 10])?; + + let input = CommonConfig { + input_files: Vec::from([file_1, file_2]), + input_files_sorted: true, + row_key_cols: row_key_cols(["key"]), + region: SleeperPartitionRegion::new(single_int_range("key", -10, 11)), + output: CompletionOptions::ArrowRecordBatch, + ..Default::default() + }; + + let query_config = LeafPartitionQueryConfig { + common: input, + explain_plans: false, + write_quantile_sketch: false, + ranges: vec![ + SleeperPartitionRegion::new(single_int_range("key", 2, 6)), + SleeperPartitionRegion::new(single_int_range("key", 4, 9)), + ], + }; + + // When + let result = run_query(&query_config).await?; + + // Then + let CompletedOutput::ArrowRecordBatch(stream) = result else { + bail!("Expected arrow record batch stream output"); + }; + + assert_eq!( + read_batches_of_int_fields(stream, ["key"]).await?, + vec![[2], [3], [4], [5], [6], [7], [8]] + ); + Ok(()) +} + +#[tokio::test] +async fn should_return_results_from_two_non_overlapping_query_ranges() -> Result<(), Error> { + // Given + let dir = tempdir()?; + let file_1 = file(&dir, "file1.parquet"); + let file_2 = file(&dir, "file2.parquet"); + + write_file_of_ints(&file_1, "key", vec![1, 3, 5, 7, 9])?; + write_file_of_ints(&file_2, "key", vec![2, 4, 6, 8, 10])?; + + let input = CommonConfig { + input_files: Vec::from([file_1, file_2]), + input_files_sorted: true, + row_key_cols: row_key_cols(["key"]), + region: SleeperPartitionRegion::new(single_int_range("key", -10, 11)), + output: CompletionOptions::ArrowRecordBatch, + ..Default::default() + }; + + let query_config = LeafPartitionQueryConfig { + common: input, + explain_plans: false, + write_quantile_sketch: false, + ranges: vec![ + SleeperPartitionRegion::new(single_int_range("key", 2, 5)), + SleeperPartitionRegion::new(single_int_range("key", 7, 9)), + ], + }; + + // When + let result = run_query(&query_config).await?; + + // Then + let CompletedOutput::ArrowRecordBatch(stream) = result else { + bail!("Expected arrow record batch stream output"); + }; + + assert_eq!( + read_batches_of_int_fields(stream, ["key"]).await?, + vec![[2], [3], [4], [7], [8]] + ); + Ok(()) +} + +#[tokio::test] +async fn should_error_with_no_query_ranges() -> Result<(), Error> { + // Given + let dir = tempdir()?; + let file_1 = file(&dir, "file1.parquet"); + let file_2 = file(&dir, "file2.parquet"); + + write_file_of_ints(&file_1, "key", vec![1, 3, 5, 7, 9])?; + write_file_of_ints(&file_2, "key", vec![2, 4, 6, 8, 10])?; + + let input = CommonConfig { + input_files: Vec::from([file_1, file_2]), + input_files_sorted: true, + row_key_cols: row_key_cols(["key"]), + region: SleeperPartitionRegion::new(single_int_range("key", 0, 3)), + output: CompletionOptions::ArrowRecordBatch, + ..Default::default() + }; + + let query_config = LeafPartitionQueryConfig { + common: input, + explain_plans: false, + write_quantile_sketch: false, + ranges: vec![], + }; + + // Then + let Err(result) = run_query(&query_config).await else { + bail!("Expected an error type here"); + }; + + assert_eq!( + format!("{result}"), + "Error during planning: No query regions specified" + ); + Ok(()) +} + +#[tokio::test] +async fn should_error_when_arrow_output_with_sketches() -> Result<(), Error> { + // Given + let dir = tempdir()?; + let file_1 = file(&dir, "file1.parquet"); + + write_file_of_ints(&file_1, "key", vec![1])?; + + let input = CommonConfig { + input_files: Vec::from([file_1]), + input_files_sorted: true, + row_key_cols: row_key_cols(["key"]), + region: SleeperPartitionRegion::new(single_int_range("key", 0, 3)), + output: CompletionOptions::ArrowRecordBatch, + ..Default::default() + }; + + let query_config = LeafPartitionQueryConfig { + common: input, + explain_plans: false, + write_quantile_sketch: true, + ranges: vec![SleeperPartitionRegion::new(single_int_range("key", 2, 5))], + }; + + // Then + let Err(result) = run_query(&query_config).await else { + bail!("Expected an error type here"); + }; + + assert_eq!( + format!("{result}"), + "Error during planning: Quantile sketch output cannot be enabled if file output not selected" + ); + Ok(()) +} + +#[tokio::test] +async fn should_return_results_as_file_with_sketch() -> Result<(), Error> { + // Given + let dir = tempdir()?; + let file_1 = file(&dir, "file1.parquet"); + let file_2 = file(&dir, "file2.parquet"); + let output = file(&dir, "output.parquet"); + let sketches = file(&dir, "output.sketches"); + + write_file_of_ints(&file_1, "key", vec![1, 3, 5, 7, 9])?; + write_file_of_ints(&file_2, "key", vec![2, 4, 6, 8, 10])?; + + let input = CommonConfig { + input_files: Vec::from([file_1, file_2]), + input_files_sorted: true, + row_key_cols: row_key_cols(["key"]), + region: SleeperPartitionRegion::new(single_int_range("key", 0, 6)), + output: CompletionOptions::File { + output_file: output.clone(), + opts: SleeperParquetOptions::default(), + }, + ..Default::default() + }; + + let query_config = LeafPartitionQueryConfig { + common: input, + explain_plans: false, + write_quantile_sketch: true, + ranges: vec![SleeperPartitionRegion::new(single_int_range("key", 1, 5))], + }; + + // When + let result = run_query(&query_config).await?; + + // Then + let CompletedOutput::File(row_counts) = result else { + bail!("Expected file output"); + }; + + // Then + assert_eq!(read_file_of_ints(&output, "key")?, vec![1, 2, 3, 4]); + assert_eq!([row_counts.rows_read, row_counts.rows_written], [4, 4]); + assert_eq!(read_sketch_min_max_ints(&sketches).await?, [1, 4]); + Ok(()) +} From 4faae2f5400c40babe48b03731c3f5c6fd4ac7c9 Mon Sep 17 00:00:00 2001 From: m09526 Date: Tue, 2 Sep 2025 16:22:07 +0000 Subject: [PATCH 20/28] Clippy warnings --- rust/sleeper_core/src/lib.rs | 2 +- rust/sleeper_core/test_util/src/lib.rs | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/rust/sleeper_core/src/lib.rs b/rust/sleeper_core/src/lib.rs index a9509ef2554..bb29874ee31 100644 --- a/rust/sleeper_core/src/lib.rs +++ b/rust/sleeper_core/src/lib.rs @@ -487,7 +487,7 @@ mod tests { match new_output { CompletionOptions::ArrowRecordBatch => {} - _ => panic!("Output should be ArrowRecordBatch"), + CompletionOptions::File { .. } => panic!("Output should be ArrowRecordBatch"), } } diff --git a/rust/sleeper_core/test_util/src/lib.rs b/rust/sleeper_core/test_util/src/lib.rs index 59e57a0b4ed..63d5d44ae94 100644 --- a/rust/sleeper_core/test_util/src/lib.rs +++ b/rust/sleeper_core/test_util/src/lib.rs @@ -130,7 +130,7 @@ pub fn read_file_of_int_fields( let mut data: Vec<[i32; N]> = Vec::new(); let metadata = ArrowReaderMetadata::load(&file, ArrowReaderOptions::default())?; for field_name in field_names { - check_non_null_field(field_name, &DataType::Int32, &metadata.schema())?; + check_non_null_field(field_name, &DataType::Int32, metadata.schema())?; } for result in ParquetRecordBatchReaderBuilder::new_with_metadata(file, metadata).build()? { let batch = result?; From 1004b870e1d7e579b73e64aa0b6d564c418c5418 Mon Sep 17 00:00:00 2001 From: patchwork01 <110390516+patchwork01@users.noreply.github.com> Date: Wed, 3 Sep 2025 08:18:30 +0000 Subject: [PATCH 21/28] 5408 Reformat compaction_test.rs --- rust/sleeper_core/tests/compaction_test.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/rust/sleeper_core/tests/compaction_test.rs b/rust/sleeper_core/tests/compaction_test.rs index a48895cef39..fab1ff1e8c5 100644 --- a/rust/sleeper_core/tests/compaction_test.rs +++ b/rust/sleeper_core/tests/compaction_test.rs @@ -15,13 +15,13 @@ */ use arrow::datatypes::{DataType, Field, Schema}; use color_eyre::eyre::Error; -use test_util::*; use sleeper_core::{ CommonConfig, CompletionOptions, SleeperParquetOptions, SleeperPartitionRegion, run_compaction, }; use std::{collections::HashMap, path::Path, sync::Arc}; use tempfile::tempdir; use test_log::test; +use test_util::*; #[test(tokio::test)] async fn should_merge_two_files() -> Result<(), Error> { From 1bdcbbbc87762de7de5c08c2827ccdbc094ddee7 Mon Sep 17 00:00:00 2001 From: patchwork01 <110390516+patchwork01@users.noreply.github.com> Date: Wed, 3 Sep 2025 08:30:01 +0000 Subject: [PATCH 22/28] 5408 Remove unused dependency declaration --- rust/Cargo.lock | 5 ----- rust/Cargo.toml | 1 - rust/apps/Cargo.toml | 1 - rust/objectstore_ext/Cargo.toml | 1 - rust/sleeper_core/Cargo.toml | 1 - rust/sleeper_df/Cargo.toml | 1 - 6 files changed, 10 deletions(-) diff --git a/rust/Cargo.lock b/rust/Cargo.lock index 2898e47c1af..bfc5dad9711 100644 --- a/rust/Cargo.lock +++ b/rust/Cargo.lock @@ -154,7 +154,6 @@ dependencies = [ "predicates", "sleeper_core", "tokio", - "tracing-subscriber", "url", ] @@ -3329,7 +3328,6 @@ dependencies = [ "thiserror", "tokio", "tokio-test", - "tracing-subscriber", "url", ] @@ -4212,7 +4210,6 @@ dependencies = [ "test_util", "tokio", "tokio-test", - "tracing-subscriber", "url", ] @@ -4228,7 +4225,6 @@ dependencies = [ "log", "sleeper_core", "tokio", - "tracing-subscriber", "url", ] @@ -4775,7 +4771,6 @@ dependencies = [ "once_cell", "regex-automata", "sharded-slab", - "smallvec", "thread_local", "tracing", "tracing-core", diff --git a/rust/Cargo.toml b/rust/Cargo.toml index f2535bf2be2..5430222c35c 100644 --- a/rust/Cargo.toml +++ b/rust/Cargo.toml @@ -35,7 +35,6 @@ bytes = { version = "1.10.1" } # Byte buffer for S3 uploadi cargo_metadata = { version = "0.22.0" } # Access cargo metadata programmtically chrono = { version = "0.4.41" } # Log helper clap = { version = "4.5.46" } # Cmd line args processing -tracing-subscriber = { version = "0.3.20" } color-eyre = { version = "0.6.5" } # Colourised version of `anyhow` cxx = { version = "1.0.174" } # Exception handling for Rust cxx-build = { version = "1.0.174" } diff --git a/rust/apps/Cargo.toml b/rust/apps/Cargo.toml index e96e45c61bd..2ea405dcdfb 100644 --- a/rust/apps/Cargo.toml +++ b/rust/apps/Cargo.toml @@ -29,7 +29,6 @@ publish = { workspace = true } arrow = { workspace = true } chrono = { workspace = true } clap = { workspace = true, features = ["derive"] } -tracing-subscriber = { workspace = true } color-eyre = { workspace = true } sleeper_core = { path = "../sleeper_core" } env_logger = { workspace = true } diff --git a/rust/objectstore_ext/Cargo.toml b/rust/objectstore_ext/Cargo.toml index c0fc61b0dd6..1fb8b7bf1cd 100644 --- a/rust/objectstore_ext/Cargo.toml +++ b/rust/objectstore_ext/Cargo.toml @@ -30,7 +30,6 @@ aws-credential-types = { workspace = true, features = ["hardcoded-credentials"] async-trait = { workspace = true } bytes = { workspace = true } chrono = { workspace = true } -tracing-subscriber = { workspace = true } color-eyre = { workspace = true } futures = { workspace = true } log = { workspace = true } diff --git a/rust/sleeper_core/Cargo.toml b/rust/sleeper_core/Cargo.toml index 4c6e2ba9665..53de212f861 100644 --- a/rust/sleeper_core/Cargo.toml +++ b/rust/sleeper_core/Cargo.toml @@ -33,7 +33,6 @@ aws-credential-types = { workspace = true, features = ["hardcoded-credentials"] aws-types = { workspace = true } bytes = { workspace = true } chrono = { workspace = true } -tracing-subscriber = { workspace = true } color-eyre = { workspace = true } cxx = { workspace = true } datafusion = { workspace = true } diff --git a/rust/sleeper_df/Cargo.toml b/rust/sleeper_df/Cargo.toml index c1f2c4641ae..3fcc280e5e5 100644 --- a/rust/sleeper_df/Cargo.toml +++ b/rust/sleeper_df/Cargo.toml @@ -33,7 +33,6 @@ crate-type = ["cdylib", "rlib"] [dependencies] aws-types = { workspace = true } chrono = { workspace = true } -tracing-subscriber = { workspace = true } color-eyre = { workspace = true } env_logger = { workspace = true } log = { workspace = true } From 868bccf2b1a25571c30394d269a6aaa36e9be161 Mon Sep 17 00:00:00 2001 From: m09526 Date: Wed, 3 Sep 2025 08:51:58 +0000 Subject: [PATCH 23/28] Remove unneeded macro --- rust/sleeper_core/test_util/src/lib.rs | 13 ------------- 1 file changed, 13 deletions(-) diff --git a/rust/sleeper_core/test_util/src/lib.rs b/rust/sleeper_core/test_util/src/lib.rs index 63d5d44ae94..142c841d028 100644 --- a/rust/sleeper_core/test_util/src/lib.rs +++ b/rust/sleeper_core/test_util/src/lib.rs @@ -226,16 +226,3 @@ pub fn int_range<'r>(min: i32, max: i32) -> ColRange<'r> { upper_inclusive: false, } } - -#[macro_export] -macro_rules! assert_error { - ($err_expr: expr, $err_type: path, $err_contents: expr) => { - let result = if let Err($err_type(err)) = $err_expr { - assert_eq!(err, $err_contents); - true - } else { - false - }; - assert!(result, "Expected different error type"); - }; -} From d20d041d87cd8ed3e42fe8ea5240133a3ceb98dc Mon Sep 17 00:00:00 2001 From: m09526 Date: Wed, 3 Sep 2025 16:26:43 +0000 Subject: [PATCH 24/28] Better error handling --- rust/sleeper_core/test_util/src/lib.rs | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/rust/sleeper_core/test_util/src/lib.rs b/rust/sleeper_core/test_util/src/lib.rs index 142c841d028..b5cd3782847 100644 --- a/rust/sleeper_core/test_util/src/lib.rs +++ b/rust/sleeper_core/test_util/src/lib.rs @@ -150,7 +150,8 @@ pub async fn read_batches_of_int_fields( check_non_null_field(field_name, &DataType::Int32, schema.as_ref())?; } let mut data: Vec<[i32; N]> = Vec::new(); - while let Some(Ok(batch)) = stream.next().await { + while let Some(batch) = stream.next().await { + let batch = batch?; let arrays: Vec<&Int32Array> = get_int_arrays(&batch, field_names)?; data.extend((0..batch.num_rows()).map(|row_number| read_row(row_number, &arrays))); } From 30f0f2c4cee9c356f228c34d58b40d1051f2c0ae Mon Sep 17 00:00:00 2001 From: m09526 Date: Thu, 4 Sep 2025 14:51:43 +0000 Subject: [PATCH 25/28] Update to builder pattern --- rust/Cargo.lock | 24 +- rust/apps/src/bin/compact.rs | 24 +- rust/apps/src/bin/query.rs | 22 +- rust/sleeper_core/src/datafusion.rs | 10 +- rust/sleeper_core/src/datafusion/compact.rs | 4 +- .../src/datafusion/leaf_partition_query.rs | 10 +- rust/sleeper_core/src/datafusion/output.rs | 12 +- rust/sleeper_core/src/lib.rs | 219 +++++++++++------- rust/sleeper_core/tests/compaction_test.rs | 14 +- rust/sleeper_core/tests/query_test.rs | 22 +- rust/sleeper_df/src/lib.rs | 42 ++-- 11 files changed, 229 insertions(+), 174 deletions(-) diff --git a/rust/Cargo.lock b/rust/Cargo.lock index bfc5dad9711..3653c6b22a5 100644 --- a/rust/Cargo.lock +++ b/rust/Cargo.lock @@ -1327,9 +1327,9 @@ dependencies = [ [[package]] name = "cxx" -version = "1.0.174" +version = "1.0.175" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e5ba77f286ce5c44c7ba02de894b057bc0a605a210e3d81fa83b92d94586c0e1" +checksum = "84aa1f8258b77022835f4ce5bd3b5aa418b969494bd7c3cb142c88424eb4c715" dependencies = [ "cc", "cxxbridge-cmd", @@ -1341,9 +1341,9 @@ dependencies = [ [[package]] name = "cxx-build" -version = "1.0.174" +version = "1.0.175" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0c56fdf6fba27288d1fda3384062692e66dc40ca41bafd15f616dd4e8b0ac909" +checksum = "d4e2aa0ea9f398b72f329197cfad624fcb16b2538d3ffb0f71f51cd19fa2a512" dependencies = [ "cc", "codespan-reporting", @@ -1356,9 +1356,9 @@ dependencies = [ [[package]] name = "cxxbridge-cmd" -version = "1.0.174" +version = "1.0.175" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "19ade5eb6d6e6ef9c5631eff7e4f74e0e7109140e775f124d76904c0e5e6a202" +checksum = "902e9553c7db1cc00baee88d6a531792d3e1aaab06ed6d1dcd606647891ea693" dependencies = [ "clap", "codespan-reporting", @@ -1370,15 +1370,15 @@ dependencies = [ [[package]] name = "cxxbridge-flags" -version = "1.0.174" +version = "1.0.175" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "99f99fe2f3f76a2ba40c5431f854efe3725c19a89f4d59966bca3ec561be940e" +checksum = "35b2b0b4d405850b0048447786b70c2502c84e4d5c4c757416abc0500336edfc" [[package]] name = "cxxbridge-macro" -version = "1.0.174" +version = "1.0.175" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2b6e5fa0545804d2d8d398a1e995203a1f2403a9f0651d50546462e61a28340e" +checksum = "fd2a8fe0dfa4a2207b80ca9492c0d5dc8752b66f5631d93b23065f40f6a943d3" dependencies = [ "indexmap", "proc-macro2", @@ -3035,9 +3035,9 @@ dependencies = [ [[package]] name = "log" -version = "0.4.27" +version = "0.4.28" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "13dc2df351e3202783a1fe0d44375f7295ffb4049267b0f3018346dc122a1d94" +checksum = "34080505efa8e45a4b816c349525ebe327ceaa8559756f0356cba97ef3bf7432" [[package]] name = "lru-slab" diff --git a/rust/apps/src/bin/compact.rs b/rust/apps/src/bin/compact.rs index ca2fe57a976..5dee2e91f95 100644 --- a/rust/apps/src/bin/compact.rs +++ b/rust/apps/src/bin/compact.rs @@ -21,7 +21,7 @@ use human_panic::setup_panic; use log::info; use num_format::{Locale, ToFormattedString}; use sleeper_core::{ - ColRange, CommonConfig, CompletionOptions, PartitionBound, SleeperParquetOptions, + ColRange, CommonConfigBuilder, OutputType, PartitionBound, SleeperParquetOptions, SleeperPartitionRegion, run_compaction, }; use std::{collections::HashMap, io::Write}; @@ -140,19 +140,19 @@ async fn main() -> color_eyre::Result<()> { dict_enc_values: true, }; - let details = CommonConfig::try_new( - None, - input_urls, - true, - args.row_keys, - args.sort_keys, - SleeperPartitionRegion::new(map), - CompletionOptions::File { + let details = CommonConfigBuilder::new() + .aws_config(None) + .input_files(input_urls) + .input_files_sorted(true) + .row_key_cols(args.row_keys) + .sort_key_cols(args.sort_keys) + .region(SleeperPartitionRegion::new(map)) + .output(OutputType::File { output_file, opts: parquet_options, - }, - args.iterator_config, - )?; + }) + .iterator_config(args.iterator_config) + .build()?; let result = run_compaction(&details).await?; info!( diff --git a/rust/apps/src/bin/query.rs b/rust/apps/src/bin/query.rs index ba1589bb957..901a114be93 100644 --- a/rust/apps/src/bin/query.rs +++ b/rust/apps/src/bin/query.rs @@ -21,7 +21,7 @@ use color_eyre::eyre::bail; use futures::StreamExt; use human_panic::setup_panic; use sleeper_core::{ - ColRange, CommonConfig, CompletedOutput, CompletionOptions, LeafPartitionQueryConfig, + ColRange, CommonConfigBuilder, CompletedOutput, LeafPartitionQueryConfig, OutputType, PartitionBound, SleeperPartitionRegion, run_query, }; use std::{collections::HashMap, io::Write}; @@ -149,16 +149,16 @@ async fn main() -> color_eyre::Result<()> { ); } - let common = CommonConfig::try_new( - None, - input_urls, - true, - args.row_keys, - args.sort_keys, - SleeperPartitionRegion::new(map), - CompletionOptions::ArrowRecordBatch, - args.iterator_config, - )?; + let common = CommonConfigBuilder::new() + .aws_config(None) + .input_files(input_urls) + .input_files_sorted(true) + .row_key_cols(args.row_keys) + .sort_key_cols(args.sort_keys) + .region(SleeperPartitionRegion::new(map)) + .output(OutputType::ArrowRecordBatch) + .iterator_config(args.iterator_config) + .build()?; let query_config = LeafPartitionQueryConfig { common, diff --git a/rust/sleeper_core/src/datafusion.rs b/rust/sleeper_core/src/datafusion.rs index fdc9dfc503d..ed94f47e442 100644 --- a/rust/sleeper_core/src/datafusion.rs +++ b/rust/sleeper_core/src/datafusion.rs @@ -59,7 +59,7 @@ mod util; pub use compact::compact; pub use config::ParquetWriterConfigurer; pub use leaf_partition_query::{LeafPartitionQuery, LeafPartitionQueryConfig}; -pub use output::CompletionOptions; +pub use output::OutputType; pub use region::SleeperPartitionRegion; /// Drives common operations in processing of `DataFusion` for Sleeper. @@ -94,7 +94,7 @@ impl<'a> SleeperOperations<'a> { // together in wrong order. cfg.options_mut().optimizer.repartition_aggregations = false; // Set upload size if outputting to a file - if let CompletionOptions::File { + if let OutputType::File { output_file: _, opts: parquet_options, } = &self.config.output @@ -124,8 +124,8 @@ impl<'a> SleeperOperations<'a> { store_factory, &self.config.input_files, match &self.config.output { - CompletionOptions::ArrowRecordBatch => None, - CompletionOptions::File { + OutputType::ArrowRecordBatch => None, + OutputType::File { output_file, opts: _, } => Some(output_file), @@ -274,7 +274,7 @@ impl<'a> SleeperOperations<'a> { frame: DataFrame, configurer: &ParquetWriterConfigurer<'_>, ) -> Result { - let CompletionOptions::File { + let OutputType::File { output_file, opts: _, } = &self.config.output diff --git a/rust/sleeper_core/src/datafusion/compact.rs b/rust/sleeper_core/src/datafusion/compact.rs index a054826726b..7bc74b90e02 100644 --- a/rust/sleeper_core/src/datafusion/compact.rs +++ b/rust/sleeper_core/src/datafusion/compact.rs @@ -18,7 +18,7 @@ use crate::{ CommonConfig, CompactionResult, datafusion::{ - CompletionOptions, SleeperOperations, + OutputType, SleeperOperations, metrics::RowCounts, output::{CompletedOutput, Completer}, sketch::{Sketcher, output_sketch}, @@ -47,7 +47,7 @@ pub async fn compact( info!("DataFusion compaction: {ops}"); // Retrieve Parquet output options - let CompletionOptions::File { + let OutputType::File { output_file, opts: _, } = &config.output diff --git a/rust/sleeper_core/src/datafusion/leaf_partition_query.rs b/rust/sleeper_core/src/datafusion/leaf_partition_query.rs index e9afd668503..2be0bd53747 100644 --- a/rust/sleeper_core/src/datafusion/leaf_partition_query.rs +++ b/rust/sleeper_core/src/datafusion/leaf_partition_query.rs @@ -18,7 +18,7 @@ use crate::{ CommonConfig, SleeperPartitionRegion, datafusion::{ - CompletionOptions, SleeperOperations, + OutputType, SleeperOperations, output::CompletedOutput, sketch::{Sketcher, output_sketch}, util::explain_plan, @@ -123,13 +123,13 @@ impl<'a> LeafPartitionQuery<'a> { && self.config.write_quantile_sketch { match &self.config.common.output { - CompletionOptions::File { + OutputType::File { output_file, opts: _, } => { output_sketch(self.store_factory, output_file, sketch_func.sketch()).await?; } - CompletionOptions::ArrowRecordBatch => { + OutputType::ArrowRecordBatch => { return plan_err!( "Quantile sketch output cannot be enabled if file output not selected" ); @@ -151,7 +151,7 @@ impl<'a> LeafPartitionQuery<'a> { ) -> Result<(Option>, DataFrame), DataFusionError> { if self.config.write_quantile_sketch { match self.config.common.output { - CompletionOptions::File { + OutputType::File { output_file: _, opts: _, } => { @@ -159,7 +159,7 @@ impl<'a> LeafPartitionQuery<'a> { let frame = sketcher.apply_sketch(frame)?; Ok((Some(sketcher), frame)) } - CompletionOptions::ArrowRecordBatch => plan_err!( + OutputType::ArrowRecordBatch => plan_err!( "Quantile sketch output cannot be enabled if file output not selected" ), } diff --git a/rust/sleeper_core/src/datafusion/output.rs b/rust/sleeper_core/src/datafusion/output.rs index 4666fb4f901..8e7f7b3d6d6 100644 --- a/rust/sleeper_core/src/datafusion/output.rs +++ b/rust/sleeper_core/src/datafusion/output.rs @@ -38,7 +38,7 @@ use url::Url; /// Defines how operation output should be given. #[derive(Debug, Default)] -pub enum CompletionOptions { +pub enum OutputType { /// `DataFusion` results will be returned as a stream of Arrow [`RecordBatch`]es. #[default] ArrowRecordBatch, @@ -51,7 +51,7 @@ pub enum CompletionOptions { }, } -impl CompletionOptions { +impl OutputType { /// Create a [`Completer`] for this type of output. #[must_use] pub fn finisher<'a>(&self, ops: &'a SleeperOperations<'a>) -> Box { @@ -126,14 +126,14 @@ impl<'a> FileOutputCompleter<'a> { impl Completer for FileOutputCompleter<'_> { fn complete_frame(&self, frame: DataFrame) -> Result { match &self.ops.config.output { - CompletionOptions::File { + OutputType::File { output_file: _, opts: parquet_options, } => { let configurer = ParquetWriterConfigurer { parquet_options }; self.ops.plan_with_parquet_output(frame, &configurer) } - CompletionOptions::ArrowRecordBatch => { + OutputType::ArrowRecordBatch => { plan_err!("Can't use FileOutputCompleter with CompletionOptions::ArrowRecordBatch") } } @@ -166,13 +166,13 @@ impl<'a> ArrowOutputCompleter<'a> { impl Completer for ArrowOutputCompleter<'_> { fn complete_frame(&self, frame: DataFrame) -> Result { match &self.ops.config.output { - CompletionOptions::File { + OutputType::File { output_file: _, opts: _, } => { plan_err!("Can't use ArrowOutputCompleter with CompletionOptions::File") } - CompletionOptions::ArrowRecordBatch => Ok(frame), + OutputType::ArrowRecordBatch => Ok(frame), } } diff --git a/rust/sleeper_core/src/lib.rs b/rust/sleeper_core/src/lib.rs index bb29874ee31..65502ad78ce 100644 --- a/rust/sleeper_core/src/lib.rs +++ b/rust/sleeper_core/src/lib.rs @@ -35,7 +35,7 @@ mod datafusion; use crate::datafusion::LeafPartitionQuery; pub use crate::datafusion::output::CompletedOutput; pub use datafusion::{ - CompletionOptions, LeafPartitionQueryConfig, SleeperPartitionRegion, + LeafPartitionQueryConfig, OutputType, SleeperPartitionRegion, sketch::{DataSketchVariant, deserialise_sketches}, }; @@ -94,21 +94,21 @@ impl Default for SleeperParquetOptions { #[derive(Debug)] pub struct CommonConfig<'a> { /// Aws credentials configuration - pub aws_config: Option, + aws_config: Option, /// Input file URLs - pub input_files: Vec, + input_files: Vec, /// Are input files individually sorted? - pub input_files_sorted: bool, + input_files_sorted: bool, /// Names of row-key columns - pub row_key_cols: Vec, + row_key_cols: Vec, /// Names of sort-key columns - pub sort_key_cols: Vec, + sort_key_cols: Vec, /// Ranges for each column to filter input files - pub region: SleeperPartitionRegion<'a>, + region: SleeperPartitionRegion<'a>, /// How output from operation should be returned - pub output: CompletionOptions, + output: OutputType, /// Iterator config. Filters, aggregators, etc. - pub iterator_config: Option, + iterator_config: Option, } impl Default for CommonConfig<'_> { @@ -120,58 +120,21 @@ impl Default for CommonConfig<'_> { row_key_cols: Vec::default(), sort_key_cols: Vec::default(), region: SleeperPartitionRegion::default(), - output: CompletionOptions::default(), + output: OutputType::default(), iterator_config: Option::default(), } } } -impl<'a> CommonConfig<'a> { - /// Creates a new configuration object. - /// - /// # Errors - /// The configuration must validate. Input files mustn't be empty - /// and the number of row key columns must match the number of region - /// dimensions. - #[allow(clippy::too_many_arguments)] - pub fn try_new( - aws_config: Option, - input_files: Vec, - input_files_sorted: bool, - row_key_cols: Vec, - sort_key_cols: Vec, - region: SleeperPartitionRegion<'a>, - output: CompletionOptions, - iterator_config: Option, - ) -> Result { - validate(&input_files, &row_key_cols, ®ion)?; - // Convert Java s3a schema to s3 - let (input_files, output) = normalise_s3a_urls(input_files, output); - Ok(Self { - aws_config, - input_files, - input_files_sorted, - row_key_cols, - sort_key_cols, - region, - output, - iterator_config, - }) - } -} - /// Change all input and output URLS from s3a to s3 scheme. -fn normalise_s3a_urls( - mut input_files: Vec, - mut output: CompletionOptions, -) -> (Vec, CompletionOptions) { +fn normalise_s3a_urls(mut input_files: Vec, mut output: OutputType) -> (Vec, OutputType) { for t in &mut input_files { if t.scheme() == "s3a" { let _ = t.set_scheme("s3"); } } - if let CompletionOptions::File { + if let OutputType::File { output_file, opts: _, } = &mut output @@ -182,29 +145,6 @@ fn normalise_s3a_urls( (input_files, output) } -/// Performs validity checks on parameters. -/// -/// # Errors -/// There must be at least one input file. -/// The length of `row_key_cols` must match the number of region dimensions. -fn validate( - input_files: &[Url], - row_key_cols: &[String], - region: &SleeperPartitionRegion<'_>, -) -> Result<()> { - if input_files.is_empty() { - bail!("No input paths supplied"); - } - if row_key_cols.len() != region.len() { - bail!( - "Length mismatch between row keys {} and partition region bounds {}", - row_key_cols.len(), - region.len() - ); - } - Ok(()) -} - impl CommonConfig<'_> { /// Get iterator for row and sort key columns in order pub fn sorting_columns_iter(&self) -> impl Iterator { @@ -230,8 +170,8 @@ impl Display for CommonConfig<'_> { self.region )?; match &self.output { - CompletionOptions::ArrowRecordBatch => write!(f, " output is Arrow RecordBatches"), - CompletionOptions::File { + OutputType::ArrowRecordBatch => write!(f, " output is Arrow RecordBatches"), + OutputType::File { output_file, opts: _, } => write!(f, "output file {output_file:?}"), @@ -239,6 +179,117 @@ impl Display for CommonConfig<'_> { } } +/// Builder for `CommonConfig`. +#[derive(Default)] +pub struct CommonConfigBuilder<'a> { + aws_config: Option, + input_files: Vec, + input_files_sorted: bool, + row_key_cols: Vec, + sort_key_cols: Vec, + region: SleeperPartitionRegion<'a>, + output: OutputType, + iterator_config: Option, +} + +impl<'a> CommonConfigBuilder<'a> { + #[must_use] + pub fn new() -> Self { + Self::default() + } + + #[must_use] + pub fn aws_config(mut self, aws_config: Option) -> Self { + self.aws_config = aws_config; + self + } + + #[must_use] + pub fn input_files(mut self, input_files: Vec) -> Self { + self.input_files = input_files; + self + } + + #[must_use] + pub fn input_files_sorted(mut self, input_files_sorted: bool) -> Self { + self.input_files_sorted = input_files_sorted; + self + } + + #[must_use] + pub fn row_key_cols(mut self, row_key_cols: Vec) -> Self { + self.row_key_cols = row_key_cols; + self + } + + #[must_use] + pub fn sort_key_cols(mut self, sort_key_cols: Vec) -> Self { + self.sort_key_cols = sort_key_cols; + self + } + + #[must_use] + pub fn region(mut self, region: SleeperPartitionRegion<'a>) -> Self { + self.region = region; + self + } + + #[must_use] + pub fn output(mut self, output: OutputType) -> Self { + self.output = output; + self + } + + #[must_use] + pub fn iterator_config(mut self, iterator_config: Option) -> Self { + self.iterator_config = iterator_config; + self + } + + /// Build the `CommonConfig`, consuming the builder and validating required fields. + /// + /// # Errors + /// The configuration must validate. Input files mustn't be empty + /// and the number of row key columns must match the number of region + /// dimensions. + pub fn build(self) -> Result> { + self.validate()?; + + // s3a normalization + let (input_files, output) = normalise_s3a_urls(self.input_files, self.output); + + Ok(CommonConfig { + aws_config: self.aws_config, + input_files, + input_files_sorted: self.input_files_sorted, + row_key_cols: self.row_key_cols, + sort_key_cols: self.sort_key_cols, + region: self.region, + output, + iterator_config: self.iterator_config, + }) + } + + /// Performs validity checks on parameters. + /// + /// # Errors + /// There must be at least one input file. + /// The length of `row_key_cols` must match the number of region dimensions. + fn validate(&self) -> Result<()> { + if self.input_files.is_empty() { + bail!("No input paths supplied"); + } + if self.row_key_cols.len() != self.region.len() { + bail!( + "Length mismatch between row keys {} and partition region bounds {}", + self.row_key_cols.len(), + self.region.len() + ); + } + Ok(()) + } +} + #[derive(Debug)] pub struct AwsConfig { pub region: String, @@ -392,7 +443,7 @@ mod tests { Url::parse("s3a://bucket/key1").unwrap(), Url::parse("s3a://bucket/key2").unwrap(), ]; - let output = CompletionOptions::File { + let output = OutputType::File { output_file: Url::parse("https://example.com/output").unwrap(), opts: SleeperParquetOptions::default(), }; @@ -404,7 +455,7 @@ mod tests { for url in new_files { assert_eq!(url.scheme(), "s3"); } - if let CompletionOptions::File { output_file, .. } = new_output { + if let OutputType::File { output_file, .. } = new_output { assert_eq!(output_file.scheme(), "https"); // unchanged } else { panic!("Output option changed unexpectedly") @@ -415,7 +466,7 @@ mod tests { fn test_no_change_for_non_s3a_urls() { // Given let input_files = vec![Url::parse("https://example.com/key").unwrap()]; - let output = CompletionOptions::File { + let output = OutputType::File { output_file: Url::parse("https://example.com/output").unwrap(), opts: SleeperParquetOptions::default(), }; @@ -425,7 +476,7 @@ mod tests { // Then assert_eq!(new_files[0].scheme(), "https"); - if let CompletionOptions::File { output_file, .. } = new_output { + if let OutputType::File { output_file, .. } = new_output { assert_eq!(output_file.scheme(), "https"); // unchanged } else { panic!("Output option changed unexpectedly") @@ -436,7 +487,7 @@ mod tests { fn test_convert_output_scheme_when_s3a() { // Given let input_files = vec![Url::parse("https://example.com/key").unwrap()]; - let output = CompletionOptions::File { + let output = OutputType::File { output_file: Url::parse("s3a://bucket/output").unwrap(), opts: SleeperParquetOptions::default(), }; @@ -445,7 +496,7 @@ mod tests { let (_, new_output) = normalise_s3a_urls(input_files, output); // Then - if let CompletionOptions::File { output_file, .. } = new_output { + if let OutputType::File { output_file, .. } = new_output { assert_eq!(output_file.scheme(), "s3"); } else { panic!("Unexpected output option type") @@ -456,7 +507,7 @@ mod tests { fn test_empty_input_files() { // Given let input_files: Vec = vec![]; - let output = CompletionOptions::File { + let output = OutputType::File { output_file: Url::parse("https://example.com/output").unwrap(), opts: SleeperParquetOptions::default(), }; @@ -475,7 +526,7 @@ mod tests { Url::parse("s3a://bucket/key1").unwrap(), Url::parse("s3a://bucket/key2").unwrap(), ]; - let output = CompletionOptions::ArrowRecordBatch; + let output = OutputType::ArrowRecordBatch; // When let (new_files, new_output) = normalise_s3a_urls(input_files.clone(), output); @@ -486,8 +537,8 @@ mod tests { } match new_output { - CompletionOptions::ArrowRecordBatch => {} - CompletionOptions::File { .. } => panic!("Output should be ArrowRecordBatch"), + OutputType::ArrowRecordBatch => {} + OutputType::File { .. } => panic!("Output should be ArrowRecordBatch"), } } diff --git a/rust/sleeper_core/tests/compaction_test.rs b/rust/sleeper_core/tests/compaction_test.rs index fab1ff1e8c5..5ceb4578667 100644 --- a/rust/sleeper_core/tests/compaction_test.rs +++ b/rust/sleeper_core/tests/compaction_test.rs @@ -16,7 +16,7 @@ use arrow::datatypes::{DataType, Field, Schema}; use color_eyre::eyre::Error; use sleeper_core::{ - CommonConfig, CompletionOptions, SleeperParquetOptions, SleeperPartitionRegion, run_compaction, + CommonConfig, OutputType, SleeperParquetOptions, SleeperPartitionRegion, run_compaction, }; use std::{collections::HashMap, path::Path, sync::Arc}; use tempfile::tempdir; @@ -39,7 +39,7 @@ async fn should_merge_two_files() -> Result<(), Error> { input_files_sorted: true, row_key_cols: row_key_cols(["key"]), region: SleeperPartitionRegion::new(single_int_range("key", 0, 5)), - output: CompletionOptions::File { + output: OutputType::File { output_file: output.clone(), opts: SleeperParquetOptions::default(), }, @@ -72,7 +72,7 @@ async fn should_merge_files_with_overlapping_data() -> Result<(), Error> { input_files_sorted: true, row_key_cols: row_key_cols(["key"]), region: SleeperPartitionRegion::new(single_int_range("key", 0, 5)), - output: CompletionOptions::File { + output: OutputType::File { output_file: output.clone(), opts: SleeperParquetOptions::default(), }, @@ -105,7 +105,7 @@ async fn should_exclude_data_not_in_region() -> Result<(), Error> { input_files_sorted: true, row_key_cols: row_key_cols(["key"]), region: SleeperPartitionRegion::new(single_int_range("key", 2, 4)), - output: CompletionOptions::File { + output: OutputType::File { output_file: output.clone(), opts: SleeperParquetOptions::default(), }, @@ -147,7 +147,7 @@ async fn should_exclude_data_not_in_multidimensional_region() -> Result<(), Erro region_entry("key1", int_range(2, 4)), region_entry("key2", int_range(13, 23)), ])), - output: CompletionOptions::File { + output: OutputType::File { output_file: output.clone(), opts: SleeperParquetOptions::default(), }, @@ -192,7 +192,7 @@ async fn should_compact_with_second_column_row_key() -> Result<(), Error> { "key2", int_range(11, 25), )])), - output: CompletionOptions::File { + output: OutputType::File { output_file: output.clone(), opts: SleeperParquetOptions::default(), }, @@ -228,7 +228,7 @@ async fn should_merge_empty_files() -> Result<(), Error> { input_files_sorted: true, row_key_cols: row_key_cols(["key"]), region: SleeperPartitionRegion::new(single_int_range("key", 0, 5)), - output: CompletionOptions::File { + output: OutputType::File { output_file: output.clone(), opts: SleeperParquetOptions::default(), }, diff --git a/rust/sleeper_core/tests/query_test.rs b/rust/sleeper_core/tests/query_test.rs index 921f748400a..7a8b4b9ffe4 100644 --- a/rust/sleeper_core/tests/query_test.rs +++ b/rust/sleeper_core/tests/query_test.rs @@ -16,8 +16,8 @@ use color_eyre::eyre::{Error, bail}; use sleeper_core::{ - CommonConfig, CompletedOutput, CompletionOptions, LeafPartitionQueryConfig, - SleeperParquetOptions, SleeperPartitionRegion, run_query, + CommonConfig, CompletedOutput, LeafPartitionQueryConfig, OutputType, SleeperParquetOptions, + SleeperPartitionRegion, run_query, }; use tempfile::tempdir; use test_util::*; @@ -37,7 +37,7 @@ async fn should_return_subset_results_with_query_subset_of_partition() -> Result input_files_sorted: true, row_key_cols: row_key_cols(["key"]), region: SleeperPartitionRegion::new(single_int_range("key", 0, 5)), - output: CompletionOptions::ArrowRecordBatch, + output: OutputType::ArrowRecordBatch, ..Default::default() }; @@ -79,7 +79,7 @@ async fn should_return_subset_results_with_query_subset_of_partition_unsorted_in input_files_sorted: false, row_key_cols: row_key_cols(["key"]), region: SleeperPartitionRegion::new(single_int_range("key", 1, 7)), - output: CompletionOptions::ArrowRecordBatch, + output: OutputType::ArrowRecordBatch, ..Default::default() }; @@ -121,7 +121,7 @@ async fn should_return_subset_results_with_overlapping_query_and_partition_range input_files_sorted: true, row_key_cols: row_key_cols(["key"]), region: SleeperPartitionRegion::new(single_int_range("key", 0, 6)), - output: CompletionOptions::ArrowRecordBatch, + output: OutputType::ArrowRecordBatch, ..Default::default() }; @@ -163,7 +163,7 @@ async fn should_return_zero_results_with_non_overlapping_query_and_partition_ran input_files_sorted: true, row_key_cols: row_key_cols(["key"]), region: SleeperPartitionRegion::new(single_int_range("key", 0, 3)), - output: CompletionOptions::ArrowRecordBatch, + output: OutputType::ArrowRecordBatch, ..Default::default() }; @@ -204,7 +204,7 @@ async fn should_return_results_from_two_overlapping_query_ranges() -> Result<(), input_files_sorted: true, row_key_cols: row_key_cols(["key"]), region: SleeperPartitionRegion::new(single_int_range("key", -10, 11)), - output: CompletionOptions::ArrowRecordBatch, + output: OutputType::ArrowRecordBatch, ..Default::default() }; @@ -248,7 +248,7 @@ async fn should_return_results_from_two_non_overlapping_query_ranges() -> Result input_files_sorted: true, row_key_cols: row_key_cols(["key"]), region: SleeperPartitionRegion::new(single_int_range("key", -10, 11)), - output: CompletionOptions::ArrowRecordBatch, + output: OutputType::ArrowRecordBatch, ..Default::default() }; @@ -292,7 +292,7 @@ async fn should_error_with_no_query_ranges() -> Result<(), Error> { input_files_sorted: true, row_key_cols: row_key_cols(["key"]), region: SleeperPartitionRegion::new(single_int_range("key", 0, 3)), - output: CompletionOptions::ArrowRecordBatch, + output: OutputType::ArrowRecordBatch, ..Default::default() }; @@ -328,7 +328,7 @@ async fn should_error_when_arrow_output_with_sketches() -> Result<(), Error> { input_files_sorted: true, row_key_cols: row_key_cols(["key"]), region: SleeperPartitionRegion::new(single_int_range("key", 0, 3)), - output: CompletionOptions::ArrowRecordBatch, + output: OutputType::ArrowRecordBatch, ..Default::default() }; @@ -368,7 +368,7 @@ async fn should_return_results_as_file_with_sketch() -> Result<(), Error> { input_files_sorted: true, row_key_cols: row_key_cols(["key"]), region: SleeperPartitionRegion::new(single_int_range("key", 0, 6)), - output: CompletionOptions::File { + output: OutputType::File { output_file: output.clone(), opts: SleeperParquetOptions::default(), }, diff --git a/rust/sleeper_df/src/lib.rs b/rust/sleeper_df/src/lib.rs index f675d844a9a..c7e8192b259 100644 --- a/rust/sleeper_df/src/lib.rs +++ b/rust/sleeper_df/src/lib.rs @@ -19,8 +19,8 @@ use color_eyre::eyre::{bail, eyre}; use libc::{EFAULT, EINVAL, EIO, size_t}; use log::{LevelFilter, error, warn}; use sleeper_core::{ - AwsConfig, ColRange, CommonConfig, CompletionOptions, PartitionBound, SleeperParquetOptions, - SleeperPartitionRegion, run_compaction, + AwsConfig, ColRange, CommonConfig, CommonConfigBuilder, OutputType, PartitionBound, + SleeperParquetOptions, SleeperPartitionRegion, run_compaction, }; use std::{ borrow::Borrow, @@ -153,27 +153,31 @@ impl<'a> TryFrom<&'a FFICompactionParams> for CommonConfig<'a> { dict_enc_values: params.dict_enc_values, }; - Self::try_new( - unpack_aws_config(params)?, - unpack_string_array(params.input_files, params.input_files_len)? - .into_iter() - .map(Url::parse) - .collect::, _>>()?, - true, - row_key_cols, - unpack_string_array(params.sort_key_cols, params.sort_key_cols_len)? - .into_iter() - .map(String::from) - .collect(), - region, - CompletionOptions::File { + CommonConfigBuilder::new() + .aws_config(unpack_aws_config(params)?) + .input_files( + unpack_string_array(params.input_files, params.input_files_len)? + .into_iter() + .map(Url::parse) + .collect::, _>>()?, + ) + .input_files_sorted(true) + .row_key_cols(row_key_cols) + .sort_key_cols( + unpack_string_array(params.sort_key_cols, params.sort_key_cols_len)? + .into_iter() + .map(String::from) + .collect(), + ) + .region(region) + .output(OutputType::File { output_file: unsafe { CStr::from_ptr(params.output_file) } .to_str() .map(Url::parse)??, opts, - }, - iterator_config, - ) + }) + .iterator_config(iterator_config) + .build() } } From cc9d3b6cab37dab7243c8b5858935751176352f8 Mon Sep 17 00:00:00 2001 From: m09526 Date: Thu, 4 Sep 2025 15:04:52 +0000 Subject: [PATCH 26/28] Some query tests updated --- rust/sleeper_core/tests/query_test.rs | 141 +++++++++++++------------- 1 file changed, 68 insertions(+), 73 deletions(-) diff --git a/rust/sleeper_core/tests/query_test.rs b/rust/sleeper_core/tests/query_test.rs index 7a8b4b9ffe4..a4b4def1a64 100644 --- a/rust/sleeper_core/tests/query_test.rs +++ b/rust/sleeper_core/tests/query_test.rs @@ -32,14 +32,13 @@ async fn should_return_subset_results_with_query_subset_of_partition() -> Result write_file_of_ints(&file_1, "key", vec![1, 3])?; write_file_of_ints(&file_2, "key", vec![2, 4])?; - let input = CommonConfig { - input_files: Vec::from([file_1, file_2]), - input_files_sorted: true, - row_key_cols: row_key_cols(["key"]), - region: SleeperPartitionRegion::new(single_int_range("key", 0, 5)), - output: OutputType::ArrowRecordBatch, - ..Default::default() - }; + let input = CommonConfigBuilder::new() + .input_files(Vec::from([file_1, file_2])) + .input_files_sorted(true) + .row_key_cols(row_key_cols(["key"])) + .region(SleeperPartitionRegion::new(single_int_range("key", 0, 5))) + .output(OutputType::ArrowRecordBatch) + .build()?; let query_config = LeafPartitionQueryConfig { common: input, @@ -74,14 +73,13 @@ async fn should_return_subset_results_with_query_subset_of_partition_unsorted_in write_file_of_ints(&file_1, "key", vec![7, 3, 5, 1])?; write_file_of_ints(&file_2, "key", vec![8, 6, 2, 4])?; - let input = CommonConfig { - input_files: Vec::from([file_1, file_2]), - input_files_sorted: false, - row_key_cols: row_key_cols(["key"]), - region: SleeperPartitionRegion::new(single_int_range("key", 1, 7)), - output: OutputType::ArrowRecordBatch, - ..Default::default() - }; + let input = CommonConfigBuilder::new() + .input_files(vec![file_1, file_2]) + .input_files_sorted(false) + .row_key_cols(row_key_cols(["key"])) + .region(SleeperPartitionRegion::new(single_int_range("key", 1, 7))) + .output(OutputType::ArrowRecordBatch) + .build()?; let query_config = LeafPartitionQueryConfig { common: input, @@ -116,14 +114,13 @@ async fn should_return_subset_results_with_overlapping_query_and_partition_range write_file_of_ints(&file_1, "key", vec![1, 3, 5, 7, 9])?; write_file_of_ints(&file_2, "key", vec![2, 4, 6, 8, 10])?; - let input = CommonConfig { - input_files: Vec::from([file_1, file_2]), - input_files_sorted: true, - row_key_cols: row_key_cols(["key"]), - region: SleeperPartitionRegion::new(single_int_range("key", 0, 6)), - output: OutputType::ArrowRecordBatch, - ..Default::default() - }; + let input = CommonConfigBuilder::new() + .input_files(vec![file_1, file_2]) + .input_files_sorted(true) + .row_key_cols(row_key_cols(["key"])) + .region(SleeperPartitionRegion::new(single_int_range("key", 0, 6))) + .output(OutputType::ArrowRecordBatch) + .build()?; let query_config = LeafPartitionQueryConfig { common: input, @@ -158,14 +155,13 @@ async fn should_return_zero_results_with_non_overlapping_query_and_partition_ran write_file_of_ints(&file_1, "key", vec![1, 3, 5, 7, 9])?; write_file_of_ints(&file_2, "key", vec![2, 4, 6, 8, 10])?; - let input = CommonConfig { - input_files: Vec::from([file_1, file_2]), - input_files_sorted: true, - row_key_cols: row_key_cols(["key"]), - region: SleeperPartitionRegion::new(single_int_range("key", 0, 3)), - output: OutputType::ArrowRecordBatch, - ..Default::default() - }; + let input = CommonConfigBuilder::new() + .input_files(vec![file_1, file_2]) + .input_files_sorted(true) + .row_key_cols(row_key_cols(["key"])) + .region(SleeperPartitionRegion::new(single_int_range("key", 0, 3))) + .output(OutputType::ArrowRecordBatch) + .build()?; let query_config = LeafPartitionQueryConfig { common: input, @@ -199,14 +195,15 @@ async fn should_return_results_from_two_overlapping_query_ranges() -> Result<(), write_file_of_ints(&file_1, "key", vec![1, 3, 5, 7, 9])?; write_file_of_ints(&file_2, "key", vec![2, 4, 6, 8, 10])?; - let input = CommonConfig { - input_files: Vec::from([file_1, file_2]), - input_files_sorted: true, - row_key_cols: row_key_cols(["key"]), - region: SleeperPartitionRegion::new(single_int_range("key", -10, 11)), - output: OutputType::ArrowRecordBatch, - ..Default::default() - }; + let input = CommonConfigBuilder::new() + .input_files(vec![file_1, file_2]) + .input_files_sorted(true) + .row_key_cols(row_key_cols(["key"])) + .region(SleeperPartitionRegion::new(single_int_range( + "key", -10, 11, + ))) + .output(OutputType::ArrowRecordBatch) + .build()?; let query_config = LeafPartitionQueryConfig { common: input, @@ -243,14 +240,15 @@ async fn should_return_results_from_two_non_overlapping_query_ranges() -> Result write_file_of_ints(&file_1, "key", vec![1, 3, 5, 7, 9])?; write_file_of_ints(&file_2, "key", vec![2, 4, 6, 8, 10])?; - let input = CommonConfig { - input_files: Vec::from([file_1, file_2]), - input_files_sorted: true, - row_key_cols: row_key_cols(["key"]), - region: SleeperPartitionRegion::new(single_int_range("key", -10, 11)), - output: OutputType::ArrowRecordBatch, - ..Default::default() - }; + let input = CommonConfigBuilder::new() + .input_files(vec![file_1, file_2]) + .input_files_sorted(true) + .row_key_cols(row_key_cols(["key"])) + .region(SleeperPartitionRegion::new(single_int_range( + "key", -10, 11, + ))) + .output(OutputType::ArrowRecordBatch) + .build()?; let query_config = LeafPartitionQueryConfig { common: input, @@ -287,14 +285,13 @@ async fn should_error_with_no_query_ranges() -> Result<(), Error> { write_file_of_ints(&file_1, "key", vec![1, 3, 5, 7, 9])?; write_file_of_ints(&file_2, "key", vec![2, 4, 6, 8, 10])?; - let input = CommonConfig { - input_files: Vec::from([file_1, file_2]), - input_files_sorted: true, - row_key_cols: row_key_cols(["key"]), - region: SleeperPartitionRegion::new(single_int_range("key", 0, 3)), - output: OutputType::ArrowRecordBatch, - ..Default::default() - }; + let input = CommonConfigBuilder::new() + .input_files(vec![file_1, file_2]) + .input_files_sorted(true) + .row_key_cols(row_key_cols(["key"])) + .region(SleeperPartitionRegion::new(single_int_range("key", 0, 3))) + .output(OutputType::ArrowRecordBatch) + .build()?; let query_config = LeafPartitionQueryConfig { common: input, @@ -323,14 +320,13 @@ async fn should_error_when_arrow_output_with_sketches() -> Result<(), Error> { write_file_of_ints(&file_1, "key", vec![1])?; - let input = CommonConfig { - input_files: Vec::from([file_1]), - input_files_sorted: true, - row_key_cols: row_key_cols(["key"]), - region: SleeperPartitionRegion::new(single_int_range("key", 0, 3)), - output: OutputType::ArrowRecordBatch, - ..Default::default() - }; + let input = CommonConfigBuilder::new() + .input_files(vec![file_1]) + .input_files_sorted(true) + .row_key_cols(row_key_cols(["key"])) + .region(SleeperPartitionRegion::new(single_int_range("key", 0, 3))) + .output(OutputType::ArrowRecordBatch) + .build()?; let query_config = LeafPartitionQueryConfig { common: input, @@ -363,17 +359,16 @@ async fn should_return_results_as_file_with_sketch() -> Result<(), Error> { write_file_of_ints(&file_1, "key", vec![1, 3, 5, 7, 9])?; write_file_of_ints(&file_2, "key", vec![2, 4, 6, 8, 10])?; - let input = CommonConfig { - input_files: Vec::from([file_1, file_2]), - input_files_sorted: true, - row_key_cols: row_key_cols(["key"]), - region: SleeperPartitionRegion::new(single_int_range("key", 0, 6)), - output: OutputType::File { + let input = CommonConfigBuilder::new() + .input_files(vec![file_1, file_2]) + .input_files_sorted(true) + .row_key_cols(row_key_cols(["key"])) + .region(SleeperPartitionRegion::new(single_int_range("key", 0, 6))) + .output(OutputType::File { output_file: output.clone(), opts: SleeperParquetOptions::default(), - }, - ..Default::default() - }; + }) + .build()?; let query_config = LeafPartitionQueryConfig { common: input, From 9f297541920b127550e1653d0d34f3a08edf8bfb Mon Sep 17 00:00:00 2001 From: m09526 Date: Thu, 4 Sep 2025 15:44:47 +0000 Subject: [PATCH 27/28] Fixes for PR --- rust/sleeper_core/src/lib.rs | 67 ++++++++----- rust/sleeper_core/tests/compaction_test.rs | 111 ++++++++++----------- rust/sleeper_core/tests/query_test.rs | 4 +- 3 files changed, 100 insertions(+), 82 deletions(-) diff --git a/rust/sleeper_core/src/lib.rs b/rust/sleeper_core/src/lib.rs index 65502ad78ce..1b5e34eb967 100644 --- a/rust/sleeper_core/src/lib.rs +++ b/rust/sleeper_core/src/lib.rs @@ -320,13 +320,9 @@ pub struct CompactionResult { /// # use url::Url; /// # use aws_types::region::Region; /// # use std::collections::HashMap; -/// # use crate::sleeper_core::{run_compaction, CommonConfig, PartitionBound, ColRange, -/// # CompletionOptions, SleeperParquetOptions, SleeperPartitionRegion}; -/// let mut compaction_input = CommonConfig::default(); -/// compaction_input.input_files_sorted = true; -/// compaction_input.input_files = vec![Url::parse("file:///path/to/file1.parquet").unwrap()]; -/// compaction_input.output = CompletionOptions::File{ output_file: Url::parse("file:///path/to/output").unwrap(), opts: SleeperParquetOptions::default() }; -/// compaction_input.row_key_cols = vec!["key".into()]; +/// # use crate::sleeper_core::{run_compaction, CommonConfig, CommonConfigBuilder, PartitionBound, ColRange, +/// # OutputType, SleeperParquetOptions, SleeperPartitionRegion}; +/// # fn main() -> Result<(), color_eyre::eyre::Report> { /// let mut region : HashMap> = HashMap::new(); /// region.insert("key".into(), ColRange { /// lower : PartitionBound::String("a"), @@ -334,11 +330,18 @@ pub struct CompactionResult { /// upper: PartitionBound::String("h"), /// upper_inclusive: true, /// }); -/// compaction_input.region = SleeperPartitionRegion::new(region); -/// +/// let mut compaction_input = CommonConfigBuilder::new() +/// .input_files_sorted(true) +/// .input_files(vec![Url::parse("file:///path/to/file1.parquet").unwrap()]) +/// .output(OutputType::File{ output_file: Url::parse("file:///path/to/output").unwrap(), opts: SleeperParquetOptions::default() }) +/// .row_key_cols(vec!["key".into()]) +/// .region(SleeperPartitionRegion::new(region)) +/// .build()?; /// # tokio_test::block_on(async { /// let result = run_compaction(&compaction_input).await; -/// # }) +/// # }); +/// # Ok(()) +/// # } /// ``` /// /// # Errors @@ -361,14 +364,10 @@ pub async fn run_compaction(config: &CommonConfig<'_>) -> Result Result<(), color_eyre::eyre::Report> { /// let mut region : HashMap> = HashMap::new(); /// region.insert("key".into(), ColRange { /// lower : PartitionBound::String("a"), @@ -376,8 +375,13 @@ pub async fn run_compaction(config: &CommonConfig<'_>) -> Result> = HashMap::new(); @@ -391,7 +395,9 @@ pub async fn run_compaction(config: &CommonConfig<'_>) -> Result = vec![]; let row_key_cols = vec!["key".to_string()]; let region = SleeperPartitionRegion::default(); + let builder = CommonConfigBuilder::new() + .input_files(input_files) + .row_key_cols(row_key_cols) + .region(region); // When - let result = validate(&input_files, &row_key_cols, ®ion); + let result = builder.validate(); // Then assert!(result.is_err()); @@ -573,7 +583,13 @@ mod tests { )])); // When - let result = validate(&input_files, &row_key_cols, ®ion); + let builder = CommonConfigBuilder::new() + .input_files(input_files) + .row_key_cols(row_key_cols) + .region(region); + + // When + let result = builder.validate(); // Then assert!(result.is_err()); @@ -601,8 +617,13 @@ mod tests { }, )])); + let builder = CommonConfigBuilder::new() + .input_files(input_files) + .row_key_cols(row_key_cols) + .region(region); + // When - let result = validate(&input_files, &row_key_cols, ®ion); + let result = builder.validate(); // Then assert!(result.is_ok()); diff --git a/rust/sleeper_core/tests/compaction_test.rs b/rust/sleeper_core/tests/compaction_test.rs index 5ceb4578667..7b5b0b8b7a5 100644 --- a/rust/sleeper_core/tests/compaction_test.rs +++ b/rust/sleeper_core/tests/compaction_test.rs @@ -16,7 +16,7 @@ use arrow::datatypes::{DataType, Field, Schema}; use color_eyre::eyre::Error; use sleeper_core::{ - CommonConfig, OutputType, SleeperParquetOptions, SleeperPartitionRegion, run_compaction, + CommonConfigBuilder, OutputType, SleeperParquetOptions, SleeperPartitionRegion, run_compaction, }; use std::{collections::HashMap, path::Path, sync::Arc}; use tempfile::tempdir; @@ -34,17 +34,16 @@ async fn should_merge_two_files() -> Result<(), Error> { write_file_of_ints(&file_1, "key", vec![1, 3])?; write_file_of_ints(&file_2, "key", vec![2, 4])?; - let input = CommonConfig { - input_files: Vec::from([file_1, file_2]), - input_files_sorted: true, - row_key_cols: row_key_cols(["key"]), - region: SleeperPartitionRegion::new(single_int_range("key", 0, 5)), - output: OutputType::File { + let input = CommonConfigBuilder::new() + .input_files(vec![file_1, file_2]) + .input_files_sorted(true) + .row_key_cols(row_key_cols(["key"])) + .region(SleeperPartitionRegion::new(single_int_range("key", 0, 5))) + .output(OutputType::File { output_file: output.clone(), opts: SleeperParquetOptions::default(), - }, - ..Default::default() - }; + }) + .build()?; // When let result = run_compaction(&input).await?; @@ -67,17 +66,16 @@ async fn should_merge_files_with_overlapping_data() -> Result<(), Error> { write_file_of_ints(&file_1, "key", vec![1, 2])?; write_file_of_ints(&file_2, "key", vec![2, 3])?; - let input = CommonConfig { - input_files: Vec::from([file_1, file_2]), - input_files_sorted: true, - row_key_cols: row_key_cols(["key"]), - region: SleeperPartitionRegion::new(single_int_range("key", 0, 5)), - output: OutputType::File { + let input = CommonConfigBuilder::new() + .input_files(vec![file_1, file_2]) + .input_files_sorted(true) + .row_key_cols(row_key_cols(["key"])) + .region(SleeperPartitionRegion::new(single_int_range("key", 0, 5))) + .output(OutputType::File { output_file: output.clone(), opts: SleeperParquetOptions::default(), - }, - ..Default::default() - }; + }) + .build()?; // When let result = run_compaction(&input).await?; @@ -100,17 +98,16 @@ async fn should_exclude_data_not_in_region() -> Result<(), Error> { write_file_of_ints(&file_1, "key", vec![1, 2])?; write_file_of_ints(&file_2, "key", vec![3, 4])?; - let input = CommonConfig { - input_files: Vec::from([file_1, file_2]), - input_files_sorted: true, - row_key_cols: row_key_cols(["key"]), - region: SleeperPartitionRegion::new(single_int_range("key", 2, 4)), - output: OutputType::File { + let input = CommonConfigBuilder::new() + .input_files(vec![file_1, file_2]) + .input_files_sorted(true) + .row_key_cols(row_key_cols(["key"])) + .region(SleeperPartitionRegion::new(single_int_range("key", 2, 4))) + .output(OutputType::File { output_file: output.clone(), opts: SleeperParquetOptions::default(), - }, - ..Default::default() - }; + }) + .build()?; // When let result = run_compaction(&input).await?; @@ -139,20 +136,19 @@ async fn should_exclude_data_not_in_multidimensional_region() -> Result<(), Erro write_file(&file_1, &data_1)?; write_file(&file_2, &data_2)?; - let input = CommonConfig { - input_files: Vec::from([file_1, file_2]), - input_files_sorted: true, - row_key_cols: row_key_cols(["key1", "key2"]), - region: SleeperPartitionRegion::new(HashMap::from([ + let input = CommonConfigBuilder::new() + .input_files(vec![file_1, file_2]) + .input_files_sorted(true) + .row_key_cols(row_key_cols(["key1", "key2"])) + .region(SleeperPartitionRegion::new(HashMap::from([ region_entry("key1", int_range(2, 4)), region_entry("key2", int_range(13, 23)), - ])), - output: OutputType::File { + ]))) + .output(OutputType::File { output_file: output.clone(), opts: SleeperParquetOptions::default(), - }, - ..Default::default() - }; + }) + .build()?; // When let result = run_compaction(&input).await?; @@ -184,20 +180,19 @@ async fn should_compact_with_second_column_row_key() -> Result<(), Error> { write_file(&file_1, &data_1)?; write_file(&file_2, &data_2)?; - let input = CommonConfig { - input_files: Vec::from([file_1, file_2]), - input_files_sorted: true, - row_key_cols: row_key_cols(["key2"]), - region: SleeperPartitionRegion::new(HashMap::from([region_entry( + let input = CommonConfigBuilder::new() + .input_files(vec![file_1, file_2]) + .input_files_sorted(true) + .row_key_cols(row_key_cols(["key2"])) + .region(SleeperPartitionRegion::new(HashMap::from([region_entry( "key2", int_range(11, 25), - )])), - output: OutputType::File { + )]))) + .output(OutputType::File { output_file: output.clone(), opts: SleeperParquetOptions::default(), - }, - ..Default::default() - }; + }) + .build()?; // When let result = run_compaction(&input).await?; @@ -223,17 +218,19 @@ async fn should_merge_empty_files() -> Result<(), Error> { write_file_of_ints(&file_1, "key", vec![])?; write_file_of_ints(&file_2, "key", vec![])?; - let input = CommonConfig { - input_files: Vec::from([file_1, file_2]), - input_files_sorted: true, - row_key_cols: row_key_cols(["key"]), - region: SleeperPartitionRegion::new(single_int_range("key", 0, 5)), - output: OutputType::File { + let input = CommonConfigBuilder::new() + .input_files(vec![file_1, file_2]) + .input_files_sorted(true) + .row_key_cols(row_key_cols(["key"])) + .region(SleeperPartitionRegion::new(HashMap::from([region_entry( + "key", + int_range(0, 5), + )]))) + .output(OutputType::File { output_file: output.clone(), opts: SleeperParquetOptions::default(), - }, - ..Default::default() - }; + }) + .build()?; // When let result = run_compaction(&input).await?; diff --git a/rust/sleeper_core/tests/query_test.rs b/rust/sleeper_core/tests/query_test.rs index a4b4def1a64..55da57e7427 100644 --- a/rust/sleeper_core/tests/query_test.rs +++ b/rust/sleeper_core/tests/query_test.rs @@ -16,8 +16,8 @@ use color_eyre::eyre::{Error, bail}; use sleeper_core::{ - CommonConfig, CompletedOutput, LeafPartitionQueryConfig, OutputType, SleeperParquetOptions, - SleeperPartitionRegion, run_query, + CommonConfigBuilder, CompletedOutput, LeafPartitionQueryConfig, OutputType, + SleeperParquetOptions, SleeperPartitionRegion, run_query, }; use tempfile::tempdir; use test_util::*; From 4b18549204a443c45733b7518c4b9b87783f0500 Mon Sep 17 00:00:00 2001 From: patchwork01 <110390516+patchwork01@users.noreply.github.com> Date: Fri, 5 Sep 2025 08:42:13 +0000 Subject: [PATCH 28/28] 5408 Revert merge problem --- rust/Cargo.toml | 1 - 1 file changed, 1 deletion(-) diff --git a/rust/Cargo.toml b/rust/Cargo.toml index ca44b3b5821..0d8477f1088 100644 --- a/rust/Cargo.toml +++ b/rust/Cargo.toml @@ -35,7 +35,6 @@ bytes = { version = "1.10.1" } # Byte buffer for S3 uploadi cargo_metadata = { version = "0.22.0" } # Access cargo metadata programmtically chrono = { version = "0.4.41" } # Log helper clap = { version = "4.5.47" } # Cmd line args processing -tracing-subscriber = { version = "0.3.20" } color-eyre = { version = "0.6.5" } # Colourised version of `anyhow` cxx = { version = "1.0.175" } # Exception handling for Rust cxx-build = { version = "1.0.175" }