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

Cannot write a column of type DataType::List containing a DataType::Struct to parquet with parallel writing #8851

Closed
vigimite opened this issue Jan 13, 2024 · 5 comments · Fixed by #8923
Labels
bug Something isn't working

Comments

@vigimite
Copy link

Describe the bug

When trying to write a column that is of type List that contains a Struct, the parquet writer throws an error Error: ParquetError(General("Incorrect number of rows, expected 4 != 0 rows")). This seems to be a regression as this works fine in datafusion v32.0.0 but not in v33 or v34. It also works using write_json instead of write_parquet

Example dataframe:

+------------------------------------+
| filters                            |
+------------------------------------+
| [{filterTypeId: 3, label: LABEL3}] |
| [{filterTypeId: 2, label: LABEL2}] |
+------------------------------------+

To Reproduce

dependencies (working):

[dependencies]
tokio = { version = "1.35.1", features = ["macros"] }
datafusion = { version = "32.0.0", features = ["backtrace"] }

dependencies (broken):

[dependencies]
tokio = { version = "1.35.1", features = ["macros"] }
datafusion = { version = "33.0.0", features = ["backtrace"] }

example.json

{"filters":[{"filterTypeId":3,"label":"LABEL3"}]}
{"filters":[{"filterTypeId":2,"label":"LABEL2"}]}

main.rs

use datafusion::{dataframe::DataFrameWriteOptions, error::DataFusionError, prelude::*};

#[tokio::main]
async fn main() -> Result<(), DataFusionError> {
    let ctx = SessionContext::new();

    let df = ctx
        .read_json("example.json", NdJsonReadOptions::default())
        .await?;

    df.write_parquet("result", DataFrameWriteOptions::default(), None)
        .await?;
    Ok(())
}

Expected behavior

The parquet writer supports writing this kind of datatype as in v32

Additional context

Maybe related to: apache/arrow-rs#1744

I found this issue trying to debug a different one that came up while trying to upgrade from v32 to v34. If the struct contains a timestamp the error instead becomes a Error: Internal("Unable to send array to writer!") with a source error internal error: entered unreachable code: cannot downcast Int64 to byte array.

An example of such a df:

+---------------------------------------------------------------------------------+
| filters                                                                         |
+---------------------------------------------------------------------------------+
| [{assignmentStartTs: 2023-11-11T11:11:11.000Z, filterTypeId: 3, label: LABEL1}] |
| [{assignmentStartTs: 2023-11-11T11:11:11.000Z, filterTypeId: 2, label: LABEL2}] |
+--------------------------------------------------------------------------------+

I tried to debug this issue myself looking into the arrow-rs implementation however I didn't manage to find the relevant commit that could have changed this behavior. Also I wasn't sure if I should open the bug in this project or in the arrow-rs project so I hope this is ok 😃.

@vigimite vigimite added the bug Something isn't working label Jan 13, 2024
@alamb alamb added the regression Something that used to work no longer does label Jan 13, 2024
@alamb
Copy link
Contributor

alamb commented Jan 13, 2024

Also I wasn't sure if I should open the bug in this project or in the arrow-rs project so I hope this is ok 😃.

I think this project is the correct place to start. We can distill your example down and see if we can get an arrow-rs only reproducer

@tustvold do you have any thoughts?

@tustvold
Copy link
Contributor

tustvold commented Jan 13, 2024

It could be something in the arrow upgrade from 47 to 48, but nothing immediately springs out to me.

Edit: the DF parallel parquet writer looks to have landed around then, I wonder if it is the cause.

@devinjdangelo
Copy link
Contributor

devinjdangelo commented Jan 13, 2024

Edit: the DF parallel parquet writer looks to have landed around then, I wonder if it is the cause.

We can try the following setting and see if that fixes this bug to verify if it is the cause.

datafusion.execution.parquet.allow_single_file_parallelism false

Edit: I was able to reproduce the error on latest main with the above setting to true, and I can confirm that setting it to false fixes the error.

@vigimite
Copy link
Author

thank you all for the quick response! For now setting the variable is a good workaround for my usecase 👍🏻.

@alamb alamb mentioned this issue Jan 14, 2024
3 tasks
@alamb alamb changed the title Cannot write a column of type DataType::List containing a DataType::Struct to parquet Cannot write a column of type DataType::List containing a DataType::Struct to parquet with parallel writing Jan 16, 2024
@alamb alamb removed the regression Something that used to work no longer does label Jan 16, 2024
@alamb
Copy link
Contributor

alamb commented Jan 16, 2024

@devinjdangelo set the feature allow_single_file_parallelism to false by default in #8854 and added test coverage so this is no longer a "regression" (instead it is a bug in a feature that is not enabled by default)

Let's leave this issue open until the code is sorted but I think it is much less severe now

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
bug Something isn't working
Projects
None yet
Development

Successfully merging a pull request may close this issue.

4 participants