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

Sarah Bird edited comment on ARROW-5144 at 4/17/19 8:38 PM:
------------------------------------------------------------

[~kszucs] this is with variations on my dataset. I have attached one piece 
which is sufficient to reproduce the error. It is web crawl data. The dtypes 
are:
{code}
argument_0                      object
argument_1                      object
argument_2                      object
argument_3                      object
argument_4                      object
argument_5                      object
argument_6                      object
argument_7                      object
arguments                       object
arguments_len                    int64
call_stack                      object
crawl_id                         int32
document_url                    object
func_name                       object
in_iframe                         bool
operation                       object
script_col                       int64
script_line                      int64
script_loc_eval                 object
script_url                      object
symbol                          object
time_stamp         datetime64[ns, UTC]
top_level_url                   object
value_1000                      object
value_len                        int64
visit_id                         int64
dtype: object
{code}
My traceback is:
{code:java}
distributed.protocol.pickle - INFO - Failed to serialize (<function safe_head 
at 0x7f3d57f2c7b8>, (<function _read_pyarrow_parquet_piece at 0x7f3d57ef9268>, 
<dask.bytes.local.LocalFileSystem object at 0x7f3db58
ea4e0>, 
ParquetDatasetPiece('javascript_10percent_value_1000_only.parquet/part.0.parquet',
 row_group=None, partition_keys=[]), ['argument_0', 'argument_1', 'argument_2', 
'argument_3', 'argument_4', 'argument_5'
, 'argument_6', 'argument_7', 'arguments', 'arguments_len', 'call_stack', 
'crawl_id', 'document_url', 'func_name', 'in_iframe', 'operation', 
'script_col', 'script_line', 'script_loc_eval', 'script_url', 'symbol
', 'time_stamp', 'top_level_url', 'value_1000', 'value_len', 'visit_id'], [], 
False, None, []), 5). Exception: no default __reduce__ due to non-trivial 
__cinit__
distributed.protocol.core - CRITICAL - Failed to Serialize                      
                                       
Traceback (most recent call last):                                              
                                           
  File 
"/home/bird/miniconda3/envs/pyarrowtest/lib/python3.7/site-packages/distributed/protocol/core.py",
 line 54, in dumps
    for key, value in data.items()                                              
                                                               
  File 
"/home/bird/miniconda3/envs/pyarrowtest/lib/python3.7/site-packages/distributed/protocol/core.py",
 line 55, in <dictcomp>
    if type(value) is Serialize}                                                
                                                                                
                                                 
  File 
"/home/bird/miniconda3/envs/pyarrowtest/lib/python3.7/site-packages/distributed/protocol/serialize.py",
 line 164, in serialize
    raise TypeError(msg, str(x)[:10000])                                        
                                                                                
                                                  
TypeError: ('Could not serialize object of type tuple.', "(<function safe_head 
at 0x7f3d57f2c7b8>, (<function _read_pyarrow_parquet_piece at 0x7f3d57ef9268>, 
<dask.bytes.local.LocalFileSystem object at 0x7f3db5
8ea4e0>, 
ParquetDatasetPiece('javascript_10percent_value_1000_only.parquet/part.0.parquet',
 row_group=None, partition_keys=[]), ['argument_0', 'argument_1', 'argument_2', 
'argument_3', 'argument_4', 'argument_5
', 'argument_6', 'argument_7', 'arguments', 'arguments_len', 'call_stack', 
'crawl_id', 'document_url', 'func_name', 'in_iframe', 'operation', 
'script_col', 'script_line', 'script_loc_eval', 'script_url', 'symbo
l', 'time_stamp', 'top_level_url', 'value_1000', 'value_len', 'visit_id'], [], 
False, None, []), 5)")                      
distributed.comm.utils - INFO - Unserializable Message: [{'op': 'update-graph', 
'tasks': {"('head-1-5-read-parquet-daaccee11e9cff29ad1ee5622ffd6c69', 0)": 
<Serialize: ('read-parquet-head-1-5-read-parquet-daacce
e11e9cff29ad1ee5622ffd6c69', 0)>, 
"('read-parquet-head-1-5-read-parquet-daaccee11e9cff29ad1ee5622ffd6c69', 0)": 
<Serialize: (<function safe_head at 0x7f3d57f2c7b8>, (<function 
_read_pyarrow_parquet_piece at 0x7
f3d57ef9268>, <dask.bytes.local.LocalFileSystem object at 0x7f3db58ea4e0>, 
ParquetDatasetPiece('javascript_10percent_value_1000_only.parquet/part.0.parquet',
 row_group=None, partition_keys=[]), ['argument_0', '
argument_1', 'argument_2', 'argument_3', 'argument_4', 'argument_5', 
'argument_6', 'argument_7', 'arguments', 'arguments_len', 'call_stack', 
'crawl_id', 'document_url', 'func_name', 'in_iframe', 'operation', 's
cript_col', 'script_line', 'script_loc_eval', 'script_url', 'symbol', 
'time_stamp', 'top_level_url', 'value_1000', 'value_len', 'visit_id'], [], 
False, None, []), 5)>}, 'dependencies': {"('head-1-5-read-parquet
-daaccee11e9cff29ad1ee5622ffd6c69', 0)": 
["('read-parquet-head-1-5-read-parquet-daaccee11e9cff29ad1ee5622ffd6c69', 0)"], 
"('read-parquet-head-1-5-read-parquet-daaccee11e9cff29ad1ee5622ffd6c69', 0)": 
[]}, 'keys'
: ["('head-1-5-read-parquet-daaccee11e9cff29ad1ee5622ffd6c69', 0)"], 
'restrictions': {}, 'loose_restrictions': None, 'priority': 
{"('read-parquet-head-1-5-read-parquet-daaccee11e9cff29ad1ee5622ffd6c69', 0)": 
0,
 "('head-1-5-read-parquet-daaccee11e9cff29ad1ee5622ffd6c69', 0)": 1}, 
'user_priority': 0, 'resources': None, 'submitting_task': None, 'retries': 
None, 'fifo_timeout': '60s', 'actors': None}]                    
distributed.comm.utils - ERROR - ('Could not serialize object of type tuple.', 
"(<function safe_head at 0x7f3d57f2c7b8>, (<function 
_read_pyarrow_parquet_piece at 0x7f3d57ef9268>, <dask.bytes.local.LocalFileSys
tem object at 0x7f3db58ea4e0>, 
ParquetDatasetPiece('javascript_10percent_value_1000_only.parquet/part.0.parquet',
 row_group=None, partition_keys=[]), ['argument_0', 'argument_1', 'argument_2', 
'argument_3', 'ar
gument_4', 'argument_5', 'argument_6', 'argument_7', 'arguments', 
'arguments_len', 'call_stack', 'crawl_id', 'document_url', 'func_name', 
'in_iframe', 'operation', 'script_col', 'script_line', 'script_loc_eval'
, 'script_url', 'symbol', 'time_stamp', 'top_level_url', 'value_1000', 
'value_len', 'visit_id'], [], False, None, []), 5)")
Traceback (most recent call last):
  File 
"/home/bird/miniconda3/envs/pyarrowtest/lib/python3.7/site-packages/distributed/batched.py",
 line 94, in _background_send
    on_error='raise')
  File 
"/home/bird/miniconda3/envs/pyarrowtest/lib/python3.7/site-packages/tornado/gen.py",
 line 729, in run
    value = future.result()
  File 
"/home/bird/miniconda3/envs/pyarrowtest/lib/python3.7/site-packages/tornado/gen.py",
 line 736, in run
    yielded = self.gen.throw(*exc_info)  # type: ignore
  File 
"/home/bird/miniconda3/envs/pyarrowtest/lib/python3.7/site-packages/distributed/comm/tcp.py",
 line 224, in write
    'recipient': self._peer_addr})
  File 
"/home/bird/miniconda3/envs/pyarrowtest/lib/python3.7/site-packages/tornado/gen.py",
 line 729, in run
    value = future.result()
  File 
"/home/bird/miniconda3/envs/pyarrowtest/lib/python3.7/site-packages/tornado/gen.py",
 line 736, in run
    yielded = self.gen.throw(*exc_info)  # type: ignore
  File 
"/home/bird/miniconda3/envs/pyarrowtest/lib/python3.7/site-packages/distributed/comm/utils.py",
 line 50, in to_frames
    res = yield offload(_to_frames)
  File 
"/home/bird/miniconda3/envs/pyarrowtest/lib/python3.7/site-packages/tornado/gen.py",
 line 729, in run
    value = future.result()
  File 
"/home/bird/miniconda3/envs/pyarrowtest/lib/python3.7/concurrent/futures/_base.py",
 line 425, in result
    return self.__get_result()
  File 
"/home/bird/miniconda3/envs/pyarrowtest/lib/python3.7/concurrent/futures/_base.py",
 line 384, in __get_result
    raise self._exception
  File 
"/home/bird/miniconda3/envs/pyarrowtest/lib/python3.7/concurrent/futures/thread.py",
 line 57, in run
    result = self.fn(*self.args, **self.kwargs)
  File 
"/home/bird/miniconda3/envs/pyarrowtest/lib/python3.7/site-packages/distributed/comm/utils.py",
 line 43, in _to_frames
    context=context))
  File 
"/home/bird/miniconda3/envs/pyarrowtest/lib/python3.7/site-packages/distributed/protocol/core.py",
 line 54, in dumps
    for key, value in data.items()
  File 
"/home/bird/miniconda3/envs/pyarrowtest/lib/python3.7/site-packages/distributed/protocol/core.py",
 line 55, in <dictcomp>
    if type(value) is Serialize}
  File 
"/home/bird/miniconda3/envs/pyarrowtest/lib/python3.7/site-packages/distributed/protocol/serialize.py",
 line 164, in serialize
    raise TypeError(msg, str(x)[:10000])
TypeError: ('Could not serialize object of type tuple.', "(<function safe_head 
at 0x7f3d57f2c7b8>, (<function _read_pyarrow_parquet_piece at 0x7f3d57ef9268>, 
<dask.bytes.local.LocalFileSystem object at 0x7f3db5
8ea4e0>, 
ParquetDatasetPiece('javascript_10percent_value_1000_only.parquet/part.0.parquet',
 row_group=None, partition_keys=[]), ['argument_0', 'argument_1', 'argument_2', 
'argument_3', 'argument_4', 'argument_5
', 'argument_6', 'argument_7', 'arguments', 'arguments_len', 'call_stack', 
'crawl_id', 'document_url', 'func_name', 'in_iframe', 'operation', 
'script_col', 'script_line', 'script_loc_eval', 'script_url', 'symbo
l', 'time_stamp', 'top_level_url', 'value_1000', 'value_len', 'visit_id'], [], 
False, None, []), 5)"){code}
[^part.0.parquet]


was (Author: birdsarah):
[~kszucs] this is with variations on my dataset. I have attached one piece 
which is sufficient to reproduce the error. It is web crawl data. The dtypes 
are:

{code:python}
argument_0                      object
argument_1                      object
argument_2                      object
argument_3                      object
argument_4                      object
argument_5                      object
argument_6                      object
argument_7                      object
arguments                       object
arguments_len                    int64
call_stack                      object
crawl_id                         int32
document_url                    object
func_name                       object
in_iframe                         bool
operation                       object
script_col                       int64
script_line                      int64
script_loc_eval                 object
script_url                      object
symbol                          object
time_stamp         datetime64[ns, UTC]
top_level_url                   object
value_1000                      object
value_len                        int64
visit_id                         int64
dtype: object
{code}

The end of my traceback is:

{code}
  File 
"/home/bird/miniconda3/envs/pyarrowtest/lib/python3.7/site-packages/distributed/protocol/core.py",
 line 55, in <dictcomp>
    if type(value) is Serialize}                            
  File 
"/home/bird/miniconda3/envs/pyarrowtest/lib/python3.7/site-packages/distributed/protocol/serialize.py",
 line 164, in serialize
    raise TypeError(msg, str(x)[:10000])          
TypeError: ('Could not serialize object of type tuple.', "(<function safe_head 
at 0x7f9eebc4b510>, (<function _read_pyarrow_parquet_piece at 0x7f9eebc08f28>, 
<dask.bytes.local.LocalFileSystem
 object at 0x7f9f44374a90>, 
ParquetDatasetPiece('javascript_10percent_value_1000_only.parquet/part.0.parquet',
 row_group=None, partition_keys=[]), ['argument_0', 'argument_1', 'argument_2', 
'argument_3', 'argument_4', 'argument_5', 'argument_6', 'argument_7', 
'arguments', 'arguments_len', 'call_stack', 'crawl_id', 'document_url', 
'func_name', 'in_iframe', 'operation', 'script_col', 'script_line', 
'script_loc_eval', 'script_url', 'symbol', 'time_stamp', 'top_level_url', 
'value_1000', 'value_len', 'visit_id'], [], False, None, []), 5)")
{code}

[^part.0.parquet] 

> [Python] ParquetDataset and ParquetPiece not serializable
> ---------------------------------------------------------
>
>                 Key: ARROW-5144
>                 URL: https://issues.apache.org/jira/browse/ARROW-5144
>             Project: Apache Arrow
>          Issue Type: Bug
>          Components: Python
>    Affects Versions: 0.13.0
>         Environment: osx python36/conda cloudpickle 0.8.1
> arrow-cpp                 0.13.0           py36ha71616b_0    conda-forge
> pyarrow                   0.13.0           py36hb37e6aa_0    conda-forge
>            Reporter: Martin Durant
>            Assignee: Krisztian Szucs
>            Priority: Critical
>              Labels: pull-request-available
>         Attachments: part.0.parquet
>
>          Time Spent: 2h 40m
>  Remaining Estimate: 0h
>
> Since 0.13.0, parquet instances are no longer serialisable, which means that 
> dask.distributed cannot pass them between processes in order to load parquet 
> in parallel.
> Example:
> ```
> >>> import cloudpickle
> >>> import pyarrow.parquet as pq
> >>> pf = pq.ParquetDataset('nation.impala.parquet')
> >>> cloudpickle.dumps(pf)
> ~/anaconda/envs/py36/lib/python3.6/site-packages/cloudpickle/cloudpickle.py 
> in dumps(obj, protocol)
>     893     try:
>     894         cp = CloudPickler(file, protocol=protocol)
> --> 895         cp.dump(obj)
>     896         return file.getvalue()
>     897     finally:
> ~/anaconda/envs/py36/lib/python3.6/site-packages/cloudpickle/cloudpickle.py 
> in dump(self, obj)
>     266         self.inject_addons()
>     267         try:
> --> 268             return Pickler.dump(self, obj)
>     269         except RuntimeError as e:
>     270             if 'recursion' in e.args[0]:
> ~/anaconda/envs/py36/lib/python3.6/pickle.py in dump(self, obj)
>     407         if self.proto >= 4:
>     408             self.framer.start_framing()
> --> 409         self.save(obj)
>     410         self.write(STOP)
>     411         self.framer.end_framing()
> ~/anaconda/envs/py36/lib/python3.6/pickle.py in save(self, obj, 
> save_persistent_id)
>     519
>     520         # Save the reduce() output and finally memoize the object
> --> 521         self.save_reduce(obj=obj, *rv)
>     522
>     523     def persistent_id(self, obj):
> ~/anaconda/envs/py36/lib/python3.6/pickle.py in save_reduce(self, func, args, 
> state, listitems, dictitems, obj)
>     632
>     633         if state is not None:
> --> 634             save(state)
>     635             write(BUILD)
>     636
> ~/anaconda/envs/py36/lib/python3.6/pickle.py in save(self, obj, 
> save_persistent_id)
>     474         f = self.dispatch.get(t)
>     475         if f is not None:
> --> 476             f(self, obj) # Call unbound method with explicit self
>     477             return
>     478
> ~/anaconda/envs/py36/lib/python3.6/pickle.py in save_dict(self, obj)
>     819
>     820         self.memoize(obj)
> --> 821         self._batch_setitems(obj.items())
>     822
>     823     dispatch[dict] = save_dict
> ~/anaconda/envs/py36/lib/python3.6/pickle.py in _batch_setitems(self, items)
>     845                 for k, v in tmp:
>     846                     save(k)
> --> 847                     save(v)
>     848                 write(SETITEMS)
>     849             elif n:
> ~/anaconda/envs/py36/lib/python3.6/pickle.py in save(self, obj, 
> save_persistent_id)
>     494             reduce = getattr(obj, "__reduce_ex__", None)
>     495             if reduce is not None:
> --> 496                 rv = reduce(self.proto)
>     497             else:
>     498                 reduce = getattr(obj, "__reduce__", None)
> ~/anaconda/envs/py36/lib/python3.6/site-packages/pyarrow/_parquet.cpython-36m-darwin.so
>  in pyarrow._parquet.ParquetSchema.__reduce_cython__()
> TypeError: no default __reduce__ due to non-trivial __cinit__
> ```
> The indicated schema instance is also referenced by the ParquetDatasetPiece s.
> ref: https://github.com/dask/distributed/issues/2597



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to