Skip to content

Commit 53ae9db

Browse files
authored
feat(rust, python): ensure ooc sort works ooc with all-constant values (#6235)
1 parent a445cfc commit 53ae9db

File tree

5 files changed

+229
-124
lines changed

5 files changed

+229
-124
lines changed
Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -1,5 +1,6 @@
11
mod io;
22
mod ooc;
33
mod sink;
4+
mod source;
45

56
pub(crate) use sink::SortSink;
Lines changed: 7 additions & 110 deletions
Original file line numberDiff line numberDiff line change
@@ -1,19 +1,18 @@
11
use std::fs::DirEntry;
2-
use std::path::PathBuf;
32

43
use polars_core::prelude::*;
5-
use polars_core::utils::{_split_offsets, accumulate_dataframes_vertical_unchecked, split_df};
4+
use polars_core::utils::_split_offsets;
65
use polars_core::POOL;
76
use polars_io::ipc::IpcReader;
87
use polars_io::SerReader;
98
use polars_ops::prelude::*;
109
use rayon::prelude::*;
1110

1211
use crate::executors::sinks::sort::io::{block_thread_until_io_thread_done, DfIter, IOThread};
13-
use crate::executors::sinks::sort::sink::sort_accumulated;
14-
use crate::operators::{DataChunk, FinalizedSink, PExecutionContext, Source, SourceResult};
12+
use crate::executors::sinks::sort::source::SortSource;
13+
use crate::operators::FinalizedSink;
1514

16-
fn read_df(entry: &DirEntry) -> PolarsResult<DataFrame> {
15+
pub(super) fn read_df(entry: &DirEntry) -> PolarsResult<DataFrame> {
1716
let path = entry.path();
1817
let file = std::fs::File::open(path)?;
1918
IpcReader::new(file).set_rechunk(false).finish()
@@ -46,8 +45,8 @@ pub(super) fn sort_ooc(
4645
let assigned_parts = det_partitions(sort_col, &partitions, reverse);
4746

4847
// partition the dataframe into proper buckets
49-
let (iter, partition) = partition_df(df, &assigned_parts)?;
50-
io_thread.dump_iter(Some(partition), iter);
48+
let (iter, unique_assigned_parts) = partition_df(df, &assigned_parts)?;
49+
io_thread.dump_iter(Some(unique_assigned_parts), iter);
5150
}
5251
PolarsResult::Ok(())
5352
})
@@ -72,7 +71,7 @@ pub(super) fn sort_ooc(
7271
})
7372
.collect::<std::io::Result<Vec<_>>>()?;
7473

75-
let source = SortSource::new(files, idx, reverse, slice);
74+
let source = SortSource::new(files, idx, reverse, slice, partitions);
7675
Ok(FinalizedSink::Source(Box::new(source)))
7776
}
7877

@@ -104,105 +103,3 @@ fn partition_df(df: DataFrame, partitions: &IdxCa) -> PolarsResult<(DfIter, IdxC
104103
};
105104
Ok((out, partitions))
106105
}
107-
108-
pub struct SortSource {
109-
files: std::vec::IntoIter<(u32, PathBuf)>,
110-
n_threads: usize,
111-
sort_idx: usize,
112-
reverse: bool,
113-
chunk_offset: IdxSize,
114-
slice: Option<(i64, usize)>,
115-
finished: bool,
116-
}
117-
118-
impl SortSource {
119-
fn new(
120-
mut files: Vec<(u32, PathBuf)>,
121-
sort_idx: usize,
122-
reverse: bool,
123-
slice: Option<(i64, usize)>,
124-
) -> Self {
125-
files.sort_unstable_by_key(|entry| entry.0);
126-
127-
let n_threads = POOL.current_num_threads();
128-
let files = files.into_iter();
129-
130-
Self {
131-
files,
132-
n_threads,
133-
sort_idx,
134-
reverse,
135-
chunk_offset: 0,
136-
slice,
137-
finished: false,
138-
}
139-
}
140-
}
141-
142-
impl Source for SortSource {
143-
fn get_batches(&mut self, _context: &PExecutionContext) -> PolarsResult<SourceResult> {
144-
match self.files.next() {
145-
None => Ok(SourceResult::Finished),
146-
Some((_, path)) => {
147-
let files = std::fs::read_dir(path)?.collect::<std::io::Result<Vec<_>>>()?;
148-
149-
// early return
150-
if self.finished {
151-
return Ok(SourceResult::Finished);
152-
}
153-
154-
// read the files in a single partition in parallel
155-
let dfs = POOL.install(|| {
156-
files
157-
.par_iter()
158-
.map(read_df)
159-
.collect::<PolarsResult<Vec<DataFrame>>>()
160-
})?;
161-
let df = accumulate_dataframes_vertical_unchecked(dfs);
162-
// sort a single partition
163-
let current_slice = self.slice;
164-
let mut df = match &mut self.slice {
165-
None => sort_accumulated(df, self.sort_idx, self.reverse, None),
166-
Some((offset, len)) => {
167-
let df_len = df.height();
168-
assert!(*offset >= 0);
169-
let out = if *offset as usize > df_len {
170-
*offset -= df_len as i64;
171-
Ok(df.slice(0, 0))
172-
} else {
173-
let out =
174-
sort_accumulated(df, self.sort_idx, self.reverse, current_slice);
175-
*len = len.saturating_sub(df_len);
176-
*offset = 0;
177-
out
178-
};
179-
if *len == 0 {
180-
self.finished = true;
181-
}
182-
out
183-
}
184-
}?;
185-
186-
// convert to chunks
187-
// TODO: make utility functions to save these allocations
188-
let chunk_offset = self.chunk_offset;
189-
let dfs = split_df(&mut df, self.n_threads)?;
190-
self.chunk_offset += dfs.len() as IdxSize;
191-
let batch = dfs
192-
.into_iter()
193-
.enumerate()
194-
.map(|(i, df)| DataChunk {
195-
chunk_index: chunk_offset + i as IdxSize,
196-
data: df,
197-
})
198-
.collect();
199-
200-
Ok(SourceResult::GotMoreData(batch))
201-
}
202-
}
203-
}
204-
205-
fn fmt(&self) -> &str {
206-
"sort_source"
207-
}
208-
}

polars/polars-lazy/polars-pipe/src/executors/sinks/sort/sink.rs

Lines changed: 6 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -55,7 +55,7 @@ impl SortSink {
5555
dist_sample: vec![],
5656
};
5757
if ooc {
58-
eprintln!("Out of core sort forced");
58+
eprintln!("OOC sort forced");
5959
out.init_ooc().unwrap();
6060
}
6161
out
@@ -101,8 +101,11 @@ impl SortSink {
101101
while let Some(df) = self.chunks.pop_front() {
102102
if df.height() > 0 {
103103
// safety: we just asserted height > 0
104-
let sample = unsafe { df.get_columns()[self.sort_idx].get_unchecked(0) };
105-
self.dist_sample.push(sample.into_static().unwrap());
104+
let sample = unsafe {
105+
let s = &df.get_columns()[self.sort_idx];
106+
s.to_physical_repr().get_unchecked(0).into_static().unwrap()
107+
};
108+
self.dist_sample.push(sample);
106109

107110
let iot = self.io_thread.lock().unwrap();
108111
let iot = iot.as_ref().unwrap();
Lines changed: 188 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,188 @@
1+
use std::fs::DirEntry;
2+
use std::path::PathBuf;
3+
4+
use polars_core::prelude::*;
5+
use polars_core::utils::{accumulate_dataframes_vertical_unchecked, split_df};
6+
use polars_core::POOL;
7+
use rayon::prelude::*;
8+
9+
use crate::executors::sinks::sort::ooc::read_df;
10+
use crate::executors::sinks::sort::sink::sort_accumulated;
11+
use crate::operators::{DataChunk, PExecutionContext, Source, SourceResult};
12+
13+
pub struct SortSource {
14+
files: std::vec::IntoIter<(u32, PathBuf)>,
15+
n_threads: usize,
16+
sort_idx: usize,
17+
reverse: bool,
18+
chunk_offset: IdxSize,
19+
slice: Option<(i64, usize)>,
20+
finished: bool,
21+
22+
// The sorted partitions
23+
// are used check if a directory is already completely sorted
24+
// if the lower boundary of a partition is equal to the upper
25+
// boundary, the whole dictionary is already sorted
26+
// this dictionary may also be very large as in the extreme case
27+
// we sort a column with a constant value, then the binary search
28+
// ensures that all files will be written to a single folder
29+
// in that case we just read the files
30+
partitions: Series,
31+
sorted_directory_in_process: Option<std::vec::IntoIter<DirEntry>>,
32+
}
33+
34+
impl SortSource {
35+
pub(super) fn new(
36+
mut files: Vec<(u32, PathBuf)>,
37+
sort_idx: usize,
38+
reverse: bool,
39+
slice: Option<(i64, usize)>,
40+
partitions: Series,
41+
) -> Self {
42+
files.sort_unstable_by_key(|entry| entry.0);
43+
44+
let n_threads = POOL.current_num_threads();
45+
let files = files.into_iter();
46+
47+
Self {
48+
files,
49+
n_threads,
50+
sort_idx,
51+
reverse,
52+
chunk_offset: 0,
53+
slice,
54+
finished: false,
55+
partitions,
56+
sorted_directory_in_process: None,
57+
}
58+
}
59+
fn finish_batch(&mut self, dfs: Vec<DataFrame>) -> Vec<DataChunk> {
60+
// TODO: make utility functions to save these allocations
61+
let chunk_offset = self.chunk_offset;
62+
self.chunk_offset += dfs.len() as IdxSize;
63+
dfs.into_iter()
64+
.enumerate()
65+
.map(|(i, df)| DataChunk {
66+
chunk_index: chunk_offset + i as IdxSize,
67+
data: df,
68+
})
69+
.collect()
70+
}
71+
}
72+
73+
impl Source for SortSource {
74+
fn get_batches(&mut self, _context: &PExecutionContext) -> PolarsResult<SourceResult> {
75+
// early return
76+
if self.finished {
77+
return Ok(SourceResult::Finished);
78+
}
79+
80+
// this branch processes the directories containing a single sort key
81+
// e.g. the lower_bound == upper_bound
82+
if let Some(files) = &mut self.sorted_directory_in_process {
83+
let read = files
84+
.take(self.n_threads)
85+
.map(|entry| read_df(&entry))
86+
.collect::<PolarsResult<Vec<DataFrame>>>()?;
87+
let mut df = match (read.len(), &mut self.slice) {
88+
(0, _) => {
89+
// depleted directory, continue with normal sorting
90+
self.sorted_directory_in_process = None;
91+
return self.get_batches(_context);
92+
}
93+
// there is not slice and we got exactly enough files
94+
// so we return, happy path
95+
(n, None) if n == self.n_threads => {
96+
return Ok(SourceResult::GotMoreData(self.finish_batch(read)))
97+
}
98+
// there is a slice, so we concat and apply the slice
99+
// and then later split over the number of threads
100+
(_, Some((offset, len))) => {
101+
let df = accumulate_dataframes_vertical_unchecked(read);
102+
let df_len = df.height();
103+
104+
// whole batch can be skipped
105+
let out = if *offset as usize >= df_len {
106+
*offset -= df_len as i64;
107+
return self.get_batches(_context);
108+
} else {
109+
let out = df.slice(*offset, *len);
110+
*len = len.saturating_sub(df_len);
111+
*offset = 0;
112+
out
113+
};
114+
if *len == 0 {
115+
self.finished = true;
116+
}
117+
out
118+
}
119+
// The number of files read are lower than the number of
120+
// batches we have to return, so we first accumulate
121+
// and then split over the number of threads
122+
(_, None) => accumulate_dataframes_vertical_unchecked(read),
123+
};
124+
let batch = split_df(&mut df, self.n_threads)?;
125+
return Ok(SourceResult::GotMoreData(self.finish_batch(batch)));
126+
}
127+
128+
match self.files.next() {
129+
None => Ok(SourceResult::Finished),
130+
Some((partition, path)) => {
131+
let files = std::fs::read_dir(path)?.collect::<std::io::Result<Vec<_>>>()?;
132+
133+
// both lower and upper can fail.
134+
// lower can fail because the search_sorted can add the sort idx at the end of the array, which is i == len
135+
if let (Ok(lower), Ok(upper)) = (
136+
self.partitions.get(partition as usize),
137+
self.partitions.get(partition as usize + 1),
138+
) {
139+
if lower == upper && !files.is_empty() {
140+
let files = files.into_iter();
141+
self.sorted_directory_in_process = Some(files);
142+
return self.get_batches(_context);
143+
}
144+
}
145+
146+
// read the files in a single partition in parallel
147+
let dfs = POOL.install(|| {
148+
files
149+
.par_iter()
150+
.map(read_df)
151+
.collect::<PolarsResult<Vec<DataFrame>>>()
152+
})?;
153+
let df = accumulate_dataframes_vertical_unchecked(dfs);
154+
// sort a single partition
155+
let current_slice = self.slice;
156+
let mut df = match &mut self.slice {
157+
None => sort_accumulated(df, self.sort_idx, self.reverse, None),
158+
Some((offset, len)) => {
159+
let df_len = df.height();
160+
assert!(*offset >= 0);
161+
let out = if *offset as usize >= df_len {
162+
*offset -= df_len as i64;
163+
Ok(df.slice(0, 0))
164+
} else {
165+
let out =
166+
sort_accumulated(df, self.sort_idx, self.reverse, current_slice);
167+
*len = len.saturating_sub(df_len);
168+
*offset = 0;
169+
out
170+
};
171+
if *len == 0 {
172+
self.finished = true;
173+
}
174+
out
175+
}
176+
}?;
177+
178+
// convert to chunks
179+
let dfs = split_df(&mut df, self.n_threads)?;
180+
Ok(SourceResult::GotMoreData(self.finish_batch(dfs)))
181+
}
182+
}
183+
}
184+
185+
fn fmt(&self) -> &str {
186+
"sort_source"
187+
}
188+
}

0 commit comments

Comments
 (0)