You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@arrow.apache.org by "Joris Van den Bossche (Jira)" <ji...@apache.org> on 2021/11/23 12:54:00 UTC

[jira] [Comment Edited] (ARROW-14767) Categorical int8 index types written as int32 in parquet files

    [ https://issues.apache.org/jira/browse/ARROW-14767?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17447989#comment-17447989 ] 

Joris Van den Bossche edited comment on ARROW-14767 at 11/23/21, 12:53 PM:
---------------------------------------------------------------------------

Thanks for the report!

I can reproduce your issue. Looking at the written Parquet file's schema, you can see that the column "B" is indicated to be of type "String":

{code}
In [29]: parquet_metadata = pq.ParquetFile(
    ...:     filesystem.open_input_file(where),
    ...: )

In [30]: parquet_metadata.schema
Out[30]: 
<pyarrow._parquet.ParquetSchema object at 0x7f60be97fe00>
required group field_id=-1 schema {
  optional int32 field_id=-1 A (Int(bitWidth=8, isSigned=true));
  optional binary field_id=-1 B (String);
}
{code}

Parquet still uses dictionary encoding for compression reasons, but it is not a separate type in their type system.

Arrow has the ability to read such dictionary encoded string columns directly into an Arrow dictionary type. But at that point it doesn't preserve the int8 type for the indices, since that is not part of the parquet spec, and rather uses Arrow's default of int32 indices.

Now, we do write the Arrow schema into the Parquet file metadata, so we can use that information to ensure a more faithful roundtrip of Arrow tables to Parquet and back.

That doesn't happen here, but in principle could be done (although that will probably require an additional cast after the fact)


was (Author: jorisvandenbossche):
Thanks for the report!

I can reproduce your issue. Looking at the written Parquet file's schema, it actually uses int8:

{code}
In [29]: parquet_metadata = pq.ParquetFile(
    ...:     filesystem.open_input_file(where),
    ...: )

In [30]: parquet_metadata.schema
Out[30]: 
<pyarrow._parquet.ParquetSchema object at 0x7f60be97fe00>
required group field_id=-1 schema {
  optional int32 field_id=-1 A (Int(bitWidth=8, isSigned=true));
  optional binary field_id=-1 B (String);
}
{code}

So that means that this happens on the read side.

I suppose reading dictionary encoded values into dictionary type doesn't (yet?) support preserving the integer indices bitwidth, and using the default int32 that Arrow uses.



> Categorical int8 index types written as int32 in parquet files
> --------------------------------------------------------------
>
>                 Key: ARROW-14767
>                 URL: https://issues.apache.org/jira/browse/ARROW-14767
>             Project: Apache Arrow
>          Issue Type: Bug
>          Components: Python
>    Affects Versions: 5.0.0
>         Environment: NAME="CentOS Linux"
> VERSION="7 (Core)"
>            Reporter: Gavin
>            Priority: Minor
>
> When converting from a pandas dataframe to a table, categorical variables are by default given an index type int8 (presumably because there are fewer than 128 categories) in the schema. When this is written to a parquet file, the schema changes such that the index type is int32 instead. This causes an inconsistency between the schemas of tables derived from pandas and those read from disk.
> A minimal recreation of the issue is as follows:
> {code:java}
> import numpy as np
> import pandas as pd
> import pyarrow as pa
> import pyarrow.parquet as pq
> df = pd.DataFrame({"A": [1, 2, 3, 4, 5], "B": ["a", "a", "b", "c", "b"]})
> dtypes = {
>     "A": np.dtype("int8"),
>     "B": pd.CategoricalDtype(categories=["a", "b", "c"], ordered=None),
> }
> df = df.astype(dtypes)
> tbl = pa.Table.from_pandas(
>     df, 
> )  
> where = "tmp.parquet"
> filesystem = pa.fs.LocalFileSystem()
> pq.write_table(
>     tbl,
>     filesystem.open_output_stream(
>         where,
>         compression=None,
>     ),
>     version="2.0",
> )
> schema = tbl.schema
> read_schema = pq.ParquetFile(
>     filesystem.open_input_file(where),
> ).schema_arrow{code}
> By printing schema and read_schema, you can the inconsistency.
> I have workarounds in place for this, but am raising the issue anyway so that you can resolve it properly.



--
This message was sent by Atlassian Jira
(v8.20.1#820001)