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

Joris Van den Bossche edited comment on ARROW-12680 at 5/17/21, 9:05 AM:
-------------------------------------------------------------------------

I think there is indeed a bug here.  Let me try and demystify some of what is 
going on.  There are 5+ temporal types in pyarrow but everything you are doing 
is currently related to just one, the timestamp type.  The timestamp type 
represents seconds, milliseconds, microseconds, or nanoseconds from the epoch.  
In addition there may or may not be a time zone string.  Finally, these types 
may or may not be in a struct (which shouldn't matter but does here...which is 
the bug).

In pandas there are 3+ ways to represent temporal information.  The 
datetime.datetime object,  A pandas.Timestamp, and an integer.

 

When you first read in your table you are getting a struct where the 'date' 
field is a timestamp with **nanosecond** resolution.

When you save your table and then reload it the timestamp is being truncated.  
This is because pq.write_table with version==1.0 (the default in pyarrow 3) 
will truncate nanosecond timestamps down to microseconds.

So when you next read in your table you are getting a struct where the 'date' 
field is a timestamp with **microsecond** resolution.

Finally, It seems this may be a regression of 
https://issues.apache.org/jira/browse/ARROW-7723

 
{code:java}
import pyarrow as pa
import datetime
pylist = [datetime.datetime.now()]
arr1 = pa.array(pylist, pa.timestamp(unit='ms'))
arr2 = pa.array(pylist, pa.timestamp(unit='ns'))
sarr = pa.StructArray.from_arrays([arr1, arr2], names=['ms', 'ns'])
table = pa.Table.from_arrays([arr1, arr2, sarr], ['ms', 'ns', 'struct'])
print(table.to_pandas())
{code}
 
{code:java}
                       ms                         ns                            
                 struct
0 2021-05-07 08:46:15.898 2021-05-07 08:46:15.898716  {'ms': 2021-05-07 
08:46:15.898000, 'ns': 16203...

{code}
 

As for workarounds...if your schema is reliable you could cast from nanosecond 
resolution to us resolution (struct casting isn't working quite right 
(ARROW-1888) so it's a bit clunky):

 
{code:java}
import pyarrow as pa
import pyarrow.compute as pc

dates = pa.array([datetime.datetime.now()], pa.timestamp(unit='ns'))
values = pa.array([200.37], pa.float64())
observations = pa.StructArray.from_arrays([dates, values], names=['dates', 
'values'])
desired_type = pa.struct([pa.field('dates', pa.timestamp(unit='us')), 
pa.field('values', pa.float64())])
tbl = pa.Table.from_arrays([observations], ['observations'])
print(tbl.to_pandas())

bad_observations = tbl.column('observations').chunks
values = [chunk.field('values') for chunk in bad_observations]
bad_dates = [chunk.field('dates') for chunk in bad_observations]
good_dates = [pc.cast(bad_dates_chunk, pa.timestamp(unit='us')) for 
bad_dates_chunk in bad_dates]
good_observations_chunks = []
for i in range(len(good_dates)):
    good_observations_chunks.append(pa.StructArray.from_arrays([good_dates[i], 
values[i]], names=['dates', 'values']))
good_observations = pa.chunked_array(good_observations_chunks)
tbl = tbl.set_column(0, 'observations', good_observations)
print(tbl.to_pandas())
{code}
 


was (Author: westonpace):
I think there is indeed a bug here.  Let me try and demystify some of what is 
going on.  There are 5+ temporal types in pyarrow but everything you are doing 
is currently related to just one, the timestamp type.  The timestamp type 
represents seconds, milliseconds, microseconds, or nanoseconds from the epoch.  
In addition there may or may not be a time zone string.  Finally, these types 
may or may not be in a struct (which shouldn't matter but does here...which is 
the bug).

In pandas there are 3+ ways to represent temporal information.  The 
datetime.datetime object,  A pandas.Timestamp, and an integer.

 

When you first read in your table you are getting a struct where the 'date' 
field is a timestamp with **nanosecond** resolution.

When you save your table and then reload it the timestamp is being truncated.  
This is because pq.write_table with version==1.0 (the default in pyarrow 3) 
will truncate nanosecond timestamps down to microseconds.

So when you next read in your table you are getting a struct where the 'date' 
field is a timestamp with **microsecond** resolution.

Finally, It seems this may be a regression of 
https://issues.apache.org/jira/browse/ARROW-7723

 
{code:java}
import pyarrow as pa
import datetimepylist = [datetime.datetime.now()]
arr1 = pa.array(pylist, pa.timestamp(unit='ms'))
arr2 = pa.array(pylist, pa.timestamp(unit='ns'))
sarr = pa.StructArray.from_arrays([arr1, arr2], names=['ms', 'ns'])
table = pa.Table.from_arrays([arr1, arr2, sarr], ['ms', 'ns', 'struct'])
print(table.to_pandas())
{code}
 
{code:java}
                       ms                         ns                            
                 struct
0 2021-05-07 08:46:15.898 2021-05-07 08:46:15.898716  {'ms': 2021-05-07 
08:46:15.898000, 'ns': 16203...

{code}
 

As for workarounds...if your schema is reliable you could cast from nanosecond 
resolution to us resolution (struct casting isn't working quite right 
(ARROW-1888) so it's a bit clunky):

 
{code:java}
import pyarrow as pa
import pyarrow.compute as pc

dates = pa.array([datetime.datetime.now()], pa.timestamp(unit='ns'))
values = pa.array([200.37], pa.float64())
observations = pa.StructArray.from_arrays([dates, values], names=['dates', 
'values'])
desired_type = pa.struct([pa.field('dates', pa.timestamp(unit='us')), 
pa.field('values', pa.float64())])
tbl = pa.Table.from_arrays([observations], ['observations'])
print(tbl.to_pandas())

bad_observations = tbl.column('observations').chunks
values = [chunk.field('values') for chunk in bad_observations]
bad_dates = [chunk.field('dates') for chunk in bad_observations]
good_dates = [pc.cast(bad_dates_chunk, pa.timestamp(unit='us')) for 
bad_dates_chunk in bad_dates]
good_observations_chunks = []
for i in range(len(good_dates)):
    good_observations_chunks.append(pa.StructArray.from_arrays([good_dates[i], 
values[i]], names=['dates', 'values']))
good_observations = pa.chunked_array(good_observations_chunks)
tbl = tbl.set_column(0, 'observations', good_observations)
print(tbl.to_pandas())
{code}
 

> [Python] StructScalar Timestamp using .to_pandas() loses/converts type
> ----------------------------------------------------------------------
>
>                 Key: ARROW-12680
>                 URL: https://issues.apache.org/jira/browse/ARROW-12680
>             Project: Apache Arrow
>          Issue Type: Bug
>          Components: Python
>    Affects Versions: 3.0.0
>            Reporter: Tim Ryles
>            Priority: Major
>
> Hi,
> We're noticing an issue where we lose type and formatting on conversion to a 
> pandas dataframe for a particular dataset we house, which contains a struct, 
> and the underlying type of the child is Timestamp rather than 
> datetime.datetime (which we believed synonymous from Pandas documentation).
>  
> Inside the StructArray we can see nicely formatted timestamp values, but when 
> we call .to_pandas() on it, we end up with epoch stamps for the date child.
> {code:java}
> import pyarrow.parquet as pq
> tbl=pq.read_table("part-00009-47f62157-cb6f-41a8-9ad6-ace65df94c6e-c000.snappy.parquet")
> tbl.column("observations").chunk(0).values pyarrow.lib.StructArray object at 
> 0x7fc8eb0cab40>
> – is_valid: all not null
> – child 0 type: timestamp[ns]
> [
> 2000-01-01 00:00:00.000000000,
> 2001-01-01 00:00:00.000000000,
> 2002-01-01 00:00:00.000000000,
> 2003-01-01 00:00:00.000000000,
> 2004-01-01 00:00:00.000000000,
> 2005-01-01 00:00:00.000000000,
> 2006-01-01 00:00:00.000000000,
> 2007-01-01 00:00:00.000000000,
> 2008-01-01 00:00:00.000000000,
> 2009-01-01 00:00:00.000000000,
> ...
> 2018-07-01 00:00:00.000000000,
> 2018-10-01 00:00:00.000000000,
> 2019-01-01 00:00:00.000000000,
> 2019-04-01 00:00:00.000000000,
> 2019-07-01 00:00:00.000000000,
> 2019-10-01 00:00:00.000000000,
> 2020-01-01 00:00:00.000000000,
> 2020-04-01 00:00:00.000000000,
> 2020-07-01 00:00:00.000000000,
> 2020-10-01 00:00:00.000000000
> ]
> – child 1 type: double
> [
> -2.69685,
> 9.27988,
> 7.26902,
> -7.55753,
> -1.62137,
> 6.84773,
> -8.21204,
> -8.97041,
> -1.14405,
> -0.710153,
> ...
> 2.1658,
> 3.05588,
> 2.3868,
> 2.10805,
> 2.39984,
> 2.54855,
> -7.26804,
> -2.35179,
> -0.867518,
> 0.150593
> ]
> {code}
> {code:java}
>  
> tbl.to_pandas()['observations'] 
> [{'date': 946684800000000000, 'value': -2.6968... 1 [{'date': 
> 946684800000000000, 'value': 57.9608... 2 [{'date': 1483228800000000000, 
> 'value': 95.904... 3 [{'date': 1214870400000000000, 'value': 19.021... 4 
> [{'date': 1199145600000000000, 'value': 1.2011... ... 636 [\{'date': 
> 1072915200000000000, 'value': 5.418}... 637 [{'date': 946684800000000000, 
> 'value': 110.695... 638 [{'date': 1009843200000000000, 'value': 3.0094... 639 
> [{'date': 1222819200000000000, 'value': 48.365... 640 [{'date': 
> 1199145600000000000, 'value': 1.5600... Name: observations, Length: 641, 
> dtype: object
> In [12]: tbl.to_pandas()["observations"].iloc[0][0]
> Out[12]: {'date': 1041379200000000000, 'value': 249.523242}
> # date is now type Int{code}
>  
> We notice that if we take the same table, save it back out to a file first, 
> and then check the chunk(0).values as above, the underlying type changes from 
> *Timestamp* to *datetime.datetime*, and that will now convert .to_pandas() 
> correctly.
> {code:java}
> pq.write_table(tbl, "output.parquet")
> tbl2=pq.read_table("output.parquet")
> tbl2.column("observations").chunk(0).values[0]
> Out[17]: <pyarrow.StructScalar: {'date': datetime.datetime(2003, 1, 1, 0, 0), 
> 'value': 249.523242}>
> tbl2.column("observations").chunk(0).to_pandas()
> Out[18]: 
> 0        [{'date': 2003-01-01 00:00:00, 'value': 249.52...
> 1        [{'date': 2008-01-01 00:00:00, 'value': 29.741...
> 2        [{'date': 2000-01-01 00:00:00, 'value': 2.3454...
> 3        [{'date': 2006-01-01 00:00:00, 'value': 1.2048...
> 4        [{'date': 2008-01-01 00:00:00, 'value': 196546...
>                                ...                        
> 29489    [{'date': 2010-01-01 00:00:00, 'value': 19.155...
> 29490    [{'date': 2012-04-30 00:00:00, 'value': 0.0}, ...
> 29491    [{'date': 2012-04-30 00:00:00, 'value': 0.0}, ...
> 29492    [{'date': 2012-04-30 00:00:00, 'value': 0.0}, ...
> 29493    [{'date': 2012-04-30 00:00:00, 'value': 10.0},...
> Length: 29494, dtype: object
> tbl2.to_pandas()["observations"].iloc[0][0]
> Out[8]: {'date': datetime.datetime(2003, 1, 1, 0, 0), 'value': 249.523242}
> # date remains as datetime.datetime{code}
>  
> Thanks in advance, and apologies if I have not followed Issue protocol on 
> this board.
> If there is a parameter that we just need to pass into .to_pandas for this to 
> take place (I can see there is date_as_object/timestamp_as_object, but these 
> have no effect), we would like to know.



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

Reply via email to