You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@arrow.apache.org by "Python User (JIRA)" <ji...@apache.org> on 2018/08/25 00:30:00 UTC

[jira] [Commented] (ARROW-2659) [Python] More graceful reading of empty String columns in ParquetDataset

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

Python User commented on ARROW-2659:
------------------------------------

I have also run into this when trying to convert from a pandas dataframe in chunks.  Here is code and data to reproduce.

schema-bug.csv
{code:none}
a,b
1,x
2,y
3,
4,
{code}

schema_bug.py
{code:python}
#!/usr/bin/env python
import sys

import pandas as pd
import pyarrow.parquet

writer = None
for chunk in pd.read_csv(sys.stdin, dtype={'a': int, 'b': str}, chunksize=2):
    print('Pandas dtypes', list(chunk.dtypes.astype(str)), file=sys.stderr)
    table = pyarrow.Table.from_pandas(chunk)
    writer = writer or pyarrow.parquet.ParquetWriter(sys.stdout.buffer, table.schema)
    writer.write_table(table)

writer.close()
{code}

Here are the results:

{code:none}
$ ./schema_bug.py < schema-bug.csv > /dev/null
Pandas dtypes ['int64', 'object']
Pandas dtypes ['int64', 'object']
Traceback (most recent call last):
  File "schema_bug.py", line 13, in <module>
    writer.write_table(table)
  File "python/lib/python3.6/site-packages/pyarrow/parquet.py", line 335, in write_table
    raise ValueError(msg)
ValueError: Table schema does not match schema used to create file:
table:
a: int64
b: null
__index_level_0__: int64
metadata
--------
{b'pandas': b'{"index_columns": ["__index_level_0__"], "column_indexes": [{"na'
            b'me": null, "field_name": null, "pandas_type": "unicode", "numpy_'
            b'type": "object", "metadata": {"encoding": "UTF-8"}}], "columns":'
            b' [{"name": "a", "field_name": "a", "pandas_type": "int64", "nump'
            b'y_type": "int64", "metadata": null}, {"name": "b", "field_name":'
            b' "b", "pandas_type": "empty", "numpy_type": "object", "metadata"'
            b': null}, {"name": null, "field_name": "__index_level_0__", "pand'
            b'as_type": "int64", "numpy_type": "int64", "metadata": null}], "p'
            b'andas_version": "0.23.4"}'} vs.
file:
a: int64
b: string
__index_level_0__: int64
metadata
--------
{b'pandas': b'{"index_columns": ["__index_level_0__"], "column_indexes": [{"na'
            b'me": null, "field_name": null, "pandas_type": "unicode", "numpy_'
            b'type": "object", "metadata": {"encoding": "UTF-8"}}], "columns":'
            b' [{"name": "a", "field_name": "a", "pandas_type": "int64", "nump'
            b'y_type": "int64", "metadata": null}, {"name": "b", "field_name":'
            b' "b", "pandas_type": "unicode", "numpy_type": "object", "metadat'
            b'a": null}, {"name": null, "field_name": "__index_level_0__", "pa'
            b'ndas_type": "int64", "numpy_type": "int64", "metadata": null}], '
            b'"pandas_version": "0.23.4"}'}
{code}

(The difference is that the first chunk (shown second) has {{pandas_type: unicode}}, where the second chunk has {{pandas_type: empty}}.)

Even if you save the initial schema and try to use it writing subsequent tables, you get this:

{code:none}
pyarrow.lib.ArrowNotImplementedError: ('No cast implemented from null to null', 'Conversion failed for column b with type object')
{code}

> [Python] More graceful reading of empty String columns in ParquetDataset
> ------------------------------------------------------------------------
>
>                 Key: ARROW-2659
>                 URL: https://issues.apache.org/jira/browse/ARROW-2659
>             Project: Apache Arrow
>          Issue Type: Bug
>          Components: Python
>    Affects Versions: 0.9.0
>            Reporter: Uwe L. Korn
>            Priority: Major
>              Labels: beginner
>             Fix For: 0.11.0
>
>         Attachments: read_parquet_dataset.error.read_table.novalidation.txt, read_parquet_dataset.error.read_table.txt
>
>
> When currently saving a {{ParquetDataset}} from Pandas, we don't get consistent schemas, even if the source was a single DataFrame. This is due to the fact that in some partitions object columns like string can become empty. Then the resulting Arrow schema will differ. In the central metadata, we will store this column as {{pa.string}} whereas in the partition file with the empty columns, this columns will be stored as {{pa.null}}.
> The two schemas are still a valid match in terms of schema evolution and we should respect that in https://github.com/apache/arrow/blob/79a22074e0b059a24c5cd45713f8d085e24f826a/python/pyarrow/parquet.py#L754 Instead of doing a {{pa.Schema.equals}} in https://github.com/apache/arrow/blob/79a22074e0b059a24c5cd45713f8d085e24f826a/python/pyarrow/parquet.py#L778 we should introduce a new method {{pa.Schema.can_evolve_to}} that is more graceful and returns {{True}} if a dataset piece has a null column where the main metadata states a nullable column of any type.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)