Skip to content

Nested list levels calculated incorrectly if list has 0 length element #282

@nevi-me

Description

@nevi-me

Describe the bug

First documented in #270 (comment).

When trying to write some combinations of nested Arrow data to Parquet, we trigger a bounds error on the level calculations.
The most obvious thing that could be going wrong is that we're not correctly accounting for empty list slot vs null list slot.

This is because the error gets triggered around the logic that does this.

To Reproduce

Try the below test:

#[test]
fn test_write_ipc_nested_lists() {
    let fields = vec![Field::new(
        "list_a",
        DataType::List(Box::new(Field::new(
            "list_b",
            DataType::List(Box::new(Field::new(
                "struct_c",
                DataType::Struct(vec![
                    Field::new("prim_d", DataType::Boolean, true),
                    Field::new(
                        "list_e",
                        DataType::LargeList(Box::new(Field::new(
                            "string_f",
                            DataType::LargeUtf8,
                            true,
                        ))),
                        false,
                    ),
                ]),
                true,
            ))),
            false,
        ))),
        true,
    )];
    let schema = Arc::new(Schema::new(fields));
    // making this nullable guarantees that one of the list items will be empty, triggering the error
    let batch = arrow::util::data_gen::create_random_batch(schema, 3, 0.35, 0.6).unwrap();

    // write ipc (to read in pyarrow, and write parquet from pyarrow)
    let file = File::create("arrow_nested_random.arrow").unwrap();
    let mut writer =
        arrow::ipc::writer::FileWriter::try_new(file, batch.schema().as_ref()).unwrap();
    writer.write(&batch).unwrap();
    writer.finish().unwrap();

    let file = File::create("arrow_nested_random_rust.parquet").unwrap();
    let mut writer =
        ArrowWriter::try_new(file.try_clone().unwrap(), batch.schema(), None)
            .expect("Unable to write file");

    // this will trigger the error in question
    writer.write(&batch).unwrap();
    writer.close().unwrap();
}

Expected behavior

The parquet file should be written correctly, and pyarrow or Spark should be able to read the data correctly.

Additional context

Not sure

Metadata

Metadata

Assignees

No one assigned

    Labels

    bugdevelopment-processRelated to development process of arrow-rsparquetChanges to the parquet crate

    Type

    No type

    Projects

    No projects

    Milestone

    No milestone

    Relationships

    None yet

    Development

    No branches or pull requests

    Issue actions