Skip to content

OverflowError when a chunk exceeds 2 GB #196

@chase-dwelle

Description

@chase-dwelle

DatasetCoder.estimate_size returns value.nbytes directly (core.py#L275). For
chunks larger than 2**31 - 1 bytes (~2 GB), this overflows the C int in Beam's Cython CallbackCoderImpl.estimate_size and raises:

OverflowError: value too large to convert to int

Noticed this hitting after upgrading to the latest version when loading datasets from GRIBs with multple atmostpheric levels, e.g., ECMWF IFS ensembles (51 members × 13 levels × 721 × 1440), where a single per-key xarray.Dataset chunk exceeds 2 GB before any rechunk.

Minimal repro:

import apache_beam as beam
import numpy as np
import xarray as xr
import xarray_beam as xbeam

ds = xr.Dataset({"x": (("a", "b"), np.zeros((20000, 14000)))})  # ~2.1 GB

with beam.Pipeline() as p:
    _ = (
        p
        | beam.Create([(xbeam.Key({}), ds)])
        | beam.Reshuffle()
    )

Capping the returned estimate at 2**31 - 1 (with a warning so oversized chunks are still visible) avoids the crash, will put up a PR.

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