Skip to content

Fix OverflowError in DatasetCoder.estimate_size for datasets > 2 GB#197

Open
chase-dwelle wants to merge 1 commit into
google:mainfrom
chase-dwelle:chase-dwelle/fix-overflow-error
Open

Fix OverflowError in DatasetCoder.estimate_size for datasets > 2 GB#197
chase-dwelle wants to merge 1 commit into
google:mainfrom
chase-dwelle:chase-dwelle/fix-overflow-error

Conversation

@chase-dwelle
Copy link
Copy Markdown

@chase-dwelle chase-dwelle commented May 14, 2026

Fixes #196

DatasetCoder.estimate_size returns value.nbytes directly. For datasets larger than ~2 GB, this overflows a C int in Apache Beam's Cython internals, crashing the pipeline with:

OverflowError: value too large to convert to int

Root cause

The overflow occurs in CoderImpl._get_nested_size, declared in Beam's Cython layer (coder_impl.pxd) as:

cpdef int _get_nested_size(self, int inner_size, bint nested)

Both the parameter and return type are int, which Cython compiles to a C 32-bit signed integer. The maximum value is 2^31 - 1 (2,147,483,647 bytes, ~2 GB). Any estimate_size result at or above 2^31 overflows this conversion.

I first noticed this when loading GRIBs from ECMWF open data, which can create large arrays (51 ensemble members, 13 pressure levels, 721x1440 grid).

Downstream impact of capping the estimate

estimate_size feeds into Beam's monitoring and optimization heuristics:

  • Byte count metrics -- MeanByteCount distribution counters per transform (used in Beam UI/logs)
  • Source splitting -- chunk sizes for parallel I/O: max(1MB, 1000 * sqrt(total_size))
  • Batch sizing -- mean element size estimates for batched operations
  • Load balancing -- GroupIntoBatches.WithShardedKey distribution decisions

Capping at 2^31 - 1 means metrics will underreport and parallelism heuristics may be slightly suboptimal for very large datasets. I prefer this to the alternative of an OverflowError that kills the pipeline.

Upstream

The real fix would be for Beam to change _get_nested_size to use int64_t instead of int. I'm introducing the workaround here since geospatial workflows offer uniquely large data.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

OverflowError when a chunk exceeds 2 GB

1 participant