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

Alessandro Molina edited comment on ARROW-13342 at 7/16/21, 1:51 PM:
---------------------------------------------------------------------

FYI, this seems to happen for numeric types. The same is true for {{int}} and 
{{double}}

{code}
import pyarrow as pa
import pyarrow.parquet

# Convert to arrow Table and save to disk
table = pa.Table.from_arrays([
    pa.DictionaryArray.from_arrays(indices=pa.array([0, 1, 1, 0, 1, 1, 0], 
type=pa.int8()), 
                                   dictionary=[10000000000000000, 
20000000000000000])
], names=["data"])
print(table)
pa.parquet.write_table(table, 'test.parquet')

# Reload data and convert back to pandas
table_rel = pa.parquet.read_table('test.parquet')
print(table_rel)
{code}

The exception seem to be {{unicode}} which preserves the dictionary form.  

In general, from my understanding, it seems arrow currently writes in 
dictionary form only the binary types ( 
https://github.com/apache/arrow/blob/14b75ee71d770ba86999e0e7a0e0b94629b91968/cpp/src/parquet/column_writer.cc#L1008
 )

This in general seems a fairly reasonable behaviour to me, because in general 
binary data/text is where you get most saving due to the delta in size between 
the data and the indices.

On the other side, I don't think that "preserving same exact type" that was 
provided to a writer is an expectation that can be always satisfied or that it 
makes sense to enforce. In some cases enforcing dictionary encoding because the 
input was dictionary encoded might lead to bigger parquet files, or for some 
types for example that might not even be possible, think of CSV or JSON, not 
all types can be represented in those formats and thus the data you read back 
might have a different type



was (Author: amol-):
FYI, this seems to happen for numeric types. The same is true for {{int}} and 
{{double}}

{code}
import pyarrow as pa
import pyarrow.parquet

# Convert to arrow Table and save to disk
table = pa.Table.from_arrays([
    pa.DictionaryArray.from_arrays(indices=pa.array([0, 1, 1, 0, 1, 1, 0], 
type=pa.int8()), 
                                   dictionary=[10000000000000000, 
20000000000000000])
], names=["data"])
print(table)
pa.parquet.write_table(table, 'test.parquet')

# Reload data and convert back to pandas
table_rel = pa.parquet.read_table('test.parquet')
print(table_rel)
{code}

The exception seem to be {{unicode}} which preserves the dictionary form.  

In general, from my understanding, it seems arrow currently writes in 
dictionary form only the binary types ( 
https://github.com/apache/arrow/blob/14b75ee71d770ba86999e0e7a0e0b94629b91968/cpp/src/parquet/column_writer.cc#L1008
 )

This in general seems a fairly reasonable behaviour to me, because in general 
binary data/text is where you get most saving due to the delta in size between 
the data and the indices.


> [Python] Categorical boolean column saved as regular boolean in parquet
> -----------------------------------------------------------------------
>
>                 Key: ARROW-13342
>                 URL: https://issues.apache.org/jira/browse/ARROW-13342
>             Project: Apache Arrow
>          Issue Type: Bug
>          Components: Parquet, Python
>    Affects Versions: 4.0.1
>            Reporter: Joao Moreira
>            Priority: Major
>
> When saving a pandas dataframe to parquet, if there is a categorical column 
> where the categories are boolean, the column is saved as regular boolean.
> This causes an issue because, when reading back the parquet file, I expect 
> the column to still be categorical.
>  
> Reproducible example:
> {code:python}
> import pandas as pd
> import pyarrow
> # Create dataframe with boolean column that is then converted to categorical
> df = pd.DataFrame({'a': [True, True, False, True, False]})
> df['a'] = df['a'].astype('category')
> # Convert to arrow Table and save to disk
> table = pyarrow.Table.from_pandas(df)
> pyarrow.parquet.write_table(table, 'test.parquet')
> # Reload data and convert back to pandas
> table_rel = pyarrow.parquet.read_table('test.parquet')
> df_rel = table_rel.to_pandas()
> {code}
> The arrow {{table}} variable correctly converts the column to an arrow 
> {{DICTIONARY}} type:
> {noformat}
> >>> df['a']
> 0     True
> 1     True
> 2    False
> 3     True
> 4    False
> Name: a, dtype: category
> Categories (2, object): [False, True]
> >>>
> >>> table
> pyarrow.Table
> a: dictionary<values=bool, indices=int8, ordered=0>
> {noformat}
> However, the reloaded column is now a regular boolean:
> {noformat}
> >>> table_rel
> pyarrow.Table
> a: bool
> >>>
> >>> df_rel['a']
> 0     True
> 1     True
> 2    False
> 3     True
> 4    False
> Name: a, dtype: bool
> {noformat}
> I would have expected the column to be read back as categorical.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to