Skip to content

ARROW-1639: [Python] Serialize RangeIndex as metadata via Table.from_pandas instead of converting to a column of integers #3868

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Closed
wants to merge 4 commits into from

Conversation

wesm
Copy link
Member

@wesm wesm commented Mar 11, 2019

This ended up being much more difficult than anticipated due to the spaghetti-like state (as the result of many hacks) of pyarrow/pandas_compat.py.

This is partly a performance and memory use optimization. It has consequences, though, namely tables will have some index data discarded when concatenated from multiple pandas DataFrame objects that were converted to Arrow. I think this is OK, though, since the preservation of pandas indexes is generally something that's handled at the granularity of a single DataFrame. One always has the option of calling reset_index to convert a RangeIndex if that's what is desired.

This patch also implements proposed extensions to the serialized pandas metadata to accommodate indexes-as-columns vs. indexes-represented-as-metadata, as described in

pandas-dev/pandas#25672

'start': level._start,
'stop': level._stop,
'step': level._step
}
Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@jreback @jorisvandenbossche is there a better way to get the start/stop/step for RangeIndex?

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

As far as I know, we don't expose those publicly, so the private ones you are using here is the best pandas has to offer.
They also have been stable over the years, so I don't think it is a problem to use them. I don't really recall the history of not exposing them (maybe because we don't really want end-users to rely too much on the fact it is special, but just see it as a memory optimized integer index).

@wesm
Copy link
Member Author

wesm commented Mar 11, 2019

Before:

size = 100_000_000
df = pd.DataFrame({'a': np.random.randn(size)})

>>> %timeit serialized = pa.serialize_pandas(df)
720 ms ± 12.4 ms per loop (mean ± std. dev. of 7 runs, 1 loop each)

>>> %timeit df_result = pa.deserialize_pandas(serialized)
417 ms ± 2.66 ms per loop (mean ± std. dev. of 7 runs, 1 loop each)

after

>>> %timeit serialized = pa.serialize_pandas(df)
381 ms ± 6.93 ms per loop (mean ± std. dev. of 7 runs, 1 loop each)

>>> %timeit df_result = pa.deserialize_pandas(serialized)
282 ms ± 6.42 ms per loop (mean ± std. dev. of 7 runs, 1 loop each)

cc @TomAugspurger @mrocklin in case of interest

@wesm
Copy link
Member Author

wesm commented Mar 12, 2019

I'll add this to the ASV suite so we can track it. @xhochy I will need you to review this as there was some changes required to the Parquet test suite

Copy link
Member

@xhochy xhochy left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I fear that this will break backwards compability. We should first introduce some tests where we have hardcoded the pandas schema JSON and then ensure that we still can read old versions. We had some problems in the past where pyarrow refused to read old Parquet files due to breaking changes in the pandas metadata. There have been none recently as we haven't touched this code for a while.

@@ -493,7 +493,7 @@ def test_recordbatchlist_to_pandas():

table = pa.Table.from_batches([batch1, batch2])
result = table.to_pandas()
data = pd.concat([data1, data2])
data = pd.concat([data1, data2]).reset_index(drop=True)
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Why is this now necessary and why did it work before?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It worked before because the pandas indexes had been converted to data columns. This change makes the RangeIndex metadata instead

types : List[pyarrow.DataType]

Returns
-------
dict
"""
num_serialized_index_levels = len([descr for descr in index_descriptors
if descr['kind'] == 'serialized'])
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

'kind' is a new attribute that is not set in old metadata descriptions. Is this somewhere handled?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I need to make a PR to resolve the pandas issue adding this. I have implemented backwards compatibility already for this, but I will add some unit tests to assert

@wesm
Copy link
Member Author

wesm commented Mar 12, 2019

@xhochy I'll add some unit tests with hard-coded "old" metadata to ensure that old files can still be read correctly.

@wesm
Copy link
Member Author

wesm commented Mar 12, 2019

By the way, the kind of compatibility we are interested in is forward compatibility:

  • Forward compatibility: new readers can read old files
  • Backward compatibility: old readers can read new files

I'll get this sorted out today

@wesm
Copy link
Member Author

wesm commented Mar 12, 2019

I added the forward compatibility tests. I also added a version number to the metadata

In [3]: size = 10                                                                                                                       

In [4]: df = pd.DataFrame({'a': np.random.randn(size)})                                                                                 

In [8]: json.loads(pa.Table.from_pandas(df).schema.metadata[b'pandas'])                                                                 
Out[8]: 
{'index_columns': [{'kind': 'range',
   'name': None,
   'start': 0,
   'stop': 10,
   'step': 1}],
 'column_indexes': [{'name': None,
   'field_name': None,
   'pandas_type': 'unicode',
   'numpy_type': 'object',
   'metadata': {'encoding': 'UTF-8'}}],
 'columns': [{'name': 'a',
   'field_name': 'a',
   'pandas_type': 'float64',
   'numpy_type': 'float64',
   'metadata': None}],
 'creator': {'library': 'pyarrow',
  'version': '0.12.1.dev381+g0ca1bfc58.d20190312'},
 'pandas_version': '0.23.4'}

@wesm
Copy link
Member Author

wesm commented Mar 13, 2019

Appveyor build: https://ci.appveyor.com/project/wesm/arrow/builds/23025530

It would be good to merge this soon if it looks OK. I am working on ARROW-4637 which conflicts with these changes because of refactoring I did in pyarrow.pandas_compat

@wesm
Copy link
Member Author

wesm commented Mar 13, 2019

I'll just rebase my wip patch on top of this branch for now so I'm not blocked

@wesm
Copy link
Member Author

wesm commented Mar 13, 2019

I'm working on a PR into pandas also, in case we need to make alterations to the metadata

Copy link
Member

@xhochy xhochy left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

+1, LGTM

Maybe @jorisvandenbossche or @TomAugspurger also want to take a look?

Copy link
Contributor

@TomAugspurger TomAugspurger left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM on a quick glance. Opened pandas-dev/pandas#25710 for the RangeIndex attribute issue, but as Joris said, I don't expect the private versions to change.

@wesm wesm closed this in 86f480a Mar 13, 2019
@wesm wesm deleted the ARROW-1639 branch March 13, 2019 17:17
@wesm
Copy link
Member Author

wesm commented Mar 13, 2019

thanks everyone

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

4 participants