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

Converted type is None according to Parquet Tools then utilizing logical types #3017

Closed
pacman82 opened this issue Nov 4, 2022 · 5 comments
Labels
bug parquet Changes to the parquet crate

Comments

@pacman82
Copy link

pacman82 commented Nov 4, 2022

Describe the bug

This regards the output written by the parquet crate. Declaring a column to containt a timestamp of microseconds using a LogicalType causes the written file to not have a converted type. At least according to parquet-tools.

To Reproduce

  1. Write a file tmp.par with a single column with type Timestamp of Microseconds, using a logical type.
use std::sync::Arc;

use parquet::{
    basic::{LogicalType, Repetition, Type},
    data_type::Int64Type,
    file::{properties::WriterProperties, writer::SerializedFileWriter},
    format::{MicroSeconds, TimeUnit},
    schema::types,
};

fn main() {
    let mut data = Vec::with_capacity(1024);
    let logical_type = LogicalType::Timestamp {
        is_adjusted_to_u_t_c: false,
        unit: TimeUnit::MICROS(MicroSeconds {}),
    };
    let field = Arc::new(
        types::Type::primitive_type_builder("col1", Type::INT64)
            .with_logical_type(Some(logical_type))
            .with_repetition(Repetition::REQUIRED)
            .build()
            .unwrap(),
    );
    let schema = Arc::new(
        types::Type::group_type_builder("schema")
            .with_fields(&mut vec![field])
            .build()
            .unwrap(),
    );

    // Write data
    let props = Arc::new(WriterProperties::builder().build());
    let mut writer = SerializedFileWriter::new(&mut data, schema, props).unwrap();
    let mut row_group_writer = writer.next_row_group().unwrap();
    let mut column_writer = row_group_writer.next_column().unwrap().unwrap();
    column_writer
        .typed::<Int64Type>()
        .write_batch(&[1, 2, 3, 4], None, None)
        .unwrap();
    column_writer.close().unwrap();
    row_group_writer.close().unwrap();
    writer.close().unwrap();

    // Write file for inspection with parqute tools
    std::fs::write("tmp.par", data).unwrap();
}
  1. Install parquet-tools in a virtual environment and inspect the file
pip install parquet-tools==0.2.11
parquet-tools inspect tmp.par

The resulting output indicates no Converted type

############ file meta data ############
created_by: parquet-rs version 26.0.0
num_columns: 1
num_rows: 4
num_row_groups: 1
format_version: 1.0
serialized_size: 143


############ Columns ############
col1

############ Column(col1) ############
name: col1
path: col1
max_definition_level: 0
max_repetition_level: 0
physical_type: INT64
logical_type: Timestamp(isAdjustedToUTC=false, timeUnit=microseconds, is_from_converted_type=false, force_set_converted_type=false)
converted_type (legacy): NONE
compression: UNCOMPRESSED (space_saved: 0%)

Expected behavior

I would have expected the converted type to show up in the Metainformation emitted by parquet-tools.

Additional context

Triggered by upstream odbc2parquet issue pacman82/odbc2parquet#284. Azure can not seem to handle the output since migration to LogicalType.
Previously misdiagnosed this to not set the converted type correctly in the schema information, this however does happen. See: #2984.

Thanks any help is appreciated!

@pacman82 pacman82 added the bug label Nov 4, 2022
@tustvold
Copy link
Contributor

tustvold commented Nov 6, 2022

I've narrowed this down to pyarrow not being able to read the converted type correctly.

>>> import pyarrow.parquet as pq
>>> pq.ParquetFile('tmp.par').schema.column(0).converted_type
'NONE'

However, fastparquet is able to read the converted type, as it is correctly encoded in the thrift definition

>>> ParquetFile('tmp.par').schema.schema_element('col1').converted_type
10

I also tried https://github.com/xitongsys/parquet-go/tree/master/tool/parquet-tools, which resulted in

./parquet-tools -cmd schema -file /home/raphael/repos/external/arrow-rs/parquet/tmp.par 
{
  "Tag": "name=Schema, repetitiontype=REQUIRED",
  "Fields": [
    {
      "Tag": "name=Col1, type=INT64, convertedtype=TIMESTAMP_MICROS, repetitiontype=REQUIRED"
    }
  ]
}

This leads me to think that this is actually a bug in the pyarrow, and therefore the C++ arrow implementation. Perhaps you might like to raise a bug there?

@pacman82
Copy link
Author

pacman82 commented Nov 6, 2022

Hi thanks for investigating this, so in context of the steps to reproduce you think the bug lies with parquet-tools ? I mean could be. However here is the output parquet-tools (the one installed via pip) gives me if I set the converted type instead of the logical type in code:

created_by: parquet-rs version 26.0.0
num_columns: 1
num_rows: 4
num_row_groups: 1
format_version: 1.0
serialized_size: 134


############ Columns ############
col1

############ Column(col1) ############
name: col1
path: col1
max_definition_level: 0
max_repetition_level: 0
physical_type: INT64
logical_type: Timestamp(isAdjustedToUTC=true, timeUnit=microseconds, is_from_converted_type=true, force_set_converted_type=false)
converted_type (legacy): TIMESTAMP_MICROS
compression: UNCOMPRESSED (space_saved: 0%)

@pacman82
Copy link
Author

pacman82 commented Nov 6, 2022

Hey, I found a (for me at least) new thing: If I set is_adjusted_to_u_t_c to true it works with the logical type. So the converted type is not written if is_adjusted_to_u_t_c is false, but if it is true it is. Is this intentional?

@tustvold
Copy link
Contributor

tustvold commented Nov 6, 2022

So the converted type is not written if is_adjusted_to_u_t_c is false, but if it is true it is. Is this intentional?

I can state categorically the converted type is always being written to the thrift metadata payload in the parquet file. As shown above readers not based on arrow C++ correctly find and read it, and as shown previously manually decoding the the metadata shows it is there. For some reason, intentional or a bug the C++ arrow reader is ignoring this in certain cases. You will need to take this up with them.

@pacman82
Copy link
Author

pacman82 commented Nov 6, 2022

Hello @tustvold ,

thanks for investigating. I will close the ticket. And open a ticken in apache/arrow. Thanks for everything.

Cheers, Markus

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

No branches or pull requests

3 participants