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

ctx.read_parquet and ctx.register_parquet don't load schema metadata #9081

Closed
l45k opened this issue Jan 31, 2024 · 9 comments · Fixed by #9134
Closed

ctx.read_parquet and ctx.register_parquet don't load schema metadata #9081

l45k opened this issue Jan 31, 2024 · 9 comments · Fixed by #9134
Labels
bug Something isn't working help wanted Extra attention is needed

Comments

@l45k
Copy link

l45k commented Jan 31, 2024

Describe the bug

I try to load a parquet file with some metadata in its schema. Both ctx.read_parquet and ctx.register_parquet will not return the schema with the metadata, even if ParquetReadOptions::default().skip_metadata(false) is provided.

To Reproduce

To test, I create a table with metadata and write it to parquet using PyArrow:

import pyarrow as pa
import pyarrow.parquet as pq

table = pa.table(
  [
  pa.array([1.0, 1.0, 1.0, 1.0, 1.0], type=pa.int64()),
  ],
  names=["col1"],
  metadata = {"col1": "Some metadata"}
)
pq.write_table(table, "test.parquet")
  
print(pq.read_table("test.parquet").schema)

Output

col1: int64
-- schema metadata --
col1: 'Some metadata'

Next, reading or registering the file with DataFusion:

#[tokio::main]
async fn main() {
  let ctx = SessionContext::new();

  let table = ctx.read_parquet("test.parquet", ParquetReadOptions::default().skip_metadata(false)).await.unwrap();
  println!("Schema {:?}", table.schema());
  println!("Metadata {:?}", table.schema().metadata());

  ctx.register_parquet("t", "test.parquet", ParquetReadOptions::default().skip_metadata(false)).await.unwrap();
  println!("Schema {:?}", ctx.table("t").await.unwrap().schema());
  println!("Metadata {:?}", ctx.table("t").await.unwrap().schema().metadata());
}

Output:

Schema DFSchema { fields: [DFField { qualifier: Some(Bare { table: "?table?" }), field: Field { name: "col1", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} } }], metadata: {}, functional_dependencies: FunctionalDependencies { deps: [] } }
Metadata {}
Schema DFSchema { fields: [DFField { qualifier: Some(Bare { table: "t" }), field: Field { name: "col1", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} } }], metadata: {}, functional_dependencies: FunctionalDependencies { deps: [] } }
Metadata {}

Expected behavior

I would expect the metadata to be the same DataFusion and PyArrow.

Additional context

I use the following DataFusion and PyArrow versions:
datafusion = { version = "35.0.0", features = ["parquet"] }
pyarrow:15.0.0

@l45k l45k added the bug Something isn't working label Jan 31, 2024
@alamb
Copy link
Contributor

alamb commented Jan 31, 2024

I wonder if you set this configuration setting on the session context as well if the same problem still happens?

datafusion.execution.parquet.skip_metadata = false

https://arrow.apache.org/datafusion/user-guide/configs.html#configuration-settings

@l45k
Copy link
Author

l45k commented Feb 1, 2024

@alamb Thanks for the suggestions. I added the above configuration to my script

async fn main() {
    let mut session_config = SessionConfig::new();
    session_config = session_config.set_bool("datafusion.execution.parquet.skip_metadata", false);
    let ctx = SessionContext::new_with_config(session_config);
    println!("Session config {:?}", ctx.copied_config());

    let table = ctx.read_parquet("test.parquet", ParquetReadOptions::default().skip_metadata(false)).await.unwrap();
    println!("Schema {:?}", table.schema());
    println!("Metadata {:?}", table.schema().metadata());

    ctx.register_parquet("t", "test.parquet", ParquetReadOptions::default().skip_metadata(false)).await.unwrap();
    println!("Schema {:?}", ctx.table("t").await.unwrap().schema());
    println!("Metadata {:?}", ctx.table("t").await.unwrap().schema().metadata());

and checked that datafusion.execution.parquet.skip_metadata sessions config is set to false. However, I still get the same result without any metadata.

Schema DFSchema { fields: [DFField { qualifier: Some(Bare { table: "?table?" }), field: Field { name: "col1", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} } }], metadata: {}, functional_dependencies: FunctionalDependencies { deps: [] } }
Metadata {}
Schema DFSchema { fields: [DFField { qualifier: Some(Bare { table: "t" }), field: Field { name: "col1", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} } }], metadata: {}, functional_dependencies: FunctionalDependencies { deps: [] } }
Metadata {}

@alamb alamb added the help wanted Extra attention is needed label Feb 1, 2024
@brayanjuls
Copy link
Contributor

hey! will this do a good first issue? If so I would like to give it a shot.

@alamb
Copy link
Contributor

alamb commented Feb 2, 2024

hey! will this do a good first issue? If so I would like to give it a shot.

Thanks @brayanjuls -- I think it would be a great way to help - basically the initial task is to debug what is going on and then ideally fix it. Since I don't know what the problem is I don't really know how complicated it will turn out to be

It would be amazing if you had a chance to give it a look and I think it would be a great learning opportunity as well

@brayanjuls
Copy link
Contributor

brayanjuls commented Feb 4, 2024

@l45k @alamb I was able to reproduce the issue and also checked the unit test used in Datafusion to test this functionality and I notice a different approach to get the schema with metadata. It seems that you are able to get the metadata only after collecting the DataFrame, so for example the following works.

use datafusion::prelude::*;

#[tokio::main]
async fn main(){
    let ctx = SessionContext::new();
    
    ctx.register_parquet("ta", "test.parquet", ParquetReadOptions::default().skip_metadata(false)).await.unwrap();
    let df = ctx.table("ta").await.unwrap();
    
    let batches = df.collect().await.unwrap();
    for batch in &batches{
        println!("Schema {:?}", batch.schema());
        println!("Metadata {:?}", batch.schema().metadata());
    }
    
} 

output:

Schema Schema { fields: [Field { name: "col1", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }], metadata: {"col1": "Some metadata"} }
Metadata {"col1": "Some metadata"}

I am still investigating if it make sense to be able to access the metadata before collecting the DataFrame.

@alamb
Copy link
Contributor

alamb commented Feb 5, 2024

That is a great find @brayanjuls -- nice 🕵️ . I wonder if the code that does schema inference is ignorning metadata somehow

This code seems to imply it is handling metadata 🤔

https://github.com/apache/arrow-datafusion/blob/840499fbd3c25bf82a6ee0641ef6888079e096da/datafusion/core/src/datasource/file_format/parquet.rs#L193-L232

@brayanjuls
Copy link
Contributor

@alamb I debugged this section of the code and it is not ignoring the metadata. The issue happens in the following code when listing the table. The metadata is being ignored when building the table schema. Creating the schema from the schema instead of fields in line 551 solved the issue in my local environment. This is the modification I did let mut builder = SchemaBuilder::from(file_schema.deref().clone()); . I am not sure if I should look for other functions where similar situations my be happening please let me know.

https://github.com/apache/arrow-datafusion/blob/840499fbd3c25bf82a6ee0641ef6888079e096da/datafusion/core/src/datasource/listing/table.rs#L541-L568

@alamb
Copy link
Contributor

alamb commented Feb 5, 2024

The metadata is being ignored when building the table schema. Creating the schema from the schema instead of fields in line 551 solved the issue in my local environment

Awesome -- thank you @brayanjuls 🙏 -- do you by any chance have time to make a PR with a fix? If not, I think other contributors may be interested and willing to help too

I am not sure if I should look for other functions where similar situations my be happening please let me know.

That would be super helpful if you have time

@brayanjuls
Copy link
Contributor

Yes, I would like to open a PR to fix it.

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

Successfully merging a pull request may close this issue.

3 participants