17
17
18
18
//! Helper functions for the table implementation
19
19
20
- use std:: mem;
21
20
use std:: sync:: Arc ;
22
21
23
22
use datafusion_catalog:: Session ;
@@ -40,7 +39,6 @@ use log::{debug, trace};
40
39
41
40
use datafusion_common:: tree_node:: { TreeNode , TreeNodeRecursion } ;
42
41
use datafusion_common:: { Column , DFSchema , DataFusionError } ;
43
- use datafusion_datasource:: file_groups:: FileGroup ;
44
42
use datafusion_expr:: { Expr , Volatility } ;
45
43
use datafusion_physical_expr:: create_physical_expr;
46
44
use object_store:: path:: Path ;
@@ -122,39 +120,6 @@ pub fn expr_applicable_for_cols(col_names: &[&str], expr: &Expr) -> bool {
122
120
/// The maximum number of concurrent listing requests
123
121
const CONCURRENCY_LIMIT : usize = 100 ;
124
122
125
- /// Partition the list of files into `n` groups
126
- pub fn split_files ( mut file_group : FileGroup , n : usize ) -> Vec < FileGroup > {
127
- if file_group. is_empty ( ) {
128
- return vec ! [ ] ;
129
- }
130
-
131
- // ObjectStore::list does not guarantee any consistent order and for some
132
- // implementations such as LocalFileSystem, it may be inconsistent. Thus
133
- // Sort files by path to ensure consistent plans when run more than once.
134
- file_group. files . sort_by ( |a, b| a. path ( ) . cmp ( b. path ( ) ) ) ;
135
-
136
- // effectively this is div with rounding up instead of truncating
137
- let chunk_size = file_group. len ( ) . div_ceil ( n) ;
138
- let mut chunks = Vec :: with_capacity ( n) ;
139
- let mut current_chunk = Vec :: with_capacity ( chunk_size) ;
140
- for file in file_group. files . drain ( ..) {
141
- current_chunk. push ( file) ;
142
- if current_chunk. len ( ) == chunk_size {
143
- let full_chunk = FileGroup :: new ( mem:: replace (
144
- & mut current_chunk,
145
- Vec :: with_capacity ( chunk_size) ,
146
- ) ) ;
147
- chunks. push ( full_chunk) ;
148
- }
149
- }
150
-
151
- if !current_chunk. is_empty ( ) {
152
- chunks. push ( FileGroup :: new ( current_chunk) )
153
- }
154
-
155
- chunks
156
- }
157
-
158
123
pub struct Partition {
159
124
/// The path to the partition, including the table prefix
160
125
path : Path ,
@@ -535,13 +500,13 @@ pub fn describe_partition(partition: &Partition) -> (&str, usize, Vec<&str>) {
535
500
mod tests {
536
501
use async_trait:: async_trait;
537
502
use datafusion_common:: config:: TableOptions ;
503
+ use datafusion_datasource:: file_groups:: FileGroup ;
538
504
use datafusion_execution:: config:: SessionConfig ;
539
505
use datafusion_execution:: runtime_env:: RuntimeEnv ;
540
506
use futures:: FutureExt ;
541
507
use object_store:: memory:: InMemory ;
542
508
use std:: any:: Any ;
543
509
use std:: ops:: Not ;
544
- // use futures::StreamExt;
545
510
546
511
use super :: * ;
547
512
use datafusion_expr:: {
@@ -561,32 +526,33 @@ mod tests {
561
526
new_partitioned_file( "e" ) ,
562
527
] ) ;
563
528
564
- let chunks = split_files ( files. clone ( ) , 1 ) ;
529
+ let chunks = files. clone ( ) . split_files ( 1 ) ;
565
530
assert_eq ! ( 1 , chunks. len( ) ) ;
566
531
assert_eq ! ( 5 , chunks[ 0 ] . len( ) ) ;
567
532
568
- let chunks = split_files ( files. clone ( ) , 2 ) ;
533
+ let chunks = files. clone ( ) . split_files ( 2 ) ;
569
534
assert_eq ! ( 2 , chunks. len( ) ) ;
570
535
assert_eq ! ( 3 , chunks[ 0 ] . len( ) ) ;
571
536
assert_eq ! ( 2 , chunks[ 1 ] . len( ) ) ;
572
537
573
- let chunks = split_files ( files. clone ( ) , 5 ) ;
538
+ let chunks = files. clone ( ) . split_files ( 5 ) ;
574
539
assert_eq ! ( 5 , chunks. len( ) ) ;
575
540
assert_eq ! ( 1 , chunks[ 0 ] . len( ) ) ;
576
541
assert_eq ! ( 1 , chunks[ 1 ] . len( ) ) ;
577
542
assert_eq ! ( 1 , chunks[ 2 ] . len( ) ) ;
578
543
assert_eq ! ( 1 , chunks[ 3 ] . len( ) ) ;
579
544
assert_eq ! ( 1 , chunks[ 4 ] . len( ) ) ;
580
545
581
- let chunks = split_files ( files , 123 ) ;
546
+ let chunks = files . clone ( ) . split_files ( 123 ) ;
582
547
assert_eq ! ( 5 , chunks. len( ) ) ;
583
548
assert_eq ! ( 1 , chunks[ 0 ] . len( ) ) ;
584
549
assert_eq ! ( 1 , chunks[ 1 ] . len( ) ) ;
585
550
assert_eq ! ( 1 , chunks[ 2 ] . len( ) ) ;
586
551
assert_eq ! ( 1 , chunks[ 3 ] . len( ) ) ;
587
552
assert_eq ! ( 1 , chunks[ 4 ] . len( ) ) ;
588
553
589
- let chunks = split_files ( FileGroup :: default ( ) , 2 ) ;
554
+ let mut empty_group = FileGroup :: default ( ) ;
555
+ let chunks = empty_group. split_files ( 2 ) ;
590
556
assert_eq ! ( 0 , chunks. len( ) ) ;
591
557
}
592
558
0 commit comments