You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@arrow.apache.org by "Aldrin (JIRA)" <ji...@apache.org> on 2018/06/01 16:02: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=16498163#comment-16498163 ] 

Aldrin commented on ARROW-2659:
-------------------------------

In trying to read from a parquet dataset using validate_schema=False, I then get the following exception (see read_parquet_dataset.error.read_table.novalidation.txt):
{noformat}
Traceback (most recent call last):
  File "toolbox/read_parquet.py", line 85, in <module>
    dsv_data_table = read_parquet_data(parsed_args.input_path)
  File "toolbox/read_parquet.py", line 64, in read_parquet_data
    use_pandas_metadata=use_pandas_metadata
  File "/Users/amontana/.local/share/virtualenvs/kitsi-zdrw075I/lib/python3.6/site-packages/pyarrow/parquet.py", line 806, in read
    all_data = lib.concat_tables(tables)
  File "table.pxi", line 1285, in pyarrow.lib.concat_tables
  File "error.pxi", line 77, in pyarrow.lib.check_status

pyarrow.lib.ArrowInvalid: Schema at index 3 was different:{noformat}
 

Given this error, it seems that validating the schema is just one piece of the overall difficulty, and some way of pushing down the valid schema evolution into concat_tables in table.pxi will also be necessary.

[~xhochy], I'm not sure if this impression is correct, and if it is, whether we would want a related Jira to track changes in (or underlying) the concat_tables function.

> [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)