You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@arrow.apache.org by "Sietse Brouwer (Jira)" <ji...@apache.org> on 2020/10/03 23:47:00 UTC

[jira] [Commented] (ARROW-6774) [Rust] Reading parquet file is slow

    [ https://issues.apache.org/jira/browse/ARROW-6774?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17206895#comment-17206895 ] 

Sietse Brouwer commented on ARROW-6774:
---------------------------------------

[~alippai], I can't get parquet::arrow::ParquetFileArrowReader to be faster than parquet::file::reader::SerializedFileReader under commit `3fae71b10c42`. Timings below, code below that, conclusions at the bottom. Interesting times in *bold.*

 
||n_rows||include utf8-column||reader||iteration unit
_(loop does not iterate over rows within batches)_||time taken||
|50_000|yes|ParquetFileArrowReader|1 batch of 50k rows|14.9s|
|50_000|yes|ParquetFileArrowReader|10 batches of 5k rows|14.8s|
|50_000|yes|ParquetFileArrowReader|50k batches of 1 row|24.0s|
|50_000|yes|SerializedFileReader|get_row_iter|*14.5s*|
| | | | | |
|50_000|no|ParquetFileArrowReader|1 batch of 50k rows|*143ms*|
|50_000|no|ParquetFileArrowReader|10 batches of 5k rows|154ms|
|50_000|no|ParquetFileArrowReader|50k batches of 1 row|6.5s|
|50_000|no|SerializedFileReader| get_row_iter|*211ms*|

 

Here is the code I used to load the dataset with ParquetFileArrowReader (see also this version of [^main.rs]):

 
{code:java}
fn read_with_arrow(file: File) -> () {
    let file_reader = SerializedFileReader::new(file).unwrap();
    let mut arrow_reader = ParquetFileArrowReader::new(Rc::new(file_reader));
    println!("Arrow schema is: {}", arrow_reader.get_schema().unwrap());
    let mut record_batch_reader = arrow_reader
        .get_record_reader(/* batch size */ 50000)
        .unwrap();

    let start = Instant::now();
    while let Some(_record) = record_batch_reader.next_batch().unwrap() {
        // no-op
    };
    let duration = start.elapsed();

    println!("{:?}", duration);
}

{code}
 

Main observations:
 * we can't tell whether the slow loading when we include the UTF8 column is because UTF8 is slow to process, or because the column is very big (100 random Russian words per cell).
 * When the big UTF-8 column is included, iterating over every row with SerializedFileReader is as fast as iterating over a few batches with ParquetFileArrowReader. Even when you skip the rows within the batches!
 * Should I try this again with  (size 10k row * 3k Float64 columns) plus one small UTF-8 column?
 * I'm not even sure what result I'm trying to reproduce-or-falsify here... whether adding a small UTF-8 column causes disproportional slowdown? Or whether switching between SerializedFileReader and ParquetFileArrowReader causes slowdown? Right now, I feel like everything and nothing is in scope of the issue. I wouldn't mind if somebody made it narrower and clearer.

> [Rust] Reading parquet file is slow
> -----------------------------------
>
>                 Key: ARROW-6774
>                 URL: https://issues.apache.org/jira/browse/ARROW-6774
>             Project: Apache Arrow
>          Issue Type: Improvement
>          Components: Rust
>    Affects Versions: 0.15.0
>            Reporter: Adam Lippai
>            Priority: Major
>         Attachments: data.py, main.rs, main.rs
>
>
> Using the example at [https://github.com/apache/arrow/tree/master/rust/parquet] is slow.
> The snippet 
> {code:none}
> let reader = SerializedFileReader::new(file).unwrap();
> let mut iter = reader.get_row_iter(None).unwrap();
> let start = Instant::now();
> while let Some(record) = iter.next() {}
> let duration = start.elapsed();
> println!("{:?}", duration);
> {code}
> Runs for 17sec for a ~160MB parquet file.
> If there is a more effective way to load a parquet file, it would be nice to add it to the readme.
> P.S.: My goal is to construct an ndarray from it, I'd be happy for any tips.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)