Skip to content

TypeError: memoryview: cannot cast view with zeros in shape or strides #4594

@sandeshlmore

Description

@sandeshlmore

What happened:

distributed.protocol.core - CRITICAL - Failed to deserialize
Traceback (most recent call last):
  File "/home/sandeshmore/PycharmProjects/dask_latest/dask_03/lib/python3.8/site-packages/distributed/protocol/core.py", line 130, in loads
    value = merge_and_deserialize(head, fs, deserializers=deserializers)
  File "/home/sandeshmore/PycharmProjects/dask_latest/dask_03/lib/python3.8/site-packages/distributed/protocol/serialize.py", line 450, in merge_and_deserialize
    return deserialize(header, merged_frames, deserializers=deserializers)
  File "/home/sandeshmore/PycharmProjects/dask_latest/dask_03/lib/python3.8/site-packages/distributed/protocol/serialize.py", line 384, in deserialize
    return loads(header, frames)
  File "/home/sandeshmore/PycharmProjects/dask_latest/dask_03/lib/python3.8/site-packages/distributed/protocol/serialize.py", line 79, in pickle_loads
    buffers[i] = memoryview(bytes(mv)).cast(mv.format, mv.shape)
TypeError: memoryview: cannot cast view with zeros in shape or strides
distributed.core - ERROR - memoryview: cannot cast view with zeros in shape or strides
Traceback (most recent call last):
  File "/home/sandeshmore/PycharmProjects/dask_latest/dask_03/lib/python3.8/site-packages/distributed/core.py", line 554, in handle_stream
    msgs = await comm.read()
  File "/home/sandeshmore/PycharmProjects/dask_latest/dask_03/lib/python3.8/site-packages/distributed/comm/tcp.py", line 217, in read
    msg = await from_frames(
  File "/home/sandeshmore/PycharmProjects/dask_latest/dask_03/lib/python3.8/site-packages/distributed/comm/utils.py", line 80, in from_frames
    res = _from_frames()
  File "/home/sandeshmore/PycharmProjects/dask_latest/dask_03/lib/python3.8/site-packages/distributed/comm/utils.py", line 63, in _from_frames
    return protocol.loads(
  File "/home/sandeshmore/PycharmProjects/dask_latest/dask_03/lib/python3.8/site-packages/distributed/protocol/core.py", line 130, in loads
    value = merge_and_deserialize(head, fs, deserializers=deserializers)
  File "/home/sandeshmore/PycharmProjects/dask_latest/dask_03/lib/python3.8/site-packages/distributed/protocol/serialize.py", line 450, in merge_and_deserialize
    return deserialize(header, merged_frames, deserializers=deserializers)
  File "/home/sandeshmore/PycharmProjects/dask_latest/dask_03/lib/python3.8/site-packages/distributed/protocol/serialize.py", line 384, in deserialize
    return loads(header, frames)
  File "/home/sandeshmore/PycharmProjects/dask_latest/dask_03/lib/python3.8/site-packages/distributed/protocol/serialize.py", line 79, in pickle_loads
    buffers[i] = memoryview(bytes(mv)).cast(mv.format, mv.shape)
TypeError: memoryview: cannot cast view with zeros in shape or strides
distributed.worker - ERROR - memoryview: cannot cast view with zeros in shape or strides

Minimal Complete Verifiable Example:

import dask.dataframe as dd
import pandas as pd

if __name__ == '__main__':
    from dask.distributed import LocalCluster, Client

    cluster = LocalCluster(n_workers=4)
    client = Client(cluster)

    newer_items_df = dd.from_pandas(pd.DataFrame([
        {'A': '1', 'B': '1', 'C': 1, 'D': 'A3'},
        {'A': '1',  'B': '1', 'C': 2, 'D': 'A4'},
        {'A': '123',  'B': '123', 'C': 3, 'D': 'A5'},
        {'A': '123', 'B': '123', 'C': 3, 'D': 'A6'},
        {'A': '123', 'B': '123', 'C': 4, 'D': 'A7'},
        {'A': '123', 'B': '123', 'C': 5, 'D': 'A8'},
           ]), npartitions=5)
    ##This works in release 2021.1.1 and not in 2021.3.0
    Interclusdistance1 = newer_items_df.groupby(['A', 'B'])[['C']].max().rename(
        columns={'C': 'InterClus_distance'}).reset_index()

    # ##This works in release 2021.3.0
    # Interclusdistance2 = newer_items_df.groupby(['A', 'B'])[['C']].max().reset_index().rename(
    #     columns={'C': 'InterClus_distance'})

    print(Interclusdistance1.compute())     ## This fails
    # print(Interclusdistance2.compute())  ##This works in newer release 2021.3.0

This issue happend After updating dask version to 2021.3.0 from 2021.1.1

Environment:

  • Dask version: 2021.3.0
  • Python version: 3.8.3
  • Operating System: running python 3.8.3 docker image
  • Install method (conda, pip, source): pip

Dependencies:
Package Version


appdirs 1.4.3
CacheControl 0.12.6
certifi 2019.11.28
chardet 3.0.4
click 7.1.2
cloudpickle 1.6.0
colorama 0.4.3
contextlib2 0.6.0
dask 2021.3.0
distlib 0.3.0
distributed 2021.3.0
distro 1.4.0
fsspec 0.8.7
HeapDict 1.0.1
html5lib 1.0.1
idna 2.8
ipaddr 2.2.0
lockfile 0.12.2
msgpack 0.6.2
numpy 1.20.1
packaging 20.3
pandas 1.2.3
pep517 0.8.2
pip 20.0.2
pkg-resources 0.0.0
progress 1.5
psutil 5.8.0
pyparsing 2.4.6
python-dateutil 2.8.1
pytoml 0.1.21
pytz 2021.1
PyYAML 5.4.1
requests 2.22.0
retrying 1.3.3
setuptools 44.0.0
six 1.14.0
sortedcontainers 2.3.0
tblib 1.7.0
toolz 0.11.1
tornado 6.1
urllib3 1.25.8
webencodings 0.5.1
wheel 0.34.2
zict 2.0.0

Metadata

Metadata

Assignees

No one assigned

    Labels

    No labels
    No labels

    Type

    No type

    Projects

    No projects

    Milestone

    No milestone

    Relationships

    None yet

    Development

    No branches or pull requests

    Issue actions