@@ -2,14 +2,16 @@ use std::fmt;
22use std:: sync:: Arc ;
33
44use datafusion:: config:: ConfigOptions ;
5- use datafusion:: datasource:: physical_plan:: { FileGroupPartitioner , FileScanConfig , FileStream } ;
5+ use datafusion:: datasource:: listing:: PartitionedFile ;
6+ use datafusion:: datasource:: physical_plan:: { FileScanConfig , FileStream } ;
67use datafusion_common:: { project_schema, Result as DFResult , Statistics } ;
78use datafusion_execution:: { SendableRecordBatchStream , TaskContext } ;
89use datafusion_physical_expr:: { EquivalenceProperties , Partitioning , PhysicalExpr } ;
910use datafusion_physical_plan:: metrics:: ExecutionPlanMetricsSet ;
1011use datafusion_physical_plan:: {
1112 DisplayAs , DisplayFormatType , ExecutionMode , ExecutionPlan , PlanProperties ,
1213} ;
14+ use itertools:: Itertools ;
1315use vortex_array:: Context ;
1416
1517use crate :: persistent:: opener:: VortexFileOpener ;
@@ -47,7 +49,7 @@ impl VortexExec {
4749
4850 let plan_properties = PlanProperties :: new (
4951 EquivalenceProperties :: new_with_orderings ( projected_schema, & orderings) ,
50- Partitioning :: UnknownPartitioning ( 1 ) ,
52+ Partitioning :: UnknownPartitioning ( file_scan_config . file_groups . len ( ) ) ,
5153 ExecutionMode :: Bounded ,
5254 ) ;
5355
@@ -60,6 +62,7 @@ impl VortexExec {
6062 ctx,
6163 } )
6264 }
65+
6366 pub ( crate ) fn into_arc ( self ) -> Arc < dyn ExecutionPlan > {
6467 Arc :: new ( self ) as _
6568 }
@@ -103,6 +106,7 @@ impl ExecutionPlan for VortexExec {
103106 partition : usize ,
104107 context : Arc < TaskContext > ,
105108 ) -> DFResult < SendableRecordBatchStream > {
109+ log:: debug!( "Executing partition {partition}" ) ;
106110 let object_store = context
107111 . runtime_env ( )
108112 . object_store ( & self . file_scan_config . object_store_url ) ?;
@@ -128,21 +132,57 @@ impl ExecutionPlan for VortexExec {
128132 fn repartitioned (
129133 & self ,
130134 target_partitions : usize ,
131- config : & ConfigOptions ,
135+ _config : & ConfigOptions ,
132136 ) -> DFResult < Option < Arc < dyn ExecutionPlan > > > {
133- let repartition_file_min_size = config. optimizer . repartition_file_min_size ;
134- let repartitioned_file_groups_option = FileGroupPartitioner :: new ( )
135- . with_target_partitions ( target_partitions)
136- . with_repartition_file_min_size ( repartition_file_min_size)
137- . with_preserve_order_within_groups ( self . properties ( ) . output_ordering ( ) . is_some ( ) )
138- . repartition_file_groups ( & self . file_scan_config . file_groups ) ;
137+ let file_groups = self . file_scan_config . file_groups . clone ( ) ;
138+
139+ let repartitioned_file_groups = repartition_by_count ( file_groups, target_partitions) ;
139140
140141 let mut new_plan = self . clone ( ) ;
141- if let Some ( repartitioned_file_groups) = repartitioned_file_groups_option {
142- let mut config = new_plan. file_scan_config ;
143- config = config. with_file_groups ( repartitioned_file_groups) ;
144- new_plan. file_scan_config = config;
145- }
142+ let mut config = new_plan. file_scan_config ;
143+ let num_partitions = repartitioned_file_groups. len ( ) ;
144+
145+ log:: debug!( "VortexExec repartitioned to {num_partitions} partitions" ) ;
146+ config = config. with_file_groups ( repartitioned_file_groups) ;
147+ new_plan. file_scan_config = config;
148+ new_plan. plan_properties . partitioning = Partitioning :: UnknownPartitioning ( num_partitions) ;
149+
146150 Ok ( Some ( Arc :: new ( new_plan) ) )
147151 }
148152}
153+
154+ fn repartition_by_count (
155+ file_groups : Vec < Vec < PartitionedFile > > ,
156+ desired_partitions : usize ,
157+ ) -> Vec < Vec < PartitionedFile > > {
158+ let all_files = file_groups. into_iter ( ) . concat ( ) ;
159+
160+ let approx_files_per_partition = all_files. len ( ) . div_ceil ( desired_partitions) ;
161+ let mut repartitioned_file_groups = Vec :: default ( ) ;
162+
163+ for chunk in & all_files. into_iter ( ) . chunks ( approx_files_per_partition) {
164+ repartitioned_file_groups. push ( chunk. collect :: < Vec < _ > > ( ) ) ;
165+ }
166+
167+ repartitioned_file_groups
168+ }
169+
170+ #[ cfg( test) ]
171+ mod tests {
172+ use super :: * ;
173+
174+ #[ test]
175+ fn basic_repartition_test ( ) {
176+ let input_file_groups = vec ! [ vec![
177+ PartitionedFile :: new( "a" , 0 ) ,
178+ PartitionedFile :: new( "b" , 0 ) ,
179+ PartitionedFile :: new( "c" , 0 ) ,
180+ PartitionedFile :: new( "d" , 0 ) ,
181+ PartitionedFile :: new( "e" , 0 ) ,
182+ ] ] ;
183+
184+ let file_groups = repartition_by_count ( input_file_groups, 2 ) ;
185+
186+ assert_eq ! ( file_groups. len( ) , 2 ) ;
187+ }
188+ }
0 commit comments