polars-stream 0.53.0

Private crate for the streaming execution engine for the Polars DataFrame library
Documentation
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
use std::collections::VecDeque;
use std::sync::{Arc, Mutex};

use futures::StreamExt;
use polars_core::prelude::PlHashMap;
use polars_error::PolarsResult;
use polars_io::pl_async::get_runtime;
use polars_mem_engine::scan_predicate::initialize_scan_predicate;
use polars_plan::dsl::PredicateFileSkip;
use polars_utils::row_counter::RowCounter;
use polars_utils::slice_enum::Slice;

use crate::async_executor::{self, AbortOnDropHandle, TaskPriority};
use crate::async_primitives::connector::{self};
use crate::execute::StreamingExecutionState;
use crate::nodes::io_sources::multi_scan::components::bridge::{BridgeRecvPort, BridgeState};
use crate::nodes::io_sources::multi_scan::components::row_deletions::{
    DeletionFilesProvider, ExternalFilterMask, RowDeletionsInit,
};
use crate::nodes::io_sources::multi_scan::config::MultiScanConfig;
use crate::nodes::io_sources::multi_scan::functions::is_compressed_source;
use crate::nodes::io_sources::multi_scan::functions::resolve_slice::resolve_to_positive_slice;
use crate::nodes::io_sources::multi_scan::pipeline::models::{
    ExtraOperations, InitializedPipelineState, ResolvedSliceInfo, StartReaderArgsConstant,
};
use crate::nodes::io_sources::multi_scan::pipeline::tasks::attach_reader_to_bridge::AttachReaderToBridge;
use crate::nodes::io_sources::multi_scan::pipeline::tasks::bridge::spawn_bridge;
use crate::nodes::io_sources::multi_scan::pipeline::tasks::reader_starter::{
    InitializedReaderState, ReaderStarter,
};
use crate::nodes::io_sources::multi_scan::reader_interface::FileReader;
use crate::nodes::io_sources::multi_scan::reader_interface::capabilities::ReaderCapabilities;

pub fn initialize_multi_scan_pipeline(
    config: Arc<MultiScanConfig>,
    execution_state: StreamingExecutionState,
) -> InitializedPipelineState {
    assert!(config.num_pipelines() > 0);

    if config.verbose {
        eprintln!(
            "[MultiScanTaskInit]: \
            {} sources, \
            reader name: {}, \
            {:?}, \
            n_readers_pre_init: {}, \
            max_concurrent_scans: {}, \
            disable_morsel_split: {}",
            config.sources.len(),
            config.file_reader_builder.reader_name(),
            config.reader_capabilities(),
            config.n_readers_pre_init(),
            config.max_concurrent_scans(),
            config.disable_morsel_split,
        );
    }

    let bridge_state = Arc::new(Mutex::new(BridgeState::NotYetStarted));

    let (bridge_handle, bridge_recv_port_tx, phase_channel_tx) = spawn_bridge(bridge_state.clone());

    let task_handle =
        AbortOnDropHandle::new(async_executor::spawn(TaskPriority::Low, async move {
            finish_initialize_multi_scan_pipeline(config, bridge_recv_port_tx, execution_state)
                .await?;
            bridge_handle.await;
            Ok(())
        }));

    InitializedPipelineState {
        task_handle,
        phase_channel_tx,
        bridge_state,
    }
}

async fn finish_initialize_multi_scan_pipeline(
    config: Arc<MultiScanConfig>,
    bridge_recv_port_tx: connector::Sender<BridgeRecvPort>,
    execution_state: StreamingExecutionState,
) -> PolarsResult<()> {
    let verbose = config.verbose;

    let (skip_files_mask, predicate) = match config.predicate_file_skip_applied {
        None => initialize_scan_predicate(
            config.predicate.as_ref(),
            config.hive_parts.as_deref(),
            config.table_statistics.as_ref(),
            verbose,
        )?,
        Some(PredicateFileSkip {
            no_residual_predicate: false,
            original_len: _,
        }) => (None, config.predicate.as_ref()),
        Some(PredicateFileSkip {
            no_residual_predicate: true,
            original_len: _,
        }) => (None, None),
    };

    if let Some(skip_files_mask) = &skip_files_mask {
        assert_eq!(skip_files_mask.len(), config.sources.len());
    }

    if verbose {
        eprintln!(
            "[MultiScanTaskInit]: \
            predicate: {:?}, \
            skip files mask: {:?}, \
            predicate to reader: {:?}",
            config.predicate.is_some().then_some("<predicate>"),
            skip_files_mask.is_some().then_some("<skip_files>"),
            predicate.is_some().then_some("<predicate>"),
        )
    }

    #[expect(clippy::never_loop)]
    loop {
        if skip_files_mask
            .as_ref()
            .is_some_and(|x| x.num_skipped_files() == x.len())
        {
            if verbose {
                eprintln!("[MultiScanTaskInit]: early return (filter excludes all files)")
            }
        } else if config.pre_slice.as_ref().is_some_and(|x| x.len() == 0) {
            if cfg!(debug_assertions) {
                panic!("should quit earlier");
            }

            if verbose {
                eprintln!("[MultiScanTaskInit]: early return (pre_slice.len == 0)")
            }
        } else {
            break;
        }

        return Ok(());
    }

    let predicate = predicate.cloned();

    let num_pipelines = config.num_pipelines();
    let reader_capabilities = config.reader_capabilities();

    if config.sources.first().is_some_and(|x| x.run_async())
        && reader_capabilities.contains(ReaderCapabilities::NEEDS_FILE_CACHE_INIT)
    {
        // In cloud execution the entries may not exist at this point due to DSL resolution
        // happening on a separate machine.
        let sources = config.sources.clone();
        let cloud_options = config.cloud_options.clone();

        get_runtime()
            .spawn(async move {
                let sources = sources.clone();
                assert!(sources.as_paths().is_some());

                polars_io::file_cache::init_entries_from_uri_list(
                    (0..sources.len())
                        .map(move |i| sources.as_paths().unwrap().get(i).unwrap().clone()),
                    cloud_options.as_deref(),
                )
                .await
            })
            .await
            .unwrap()?;
    }

    // Row index should only be pushed if we have a predicate or negative slice as there is a
    // serial synchronization cost from needing to track the row position.
    if config.row_index.is_some() {
        debug_assert!(
            config.predicate.is_some() || matches!(config.pre_slice, Some(Slice::Negative { .. }))
        );
    }

    let ResolvedSliceInfo {
        scan_source_idx,
        row_index,
        pre_slice,
        initialized_readers,
        row_deletions,
    } = match config.pre_slice {
        // This can hugely benefit NDJSON, as it can read backwards.
        Some(Slice::Negative { .. })
            if config.sources.len() == 1
                && reader_capabilities.contains(ReaderCapabilities::NEGATIVE_PRE_SLICE)
                && (config.row_index.is_none()
                    || reader_capabilities.contains(ReaderCapabilities::ROW_INDEX))
                && (config.deletion_files.is_none()
                    || reader_capabilities.contains(ReaderCapabilities::EXTERNAL_FILTER_MASK))
                && !get_runtime()
                    .spawn(is_compressed_source(
                        config.sources.get(0).unwrap().into_owned()?,
                        config.cloud_options.clone(),
                    ))
                    .await
                    .unwrap()? =>
        {
            if verbose {
                eprintln!("[MultiScanTaskInit]: Single file negative slice");
            }

            ResolvedSliceInfo {
                scan_source_idx: 0,
                row_index: config.row_index.clone(),
                pre_slice: config.pre_slice.clone(),
                initialized_readers: None,
                row_deletions: Default::default(),
            }
        },
        _ => {
            if let Some(Slice::Negative { .. }) = config.pre_slice {
                if verbose {
                    eprintln!("[MultiScanTaskInit]: Begin resolving negative slice to positive");
                }
            }

            resolve_to_positive_slice(&config, &execution_state).await?
        },
    };

    let initialized_row_deletions: Arc<PlHashMap<usize, ExternalFilterMask>> =
        Arc::new(row_deletions);

    let cast_columns_policy = config.cast_columns_policy.clone();
    let missing_columns_policy = config.missing_columns_policy;
    let include_file_paths = config.include_file_paths.clone();

    let extra_ops = ExtraOperations {
        row_index,
        row_index_col_idx: config.row_index.as_ref().map_or(usize::MAX, |x| {
            config.final_output_schema.index_of(&x.name).unwrap()
        }),
        pre_slice,
        include_file_paths,
        file_path_col_idx: config.include_file_paths.as_ref().map_or(usize::MAX, |x| {
            config.final_output_schema.index_of(x).unwrap()
        }),
        predicate,
    };

    if verbose {
        eprintln!(
            "[MultiScanTaskInit]: \
            scan_source_idx: {}, \
            extra_ops: {:?}",
            scan_source_idx, &extra_ops,
        )
    }

    // Pre-initialized readers if we resolved a negative slice.
    let mut initialized_readers: VecDeque<(Box<dyn FileReader>, RowCounter)> = initialized_readers
        .map(|(idx, readers)| {
            // Sanity check
            assert_eq!(idx, scan_source_idx);
            readers
        })
        .unwrap_or_default();

    let has_row_index_or_slice = extra_ops.has_row_index_or_slice();

    let config = config.clone();

    // Buffered initialization stream. This concurrently calls `FileReader::initialize()`,
    // allowing for e.g. concurrent Parquet metadata fetch.
    let readers_init_iter = {
        let skip_files_mask = skip_files_mask.clone();

        let mut range = {
            // If a negative slice was initialized, the length of the initialized readers will be the exact
            // stopping position.
            let end = if initialized_readers.is_empty() {
                config.sources.len()
            } else {
                scan_source_idx + initialized_readers.len()
            };

            scan_source_idx..end
        };

        if verbose {
            let n_filtered = skip_files_mask.clone().map_or(0, |x| {
                x.sliced(range.start, range.len()).num_skipped_files()
            });
            let n_readers_init = range.len() - n_filtered;

            eprintln!(
                "\
                [MultiScanTaskInit]: Readers init: {} / ({} total) \
                (range: {:?}, filtered out: {})",
                n_readers_init,
                config.sources.len(),
                &range,
                n_filtered,
            )
        }

        if let Some(skip_files_mask) = &skip_files_mask {
            range.end = range
                .end
                .min(skip_files_mask.len() - skip_files_mask.trailing_skipped_files());
        }

        let range = range.filter(move |scan_source_idx| {
            let can_skip = !has_row_index_or_slice
                && skip_files_mask
                    .as_ref()
                    .is_some_and(|x| x.is_skipped_file(*scan_source_idx));

            !can_skip
        });

        let sources = config.sources.clone();
        let cloud_options = config.cloud_options.clone();
        let file_reader_builder = config.file_reader_builder.clone();
        let deletion_files_provider = DeletionFilesProvider::new(
            config.deletion_files.clone(),
            &execution_state,
            config.io_metrics(),
        );

        futures::stream::iter(range)
            .map(move |scan_source_idx| {
                let sources = sources.clone();
                let cloud_options = cloud_options.clone();
                let file_reader_builder = file_reader_builder.clone();
                let deletion_files_provider = deletion_files_provider.clone();
                let initialized_row_deletions = initialized_row_deletions.clone();

                let maybe_initialized = initialized_readers.pop_front();
                let scan_source = sources.get(scan_source_idx).unwrap().into_owned();

                AbortOnDropHandle::new(async_executor::spawn(TaskPriority::Low, async move {
                    let (scan_source, reader, n_rows_in_file) = async {
                        if verbose {
                            eprintln!("[MultiScan]: Initialize source {scan_source_idx}");
                        }

                        let scan_source = scan_source?;

                        if let Some((reader, n_rows_in_file)) = maybe_initialized {
                            return PolarsResult::Ok((scan_source, reader, Some(n_rows_in_file)));
                        }

                        let mut reader = file_reader_builder.build_file_reader(
                            scan_source.clone(),
                            cloud_options.clone(),
                            scan_source_idx,
                        );

                        reader.initialize().await?;
                        let opt_n_rows = reader
                            .fast_n_rows_in_file()
                            .await?
                            .map(|num_phys_rows| RowCounter::new(num_phys_rows, 0));

                        PolarsResult::Ok((scan_source, reader, opt_n_rows))
                    }
                    .await?;

                    let row_deletions: Option<RowDeletionsInit> = initialized_row_deletions
                        .get(&scan_source_idx)
                        .map(|x| RowDeletionsInit::Initialized(x.clone()))
                        .or_else(|| {
                            deletion_files_provider.spawn_row_deletions_init(
                                scan_source_idx,
                                cloud_options,
                                num_pipelines,
                                verbose,
                            )
                        });

                    Ok(InitializedReaderState {
                        scan_source_idx,
                        scan_source,
                        reader,
                        n_rows_in_file,
                        row_deletions,
                    })
                }))
            })
            .buffered(config.n_readers_pre_init().max(1))
    };

    let sources = config.sources.clone();
    let readers_init_iter = readers_init_iter.boxed();
    let hive_parts = config.hive_parts.clone();
    let final_output_schema = config.final_output_schema.clone();
    let file_projection_builder = config.file_projection_builder.clone();
    let max_concurrent_scans = config.max_concurrent_scans();
    let disable_morsel_split = config.disable_morsel_split;

    let (started_reader_tx, started_reader_rx) =
        tokio::sync::mpsc::channel(max_concurrent_scans.max(2) - 1);

    let reader_starter_handle = AbortOnDropHandle::new(async_executor::spawn(
        TaskPriority::Low,
        ReaderStarter {
            reader_capabilities,
            n_sources: sources.len(),

            readers_init_iter,
            started_reader_tx,
            max_concurrent_scans,
            skip_files_mask,
            extra_ops,
            constant_args: StartReaderArgsConstant {
                hive_parts,
                final_output_schema,
                reader_capabilities,
                file_projection_builder,
                cast_columns_policy,
                missing_columns_policy,
                forbid_extra_columns: config.forbid_extra_columns.clone(),
                num_pipelines,
                disable_morsel_split,
                verbose,
            },
            verbose,
        }
        .run(),
    ));

    let attach_to_bridge_handle = AbortOnDropHandle::new(async_executor::spawn(
        TaskPriority::Low,
        AttachReaderToBridge {
            started_reader_rx,
            bridge_recv_port_tx,
            verbose,
        }
        .run(),
    ));

    attach_to_bridge_handle.await?;
    reader_starter_handle.await?;

    Ok(())
}