datafusion-datasource-arrow 53.1.0

datafusion-datasource-arrow
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
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements.  See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership.  The ASF licenses this file
// to you 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.

//! Execution plan for reading Arrow IPC files
//!
//! # Naming Note
//!
//! The naming in this module can be confusing:
//! - `ArrowFileOpener` handles the Arrow IPC **file format**
//!   (with footer, supports parallel reading)
//! - `ArrowStreamFileOpener` handles the Arrow IPC **stream format**
//!   (without footer, sequential only)
//! - `ArrowSource` is the unified `FileSource` implementation that uses either opener
//!   depending on the format specified at construction
//!
//! Despite the name "ArrowStreamFileOpener", it still reads from files - the "Stream"
//! refers to the Arrow IPC stream format, not streaming I/O. Both formats can be stored
//! in files on disk or object storage.

use std::sync::Arc;
use std::{any::Any, io::Cursor};

use datafusion_datasource::{TableSchema, as_file_source};

use arrow::buffer::Buffer;
use arrow::ipc::reader::{FileDecoder, FileReader, StreamReader};
use datafusion_common::error::Result;
use datafusion_common::exec_datafusion_err;
use datafusion_datasource::PartitionedFile;
use datafusion_datasource::file::FileSource;
use datafusion_datasource::file_scan_config::FileScanConfig;
use datafusion_datasource::projection::{ProjectionOpener, SplitProjection};
use datafusion_physical_expr_common::sort_expr::LexOrdering;
use datafusion_physical_plan::metrics::ExecutionPlanMetricsSet;
use datafusion_physical_plan::projection::ProjectionExprs;

use datafusion_datasource::file_stream::FileOpenFuture;
use datafusion_datasource::file_stream::FileOpener;
use futures::StreamExt;
use itertools::Itertools;
use object_store::{GetOptions, GetRange, GetResultPayload, ObjectStore, ObjectStoreExt};

/// Enum indicating which Arrow IPC format to use
#[derive(Clone, Copy, Debug)]
enum ArrowFormat {
    /// Arrow IPC file format (with footer, supports parallel reading)
    File,
    /// Arrow IPC stream format (without footer, sequential only)
    Stream,
}

/// `FileOpener` for Arrow IPC stream format. Supports only sequential reading.
pub(crate) struct ArrowStreamFileOpener {
    object_store: Arc<dyn ObjectStore>,
    projection: Option<Vec<usize>>,
}

impl FileOpener for ArrowStreamFileOpener {
    fn open(&self, partitioned_file: PartitionedFile) -> Result<FileOpenFuture> {
        if partitioned_file.range.is_some() {
            return Err(exec_datafusion_err!(
                "ArrowStreamFileOpener does not support range-based reading"
            ));
        }
        let object_store = Arc::clone(&self.object_store);
        let projection = self.projection.clone();

        Ok(Box::pin(async move {
            let r = object_store
                .get(&partitioned_file.object_meta.location)
                .await?;

            let stream = match r.payload {
                #[cfg(not(target_arch = "wasm32"))]
                GetResultPayload::File(file, _) => futures::stream::iter(
                    StreamReader::try_new(file.try_clone()?, projection.clone())?,
                )
                .map(|r| r.map_err(Into::into))
                .boxed(),
                GetResultPayload::Stream(_) => {
                    let bytes = r.bytes().await?;
                    let cursor = Cursor::new(bytes);
                    futures::stream::iter(StreamReader::try_new(
                        cursor,
                        projection.clone(),
                    )?)
                    .map(|r| r.map_err(Into::into))
                    .boxed()
                }
            };

            Ok(stream)
        }))
    }
}

/// `FileOpener` for Arrow IPC file format. Supports range-based parallel reading.
pub(crate) struct ArrowFileOpener {
    object_store: Arc<dyn ObjectStore>,
    projection: Option<Vec<usize>>,
}

impl FileOpener for ArrowFileOpener {
    fn open(&self, partitioned_file: PartitionedFile) -> Result<FileOpenFuture> {
        let object_store = Arc::clone(&self.object_store);
        let projection = self.projection.clone();

        Ok(Box::pin(async move {
            let range = partitioned_file.range.clone();
            match range {
                None => {
                    let r = object_store
                        .get(&partitioned_file.object_meta.location)
                        .await?;
                    let stream = match r.payload {
                        #[cfg(not(target_arch = "wasm32"))]
                        GetResultPayload::File(file, _) => futures::stream::iter(
                            FileReader::try_new(file.try_clone()?, projection.clone())?,
                        )
                        .map(|r| r.map_err(Into::into))
                        .boxed(),
                        GetResultPayload::Stream(_) => {
                            let bytes = r.bytes().await?;
                            let cursor = Cursor::new(bytes);
                            futures::stream::iter(FileReader::try_new(
                                cursor,
                                projection.clone(),
                            )?)
                            .map(|r| r.map_err(Into::into))
                            .boxed()
                        }
                    };

                    Ok(stream)
                }
                Some(range) => {
                    // range is not none, the file maybe split into multiple parts to scan in parallel
                    // get footer_len firstly
                    let get_option = GetOptions {
                        range: Some(GetRange::Suffix(10)),
                        ..Default::default()
                    };
                    let get_result = object_store
                        .get_opts(&partitioned_file.object_meta.location, get_option)
                        .await?;
                    let footer_len_buf = get_result.bytes().await?;
                    let footer_len = arrow_ipc::reader::read_footer_length(
                        footer_len_buf[..].try_into().unwrap(),
                    )?;
                    // read footer according to footer_len
                    let get_option = GetOptions {
                        range: Some(GetRange::Suffix(10 + (footer_len as u64))),
                        ..Default::default()
                    };
                    let get_result = object_store
                        .get_opts(&partitioned_file.object_meta.location, get_option)
                        .await?;
                    let footer_buf = get_result.bytes().await?;
                    let footer = arrow_ipc::root_as_footer(
                        footer_buf[..footer_len].try_into().unwrap(),
                    )
                    .map_err(|err| {
                        exec_datafusion_err!("Unable to get root as footer: {err:?}")
                    })?;
                    // build decoder according to footer & projection
                    let schema =
                        arrow_ipc::convert::fb_to_schema(footer.schema().unwrap());
                    let mut decoder = FileDecoder::new(schema.into(), footer.version());
                    if let Some(projection) = projection {
                        decoder = decoder.with_projection(projection);
                    }
                    let dict_ranges = footer
                        .dictionaries()
                        .iter()
                        .flatten()
                        .map(|block| {
                            let block_len =
                                block.bodyLength() as u64 + block.metaDataLength() as u64;
                            let block_offset = block.offset() as u64;
                            block_offset..block_offset + block_len
                        })
                        .collect_vec();
                    let dict_results = object_store
                        .get_ranges(&partitioned_file.object_meta.location, &dict_ranges)
                        .await?;
                    for (dict_block, dict_result) in
                        footer.dictionaries().iter().flatten().zip(dict_results)
                    {
                        decoder
                            .read_dictionary(dict_block, &Buffer::from(dict_result))?;
                    }

                    // filter recordbatches according to range
                    let recordbatches = footer
                        .recordBatches()
                        .iter()
                        .flatten()
                        .filter(|block| {
                            let block_offset = block.offset() as u64;
                            block_offset >= range.start as u64
                                && block_offset < range.end as u64
                        })
                        .copied()
                        .collect_vec();

                    let recordbatch_ranges = recordbatches
                        .iter()
                        .map(|block| {
                            let block_len =
                                block.bodyLength() as u64 + block.metaDataLength() as u64;
                            let block_offset = block.offset() as u64;
                            block_offset..block_offset + block_len
                        })
                        .collect_vec();

                    let recordbatch_results = object_store
                        .get_ranges(
                            &partitioned_file.object_meta.location,
                            &recordbatch_ranges,
                        )
                        .await?;

                    let stream = futures::stream::iter(
                        recordbatches
                            .into_iter()
                            .zip(recordbatch_results)
                            .filter_map(move |(block, data)| {
                                decoder
                                    .read_record_batch(&block, &Buffer::from(data))
                                    .transpose()
                            }),
                    )
                    .map(|r| r.map_err(Into::into))
                    .boxed();

                    Ok(stream)
                }
            }
        }))
    }
}

/// `FileSource` for both Arrow IPC file and stream formats
#[derive(Clone)]
pub struct ArrowSource {
    format: ArrowFormat,
    metrics: ExecutionPlanMetricsSet,
    projection: SplitProjection,
    table_schema: TableSchema,
}

impl ArrowSource {
    /// Creates an [`ArrowSource`] for file format
    pub fn new_file_source(table_schema: impl Into<TableSchema>) -> Self {
        let table_schema = table_schema.into();
        Self {
            format: ArrowFormat::File,
            metrics: ExecutionPlanMetricsSet::new(),
            projection: SplitProjection::unprojected(&table_schema),
            table_schema,
        }
    }

    /// Creates an [`ArrowSource`] for stream format
    pub fn new_stream_file_source(table_schema: impl Into<TableSchema>) -> Self {
        let table_schema = table_schema.into();
        Self {
            format: ArrowFormat::Stream,
            metrics: ExecutionPlanMetricsSet::new(),
            projection: SplitProjection::unprojected(&table_schema),
            table_schema,
        }
    }
}

impl FileSource for ArrowSource {
    fn create_file_opener(
        &self,
        object_store: Arc<dyn ObjectStore>,
        _base_config: &FileScanConfig,
        _partition: usize,
    ) -> Result<Arc<dyn FileOpener>> {
        let split_projection = self.projection.clone();

        let opener: Arc<dyn FileOpener> = match self.format {
            ArrowFormat::File => Arc::new(ArrowFileOpener {
                object_store,
                projection: Some(split_projection.file_indices.clone()),
            }),
            ArrowFormat::Stream => Arc::new(ArrowStreamFileOpener {
                object_store,
                projection: Some(split_projection.file_indices.clone()),
            }),
        };
        ProjectionOpener::try_new(
            split_projection,
            opener,
            self.table_schema.file_schema(),
        )
    }

    fn as_any(&self) -> &dyn Any {
        self
    }

    fn with_batch_size(&self, _batch_size: usize) -> Arc<dyn FileSource> {
        Arc::new(Self { ..self.clone() })
    }

    fn metrics(&self) -> &ExecutionPlanMetricsSet {
        &self.metrics
    }

    fn file_type(&self) -> &str {
        match self.format {
            ArrowFormat::File => "arrow",
            ArrowFormat::Stream => "arrow_stream",
        }
    }

    fn repartitioned(
        &self,
        target_partitions: usize,
        repartition_file_min_size: usize,
        output_ordering: Option<LexOrdering>,
        config: &FileScanConfig,
    ) -> Result<Option<FileScanConfig>> {
        match self.format {
            ArrowFormat::Stream => {
                // The Arrow IPC stream format doesn't support range-based parallel reading
                // because it lacks a footer with the information that would be needed to
                // make range-based parallel reading practical. Without the data in the
                // footer you would either need to read the the entire file and record the
                // offsets of the record batches and dictionaries, essentially recreating
                // the footer's contents, or else each partition would need to read the
                // entire file up to the correct offset which is a lot of duplicate I/O.
                // We're opting to avoid that entirely by only acting on a single partition
                // and reading sequentially.
                Ok(None)
            }
            ArrowFormat::File => {
                // Use the default trait implementation logic for file format
                use datafusion_datasource::file_groups::FileGroupPartitioner;

                if config.file_compression_type.is_compressed() {
                    return Ok(None);
                }

                let repartitioned_file_groups_option = FileGroupPartitioner::new()
                    .with_target_partitions(target_partitions)
                    .with_repartition_file_min_size(repartition_file_min_size)
                    .with_preserve_order_within_groups(output_ordering.is_some())
                    .repartition_file_groups(&config.file_groups);

                if let Some(repartitioned_file_groups) = repartitioned_file_groups_option
                {
                    let mut source = config.clone();
                    source.file_groups = repartitioned_file_groups;
                    return Ok(Some(source));
                }
                Ok(None)
            }
        }
    }

    fn table_schema(&self) -> &TableSchema {
        &self.table_schema
    }

    fn try_pushdown_projection(
        &self,
        projection: &ProjectionExprs,
    ) -> Result<Option<Arc<dyn FileSource>>> {
        let mut source = self.clone();
        source.projection = SplitProjection::new(
            self.table_schema().file_schema(),
            &source.projection.source.try_merge(projection)?,
        );
        Ok(Some(Arc::new(source)))
    }

    fn projection(&self) -> Option<&ProjectionExprs> {
        Some(&self.projection.source)
    }
}

/// `FileOpener` wrapper for both Arrow IPC file and stream formats
pub struct ArrowOpener {
    pub inner: Arc<dyn FileOpener>,
}

impl FileOpener for ArrowOpener {
    fn open(&self, partitioned_file: PartitionedFile) -> Result<FileOpenFuture> {
        self.inner.open(partitioned_file)
    }
}

impl ArrowOpener {
    /// Creates a new [`ArrowOpener`]
    pub fn new(inner: Arc<dyn FileOpener>) -> Self {
        Self { inner }
    }

    pub fn new_file_opener(
        object_store: Arc<dyn ObjectStore>,
        projection: Option<Vec<usize>>,
    ) -> Self {
        Self {
            inner: Arc::new(ArrowFileOpener {
                object_store,
                projection,
            }),
        }
    }

    pub fn new_stream_file_opener(
        object_store: Arc<dyn ObjectStore>,
        projection: Option<Vec<usize>>,
    ) -> Self {
        Self {
            inner: Arc::new(ArrowStreamFileOpener {
                object_store,
                projection,
            }),
        }
    }
}

impl From<ArrowSource> for Arc<dyn FileSource> {
    fn from(source: ArrowSource) -> Self {
        as_file_source(source)
    }
}

#[cfg(test)]
mod tests {
    use std::{fs::File, io::Read};

    use arrow::datatypes::{DataType, Field, Schema};
    use arrow_ipc::reader::{FileReader, StreamReader};
    use bytes::Bytes;
    use datafusion_datasource::file_scan_config::FileScanConfigBuilder;
    use datafusion_execution::object_store::ObjectStoreUrl;
    use object_store::memory::InMemory;

    use super::*;

    #[tokio::test]
    async fn test_file_opener_without_ranges() -> Result<()> {
        for filename in ["example.arrow", "example_stream.arrow"] {
            let path = format!("tests/data/{filename}");
            let path_str = path.as_str();
            let mut file = File::open(path_str)?;
            let file_size = file.metadata()?.len();

            let mut buffer = Vec::new();
            file.read_to_end(&mut buffer)?;
            let bytes = Bytes::from(buffer);

            let object_store = Arc::new(InMemory::new());
            let partitioned_file = PartitionedFile::new(filename, file_size);
            object_store
                .put(&partitioned_file.object_meta.location, bytes.into())
                .await?;

            let schema = match FileReader::try_new(File::open(path_str)?, None) {
                Ok(reader) => reader.schema(),
                Err(_) => StreamReader::try_new(File::open(path_str)?, None)?.schema(),
            };

            let source: Arc<dyn FileSource> = if filename.contains("stream") {
                Arc::new(ArrowSource::new_stream_file_source(schema))
            } else {
                Arc::new(ArrowSource::new_file_source(schema))
            };

            let scan_config = FileScanConfigBuilder::new(
                ObjectStoreUrl::local_filesystem(),
                source.clone(),
            )
            .build();

            let file_opener = source.create_file_opener(object_store, &scan_config, 0)?;
            let mut stream = file_opener.open(partitioned_file)?.await?;

            assert!(stream.next().await.is_some());
        }

        Ok(())
    }

    #[tokio::test]
    async fn test_file_opener_with_ranges() -> Result<()> {
        let filename = "example.arrow";
        let path = format!("tests/data/{filename}");
        let path_str = path.as_str();
        let mut file = File::open(path_str)?;
        let file_size = file.metadata()?.len();

        let mut buffer = Vec::new();
        file.read_to_end(&mut buffer)?;
        let bytes = Bytes::from(buffer);

        let object_store = Arc::new(InMemory::new());
        let partitioned_file = PartitionedFile::new_with_range(
            filename.into(),
            file_size,
            0,
            (file_size - 1) as i64,
        );
        object_store
            .put(&partitioned_file.object_meta.location, bytes.into())
            .await?;

        let schema = FileReader::try_new(File::open(path_str)?, None)?.schema();

        let source = Arc::new(ArrowSource::new_file_source(schema));

        let scan_config = FileScanConfigBuilder::new(
            ObjectStoreUrl::local_filesystem(),
            source.clone(),
        )
        .build();

        let file_opener = source.create_file_opener(object_store, &scan_config, 0)?;
        let mut stream = file_opener.open(partitioned_file)?.await?;

        assert!(stream.next().await.is_some());

        Ok(())
    }

    #[tokio::test]
    async fn test_stream_opener_errors_with_ranges() -> Result<()> {
        let filename = "example_stream.arrow";
        let path = format!("tests/data/{filename}");
        let path_str = path.as_str();
        let mut file = File::open(path_str)?;
        let file_size = file.metadata()?.len();

        let mut buffer = Vec::new();
        file.read_to_end(&mut buffer)?;
        let bytes = Bytes::from(buffer);

        let object_store = Arc::new(InMemory::new());
        let partitioned_file = PartitionedFile::new_with_range(
            filename.into(),
            file_size,
            0,
            (file_size - 1) as i64,
        );
        object_store
            .put(&partitioned_file.object_meta.location, bytes.into())
            .await?;

        let schema = StreamReader::try_new(File::open(path_str)?, None)?.schema();

        let source = Arc::new(ArrowSource::new_stream_file_source(schema));

        let scan_config = FileScanConfigBuilder::new(
            ObjectStoreUrl::local_filesystem(),
            source.clone(),
        )
        .build();

        let file_opener = source.create_file_opener(object_store, &scan_config, 0)?;
        let result = file_opener.open(partitioned_file);
        assert!(result.is_err());

        Ok(())
    }

    #[tokio::test]
    async fn test_arrow_stream_repartitioning_not_supported() -> Result<()> {
        let schema =
            Arc::new(Schema::new(vec![Field::new("f0", DataType::Int64, false)]));
        let source = ArrowSource::new_stream_file_source(schema);

        let config = FileScanConfigBuilder::new(
            ObjectStoreUrl::local_filesystem(),
            Arc::new(source.clone()) as Arc<dyn FileSource>,
        )
        .build();

        for target_partitions in [2, 4, 8, 16] {
            let result =
                source.repartitioned(target_partitions, 1024 * 1024, None, &config)?;

            assert!(
                result.is_none(),
                "Stream format should not support repartitioning with {target_partitions} partitions",
            );
        }

        Ok(())
    }

    #[tokio::test]
    async fn test_stream_opener_with_projection() -> Result<()> {
        let filename = "example_stream.arrow";
        let path = format!("tests/data/{filename}");
        let path_str = path.as_str();
        let mut file = File::open(path_str)?;
        let file_size = file.metadata()?.len();

        let mut buffer = Vec::new();
        file.read_to_end(&mut buffer)?;
        let bytes = Bytes::from(buffer);

        let object_store = Arc::new(InMemory::new());
        let partitioned_file = PartitionedFile::new(filename, file_size);
        object_store
            .put(&partitioned_file.object_meta.location, bytes.into())
            .await?;

        let opener = ArrowStreamFileOpener {
            object_store,
            projection: Some(vec![0]), // just the first column
        };

        let mut stream = opener.open(partitioned_file)?.await?;

        if let Some(batch) = stream.next().await {
            let batch = batch?;
            assert_eq!(
                batch.num_columns(),
                1,
                "Projection should result in 1 column"
            );
        } else {
            panic!("Expected at least one batch");
        }

        Ok(())
    }
}