The int96 deprecation is slowly bubbling up the stack. There are still 
discussions in spark on how to make the change. So for now even though it's 
deprecated it is still used in some places. This should get resolved in the 
near future. 

Julien

> On Sep 8, 2017, at 14:12, Wes McKinney <wesmck...@gmail.com> wrote:
> 
> Turning on int96 timestamps is the solution right now. To save
> yourself some typing, you could declare
> 
> parquet_options = {
>    'compression': ...,
>    'use_deprecated_int96_timestamps': True
> }
> 
> pq.write_table(..., **parquet_options)
> 
>> On Fri, Sep 8, 2017 at 5:08 PM, Brian Wylie <briford.wy...@gmail.com> wrote:
>> So, this is certainly good for future versions of Arrow. Do you have any
>> specific recommendations for a workaround currently?
>> 
>> Saving a parquet file with datetimes will obviously be a common use case
>> and if I'm understanding it correctly, right now saving a Parquet file with
>> PyArrow that file will not be readable by Spark at this point. Yes?  (I'm
>> asking this as opposed to stating this).
>> 
>> -Brian
>> 
>>> On Fri, Sep 8, 2017 at 2:58 PM, Wes McKinney <wesmck...@gmail.com> wrote:
>>> 
>>> Indeed, INT96 is deprecated in the Parquet format. There are other
>>> issues with Spark (it places restrictions on table field names, for
>>> example), so it may be worth adding an option like
>>> 
>>> pq.write_table(table, where, flavor='spark')
>>> 
>>> or maybe better
>>> 
>>> pq.write_table(table, where, flavor='spark-2.2')
>>> 
>>> and this would set the correct options for that version of Spark.
>>> 
>>> I created https://issues.apache.org/jira/browse/ARROW-1499 as a place
>>> to discuss further
>>> 
>>> - Wes
>>> 
>>> 
>>> On Fri, Sep 8, 2017 at 4:28 PM, Brian Wylie <briford.wy...@gmail.com>
>>> wrote:
>>>> Okay,
>>>> 
>>>> So after some additional debugging, I can get around this if I set
>>>> 
>>>> use_deprecated_int96_timestamps=True
>>>> 
>>>> on the pq.write_table(arrow_table, filename, compression=compression,
>>>> use_deprecated_int96_timestamps=True) call.
>>>> 
>>>> But that just feels SO wrong....as I'm sure it's deprecated for a reason
>>>> (i.e. this will bite me later and badly)
>>>> 
>>>> 
>>>> I also see this issue (or at least a related issue) reference in this
>>> Jeff
>>>> Knupp blog...
>>>> 
>>>> https://www.enigma.com/blog/moving-to-parquet-files-as-a-
>>> system-of-record
>>>> 
>>>> So shrug... any suggestions are greatly appreciated :)
>>>> 
>>>> -Brian
>>>> 
>>>> On Fri, Sep 8, 2017 at 12:36 PM, Brian Wylie <briford.wy...@gmail.com>
>>>> wrote:
>>>> 
>>>>> Apologies if this isn't quite the right place to ask this question, but
>>> I
>>>>> figured Wes/others might know right off the bat :)
>>>>> 
>>>>> 
>>>>> Context:
>>>>> - Mac OSX Laptop
>>>>> - PySpark: 2.2.0
>>>>> - PyArrow: 0.6.0
>>>>> - Pandas: 0.19.2
>>>>> 
>>>>> Issue Explanation:
>>>>> - I'm converting my Pandas dataframe to a Parquet file with code very
>>>>> similar to
>>>>>       - http://wesmckinney.com/blog/python-parquet-update/
>>>>> - My Pandas DataFrame has a datetime index:  http_df.index.dtype =
>>>>> dtype('<M8[ns]')
>>>>> - When loading the saved parquet file I get the error below
>>>>> - If I remove that index everything works fine
>>>>> 
>>>>> ERROR:
>>>>> - Py4JJavaError: An error occurred while calling o34.parquet.
>>>>> : org.apache.spark.SparkException: Job aborted due to stage failure:
>>>>> Task 0 in stage 0.0 failed 1 times, most recent failure: Lost task 0.0
>>>>> in stage 0.0 (TID 0, localhost, executor driver):
>>>>> org.apache.spark.sql.AnalysisException: Illegal Parquet type: INT64
>>>>> (TIMESTAMP_MICROS);
>>>>> 
>>>>> Full Code to reproduce:
>>>>> - https://github.com/Kitware/bat/blob/master/notebooks/Bro_
>>>>> to_Parquet.ipynb
>>>>> 
>>>>> 
>>>>> Thanks in advance, also big fan of all this stuff... "be the chicken" :)
>>>>> 
>>>>> -Brian
>>>>> 
>>>>> 
>>>>> 
>>>>> 
>>> 

Reply via email to