You signed in with another tab or window. Reload to refresh your session.You signed out in another tab or window. Reload to refresh your session.You switched accounts on another tab or window. Reload to refresh your session.Dismiss alert
Applying map_blocks to a Dask Array constructed by from_array from HDF5 dataset raises TypeError: h5py objects cannot be pickled. This only occurs when using distributed.
This maybe related to #860 and #861. In fact, the below minimal verifiable example is borrowed from test_h5py_serialize mentioned in #860.
Minimal Complete Verifiable Example:
# from dask.utils import SerializableLockimportdistributedimporth5pyimportdask.arrayasdaclient=distributed.Client() # Error does not occur if this is commented out fn='temporary_file_for_test.h5'lock=True# lock = SerializableLock('hdf5') and lock = False also leads to an error.withh5py.File(fn, mode='w') asf:
x=f.create_dataset('/group/x', shape=(4,), dtype='i4',
chunks=(2,))
x[:] = [1, 2, 3, 4]
withh5py.File(fn, mode='r') asf:
dset=f['/group/x']
x=da.from_array(dset, chunks=dset.chunks, lock=lock)
deffunc(array):
returnarray**2res=da.map_blocks(func, x).compute()
Anything else we need to know?:
Traceback on Jupyter Lab looks like following:
2025-02-1322:00:12,875-distributed.protocol.pickle-ERROR-Failedtoserialize<ToPickle: HighLevelGraphwith1layers.
<dask.highlevelgraph.HighLevelGraphobjectat0x7fc9a461f350>0.140504024525248>.
Traceback (mostrecentcalllast):
File"[/home/sm69/.conda/envs/pyathena/lib/python3.13/site-packages/distributed/protocol/pickle.py", line73](http://127.0.0.1:20201/home/sm69/.conda/envs/pyathena/lib/python3.13/site-packages/distributed/protocol/pickle.py#line=72), in dumpsresult=cloudpickle.dumps(x, **dump_kwargs)
File"[/home/sm69/.conda/envs/pyathena/lib/python3.13/site-packages/cloudpickle/cloudpickle.py", line1537](http://127.0.0.1:20201/home/sm69/.conda/envs/pyathena/lib/python3.13/site-packages/cloudpickle/cloudpickle.py#line=1536), in dumpscp.dump(obj)
~~~~~~~^^^^^File"[/home/sm69/.conda/envs/pyathena/lib/python3.13/site-packages/cloudpickle/cloudpickle.py", line1303](http://127.0.0.1:20201/home/sm69/.conda/envs/pyathena/lib/python3.13/site-packages/cloudpickle/cloudpickle.py#line=1302), in dumpreturnsuper().dump(obj)
~~~~~~~~~~~~^^^^^File"[/home/sm69/.conda/envs/pyathena/lib/python3.13/site-packages/h5py/_hl/base.py", line366](http://127.0.0.1:20201/home/sm69/.conda/envs/pyathena/lib/python3.13/site-packages/h5py/_hl/base.py#line=365), in __getnewargs__raiseTypeError("h5py objects cannot be pickled")
TypeError: h5pyobjectscannotbepickledDuringhandlingoftheaboveexception, anotherexceptionoccurred:
Traceback (mostrecentcalllast):
File"[/home/sm69/.conda/envs/pyathena/lib/python3.13/site-packages/distributed/protocol/pickle.py", line77](http://127.0.0.1:20201/home/sm69/.conda/envs/pyathena/lib/python3.13/site-packages/distributed/protocol/pickle.py#line=76), in dumpsresult=cloudpickle.dumps(x, **dump_kwargs)
File"[/home/sm69/.conda/envs/pyathena/lib/python3.13/site-packages/cloudpickle/cloudpickle.py", line1537](http://127.0.0.1:20201/home/sm69/.conda/envs/pyathena/lib/python3.13/site-packages/cloudpickle/cloudpickle.py#line=1536), in dumpscp.dump(obj)
~~~~~~~^^^^^File"[/home/sm69/.conda/envs/pyathena/lib/python3.13/site-packages/cloudpickle/cloudpickle.py", line1303](http://127.0.0.1:20201/home/sm69/.conda/envs/pyathena/lib/python3.13/site-packages/cloudpickle/cloudpickle.py#line=1302), in dumpreturnsuper().dump(obj)
~~~~~~~~~~~~^^^^^File"[/home/sm69/.conda/envs/pyathena/lib/python3.13/site-packages/h5py/_hl/base.py", line366](http://127.0.0.1:20201/home/sm69/.conda/envs/pyathena/lib/python3.13/site-packages/h5py/_hl/base.py#line=365), in __getnewargs__raiseTypeError("h5py objects cannot be pickled")
TypeError: h5pyobjectscannotbepickled---------------------------------------------------------------------------TypeErrorTraceback (mostrecentcalllast)
File [~/.conda/envs/pyathena/lib/python3.13/site-packages/distributed/protocol/pickle.py:73](http://127.0.0.1:20201/~/.conda/envs/pyathena/lib/python3.13/site-packages/distributed/protocol/pickle.py#line=72), in dumps(x, buffer_callback, protocol)72buffers.clear()
--->73result=cloudpickle.dumps(x, **dump_kwargs)
74exceptException:
File [~/.conda/envs/pyathena/lib/python3.13/site-packages/cloudpickle/cloudpickle.py:1537](http://127.0.0.1:20201/~/.conda/envs/pyathena/lib/python3.13/site-packages/cloudpickle/cloudpickle.py#line=1536), in dumps(obj, protocol, buffer_callback)1536cp=Pickler(file, protocol=protocol, buffer_callback=buffer_callback)
->1537cp.dump(obj)
1538returnfile.getvalue()
File [~/.conda/envs/pyathena/lib/python3.13/site-packages/cloudpickle/cloudpickle.py:1303](http://127.0.0.1:20201/~/.conda/envs/pyathena/lib/python3.13/site-packages/cloudpickle/cloudpickle.py#line=1302), in Pickler.dump(self, obj)1302try:
->1303returnsuper().dump(obj)
1304exceptRuntimeErrorase:
File [~/.conda/envs/pyathena/lib/python3.13/site-packages/h5py/_hl/base.py:366](http://127.0.0.1:20201/~/.conda/envs/pyathena/lib/python3.13/site-packages/h5py/_hl/base.py#line=365), in HLObject.__getnewargs__(self)357"""Disable pickle. 358 359 Handles for HDF5 objects can't be reliably deserialised, because the (...) 364 limitations, look at the h5pickle project on PyPI. 365 """-->366raiseTypeError("h5py objects cannot be pickled")
TypeError: h5pyobjectscannotbepickledDuringhandlingoftheaboveexception, anotherexceptionoccurred:
TypeErrorTraceback (mostrecentcalllast)
File [~/.conda/envs/pyathena/lib/python3.13/site-packages/distributed/protocol/serialize.py:366](http://127.0.0.1:20201/~/.conda/envs/pyathena/lib/python3.13/site-packages/distributed/protocol/serialize.py#line=365), in serialize(x, serializers, on_error, context, iterate_collection)365try:
-->366header, frames=dumps(x, context=context) ifwants_contextelsedumps(x)
367header["serializer"] =nameFile [~/.conda/envs/pyathena/lib/python3.13/site-packages/distributed/protocol/serialize.py:78](http://127.0.0.1:20201/~/.conda/envs/pyathena/lib/python3.13/site-packages/distributed/protocol/serialize.py#line=77), in pickle_dumps(x, context)76writeable.append(notf.readonly)
--->78frames[0] =pickle.dumps(
79x,
80buffer_callback=buffer_callback,
81protocol=context.get("pickle-protocol", None) ifcontextelseNone,
82 )
83header= {
84"serializer": "pickle",
85"writeable": tuple(writeable),
86 }
File [~/.conda/envs/pyathena/lib/python3.13/site-packages/distributed/protocol/pickle.py:77](http://127.0.0.1:20201/~/.conda/envs/pyathena/lib/python3.13/site-packages/distributed/protocol/pickle.py#line=76), in dumps(x, buffer_callback, protocol)76buffers.clear()
--->77result=cloudpickle.dumps(x, **dump_kwargs)
78exceptException:
File [~/.conda/envs/pyathena/lib/python3.13/site-packages/cloudpickle/cloudpickle.py:1537](http://127.0.0.1:20201/~/.conda/envs/pyathena/lib/python3.13/site-packages/cloudpickle/cloudpickle.py#line=1536), in dumps(obj, protocol, buffer_callback)1536cp=Pickler(file, protocol=protocol, buffer_callback=buffer_callback)
->1537cp.dump(obj)
1538returnfile.getvalue()
File [~/.conda/envs/pyathena/lib/python3.13/site-packages/cloudpickle/cloudpickle.py:1303](http://127.0.0.1:20201/~/.conda/envs/pyathena/lib/python3.13/site-packages/cloudpickle/cloudpickle.py#line=1302), in Pickler.dump(self, obj)1302try:
->1303returnsuper().dump(obj)
1304exceptRuntimeErrorase:
File [~/.conda/envs/pyathena/lib/python3.13/site-packages/h5py/_hl/base.py:366](http://127.0.0.1:20201/~/.conda/envs/pyathena/lib/python3.13/site-packages/h5py/_hl/base.py#line=365), in HLObject.__getnewargs__(self)357"""Disable pickle. 358 359 Handles for HDF5 objects can't be reliably deserialised, because the (...) 364 limitations, look at the h5pickle project on PyPI. 365 """-->366raiseTypeError("h5py objects cannot be pickled")
TypeError: h5pyobjectscannotbepickledTheaboveexceptionwasthedirectcauseofthefollowingexception:
TypeErrorTraceback (mostrecentcalllast)
CellIn[1], line1917deffunc(array):
18returnarray**2--->19res=da.map_blocks(func, x).compute()
File [~/.conda/envs/pyathena/lib/python3.13/site-packages/dask/base.py:372](http://127.0.0.1:20201/~/.conda/envs/pyathena/lib/python3.13/site-packages/dask/base.py#line=371), in DaskMethodsMixin.compute(self, **kwargs)348defcompute(self, **kwargs):
349"""Compute this dask collection 350 351 This turns a lazy Dask collection into its in-memory equivalent. (...) 370 dask.compute 371 """-->372 (result,) =compute(self, traverse=False, **kwargs)
373returnresultFile [~/.conda/envs/pyathena/lib/python3.13/site-packages/dask/base.py:660](http://127.0.0.1:20201/~/.conda/envs/pyathena/lib/python3.13/site-packages/dask/base.py#line=659), in compute(traverse, optimize_graph, scheduler, get, *args, **kwargs)657postcomputes.append(x.__dask_postcompute__())
659withshorten_traceback():
-->660results=schedule(dsk, keys, **kwargs)
662returnrepack([f(r, *a) forr, (f, a) inzip(results, postcomputes)])
File [~/.conda/envs/pyathena/lib/python3.13/site-packages/distributed/protocol/serialize.py:392](http://127.0.0.1:20201/~/.conda/envs/pyathena/lib/python3.13/site-packages/distributed/protocol/serialize.py#line=391), in serialize(x, serializers, on_error, context, iterate_collection)390exceptException:
391raiseTypeError(msg) fromexc-->392raiseTypeError(msg, str_x) fromexc393else: # pragma: nocover394raiseValueError(f"{on_error=}; expected 'message' or 'raise'")
TypeError: ('Could not serialize object of type HighLevelGraph', '<ToPickle: HighLevelGraph with 1 layers.\n<dask.highlevelgraph.HighLevelGraph object at 0x7fc9a461f350>\n 0. 140504024525248\n>')
Describe the issue:
Applying
map_blocks
to a Dask Array constructed byfrom_array
from HDF5 dataset raisesTypeError: h5py objects cannot be pickled
. This only occurs when usingdistributed
.This maybe related to #860 and #861. In fact, the below minimal verifiable example is borrowed from
test_h5py_serialize
mentioned in #860.Minimal Complete Verifiable Example:
Anything else we need to know?:
Traceback on Jupyter Lab looks like following:
Environment:
The text was updated successfully, but these errors were encountered: