Skip to content

[Bug]: StateBackedIterable serializes elements size for every element when ComposedCombine is used #33620

@stankiewicz

Description

@stankiewicz

What happened?

StateBackedIterable which is part of FnHarness is reporting size for every element without looking at cost of encoding. Other parts of SDK take into account isRegisterByteSizeObserverCheap and depending on response they sample.

To reproduce:

  1. Implement ComposedCombine with 1k of accumulators
  2. Create input of millions of elements
  3. Combine globally.
  4. Run on Dataflow v2

Straggler - "Operation ongoing" with following stacktrace fragment:


[..]
 at app//org.apache.beam.sdk.transforms.CombineFns$ComposedAccumulatorCoder.encode(CombineFns.java:649)
  at app//org.apache.beam.sdk.coders.Coder.getEncodedElementByteSize(Coder.java:297)
  at app//org.apache.beam.sdk.coders.LengthPrefixCoder.getEncodedElementByteSize(LengthPrefixCoder.java:109)
  at app//org.apache.beam.sdk.coders.Coder.registerByteSizeObserver(Coder.java:291)
  at app//org.apache.beam.fn.harness.state.StateBackedIterable$WrappedObservingIterator.next(StateBackedIterable.java:141)
  at app//org.apache.beam.sdk.transforms.CombineFns$ProjectionIterable$1.next(CombineFns.java:638)
  at app//com.google.common.collect.Iterators$5.computeNext(Iterators.java:672)
  at app//com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:145)
  at app//com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:140)
  at app//foo.bar.MyAccumulator.mergeAccumulators(MyAccumulator.java:418)
[..]

Expected outcome - no straggler.

Issue Priority

Priority: 2 (default / most bugs should be filed as P2)

Issue Components

  • Component: Python SDK
  • Component: Java SDK
  • Component: Go SDK
  • Component: Typescript SDK
  • Component: IO connector
  • Component: Beam YAML
  • Component: Beam examples
  • Component: Beam playground
  • Component: Beam katas
  • Component: Website
  • Component: Infrastructure
  • Component: Spark Runner
  • Component: Flink Runner
  • Component: Samza Runner
  • Component: Twister2 Runner
  • Component: Hazelcast Jet Runner
  • Component: Google Cloud Dataflow Runner

Metadata

Metadata

Assignees

No one assigned

    Type

    No type
    No fields configured for issues without a type.

    Projects

    No projects

    Relationships

    None yet

    Development

    No branches or pull requests

    Issue actions