Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Array reader for list columns fails to decode if batches fall on row group boundaries #2025

Closed
tjwilson90 opened this issue Jul 7, 2022 · 2 comments · Fixed by #2027
Closed
Labels
bug parquet Changes to the parquet crate

Comments

@tjwilson90
Copy link
Contributor

This appears very similar to https://issues.apache.org/jira/browse/ARROW-9790, but specifically for list columns.

use arrow::array::{Int32Builder, ListBuilder};
use arrow::datatypes::{DataType, Field, Schema};
use arrow::record_batch::RecordBatch;
use parquet::arrow::{ArrowReader, ArrowWriter, ParquetFileArrowReader};
use parquet::file::properties::WriterProperties;
use parquet::file::reader::SerializedFileReader;
use std::error::Error;
use std::sync::Arc;
use tempfile::NamedTempFile;

fn main() -> Result<(), Box<dyn Error>> {
    let schema = Arc::new(Schema::new(vec![
        Field::new("int", DataType::Int32, false),
        Field::new(
            "list",
            DataType::List(Box::new(Field::new("item", DataType::Int32, true))),
            false,
        ),
    ]));
    let temp_file = NamedTempFile::new()?;

    let mut writer = ArrowWriter::try_new(
        temp_file.reopen()?,
        schema.clone(),
        Some(
            WriterProperties::builder()
                .set_max_row_group_size(8)
                .build(),
        ),
    )?;
    for _ in 0..2 {
        let mut int_builder = Int32Builder::new(10);
        let mut list_builder = ListBuilder::new(Int32Builder::new(10));
        for i in 0..10 {
            int_builder.append_value(i)?;
            list_builder.append(true)?;
        }
        let batch = RecordBatch::try_new(
            schema.clone(),
            vec![
                Arc::new(int_builder.finish()),
                Arc::new(list_builder.finish()),
            ],
        )?;
        writer.write(&batch)?;
    }
    writer.close()?;

    let file_reader = Arc::new(SerializedFileReader::new(temp_file.reopen()?)?);
    let mut file_reader = ParquetFileArrowReader::new(file_reader);
    let mut record_reader = file_reader.get_record_reader(8)?;
    assert_eq!(8, record_reader.next().unwrap()?.num_rows());
    assert_eq!(8, record_reader.next().unwrap()?.num_rows());
    assert_eq!(4, record_reader.next().unwrap()?.num_rows());
    Ok(())
}

Fails with Error: ParquetError("Parquet error: Not all children array length are the same!")

@tjwilson90 tjwilson90 added the bug label Jul 7, 2022
@tustvold
Copy link
Contributor

tustvold commented Jul 7, 2022

Thank you, I can reproduce this and know why it occurs, it relates to how RecordReader detects the end of records when it reaches the end of a chunk. I need to have a think about how best to fix this.

tustvold added a commit to tustvold/arrow-rs that referenced this issue Jul 8, 2022
tustvold added a commit to tustvold/arrow-rs that referenced this issue Jul 8, 2022
tustvold added a commit to tustvold/arrow-rs that referenced this issue Jul 8, 2022
tustvold added a commit to tustvold/arrow-rs that referenced this issue Jul 8, 2022
tustvold added a commit to tustvold/arrow-rs that referenced this issue Jul 8, 2022
tustvold added a commit to tustvold/arrow-rs that referenced this issue Jul 8, 2022
@alamb
Copy link
Contributor

alamb commented Jul 8, 2022

Thanks for the report @tjwilson90 -- @tustvold is actively working on a fix and we'll try and include it in 18.0.0

tustvold added a commit that referenced this issue Jul 8, 2022
* Fix record delimiting on row group boundaries (#2025)

* Better tests

* Clippy
tustvold added a commit to tustvold/arrow-rs that referenced this issue Jul 8, 2022
@alamb alamb added the parquet Changes to the parquet crate label Jul 8, 2022
tustvold added a commit to tustvold/arrow-rs that referenced this issue Jul 8, 2022
tustvold added a commit to tustvold/arrow-rs that referenced this issue Jul 8, 2022
tustvold added a commit that referenced this issue Jul 15, 2022
* Add more tests of #2025

* Update parquet/src/arrow/arrow_reader.rs

Co-authored-by: Andrew Lamb <andrew@nerdnetworks.org>

Co-authored-by: Andrew Lamb <andrew@nerdnetworks.org>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
bug parquet Changes to the parquet crate
Projects
None yet
Development

Successfully merging a pull request may close this issue.

3 participants