[
https://issues.apache.org/jira/browse/ARROW-12762?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17347376#comment-17347376
]
Joris Van den Bossche edited comment on ARROW-12762 at 5/19/21, 7:48 AM:
-------------------------------------------------------------------------
[~jjgalvez] thanks, that helps.
Looking at both schemas, there is one difference: the name that is used for the
field inside the list column ("element" vs "item").
By default, Arrow uses "item", but you can manually specify a different name as
well when constructing a list type:
{code}
>>> pa.list_(pa.int64())
ListType(list<item: int64>)
>>> pa.list_(pa.field("element", pa.int64()))
ListType(list<element: int64>)
{code}
When writing Parquet files, Arrow also uses this "item", while the official
Parquet format says this should be "element". There was recently a PR to add an
option to write a compliant parquet file:
https://github.com/apache/arrow/pull/9489
Now, so that is the reason I couldn't replicate it. If I explicitly write a
compliant Parquet file (as you probably had with pyspark as well), I indeed can
reproduce it.
So the underlying reason is that pickling/unpickling doesn't preserve the list
field name. A smaller reproducer without parquet:
{code}
>>> list_type = pa.list_(pa.field("element", pa.int64()))
>>> list_type
ListType(list<element: int64>)
>>> pickle.loads(pickle.dumps(list_type))
ListType(list<item: int64>)
{code}
was (Author: jorisvandenbossche):
[~jjgalvez] thanks, that helps.
Looking at both schemas, there is one difference: the name that is used for the
field inside the list column ("element" vs "item").
By default, Arrow uses "item", but you can manually specify a different name as
well when constructing a list type:
{code}
>>> pa.list_(pa.int64())
ListType(list<item: int64>)
>>> pa.list_(pa.field("element", pa.int64()))
ListType(list<element: int64>)
{code}
When writing Parquet files, Arrow also uses this "item", while the official
Parquet format says this should be "element". There was recently a PR to add an
option to write a compliant parquet file:
https://github.com/apache/arrow/pull/9489
Now, so that is the reason I couldn't replicate it. If I explicitly write a
compliant Parquet file (as you probably had with pyspark as well), I indeed can
reproduce it.
So the underlying reason is that pickling/unpickling doesn't preserve the list
field name. A smaller reproducer without parquet:
{code}
>>> schema = pa.schema([("col", pa.list_(pa.field("element", pa.int64()))), ])
>>> schema
col: list<element: int64>
child 0, element: int64
col: list<item: int64>
child 0, item: int64
{code}
> [Python] pyarrow.lib.Schema equality fails after pickle and unpickle
> --------------------------------------------------------------------
>
> Key: ARROW-12762
> URL: https://issues.apache.org/jira/browse/ARROW-12762
> Project: Apache Arrow
> Issue Type: Bug
> Affects Versions: 4.0.0
> Reporter: Juan Galvez
> Priority: Major
>
> Here is a small reproducer:
> {code:python}
> import pandas as pd
> from pyspark.sql import SparkSession
> import pyarrow.parquet as pq
> import pickle
> df = pd.DataFrame(
> {
> "A": [
> ["aa", "bb "],
> ["c"],
> ["d", "ee", "", "f"],
> ["ggg", "H"],
> [""],
> ]
> }
> )
> spark = SparkSession.builder.appName("GenSparkData").getOrCreate()
> spark_df = spark.createDataFrame(df)
> spark_df.write.parquet("list_str.pq", "overwrite")
> ds = pq.ParquetDataset("list_str.pq")
> assert pickle.loads(pickle.dumps(ds.schema)) == ds.schema # PASSES
> assert pickle.loads(pickle.dumps(ds.schema.to_arrow_schema())) ==
> ds.schema.to_arrow_schema() # FAILS
> {code}
--
This message was sent by Atlassian Jira
(v8.3.4#803005)