Skip to content

Inject SDK-side flattens while handling input/output coder mismatch in flattens. #34641

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Merged
merged 5 commits into from
Apr 16, 2025

Conversation

shunping
Copy link
Collaborator

@shunping shunping commented Apr 16, 2025

Following the idea of identity transform (#32930 (comment)) and valuable discussions with @lostluck, I have developed a "simpler and more general fix" for Prism runner issues involving Flatten. This approach works not only for Flatten->Flatten scenarios, but also for others like GroupBy->Flatten.

The key observations are:

  • Prism's Runner Flatten only passes data through from its input PCollections.
  • No subsequent step within the runner enforces the correct output coder for this Runner Flatten output.

Previous approaches (including #34602) attempted to fix this by by overwriting the upstream coders with the Flatten output coder. This works in many scenarios, especially when the upstream transform is a SDK-side transform or another Flatten (where #34602 ensured coder propagation).

However, modifying upstream PCollection coders can cause side effects.

  • In the groupby->flatten example (org.apache.beam.sdk.transforms.FlattenTest.testFlattenWithDifferentInputAndOutputCoders2), changing GroupBy's coder has no effect on the actual encoded data, because GroupBy always generate K,Iterable<V>. Downstream transforms expecting data encoded with the Flatten's coder then fail during decoding.
  • Another example (see issue [Bug]: Prism crashed if Flatten and GroupByKey share the same input #34643) involves a pipeline like C = [A, B] | Flatten(), D = A | GroupByKey(). If Flatten overwrites the coder associated with PCollection A, the GroupByKey on A may fail due to coder incompatibility.

The proposed solution avoids the pitfalls of modifying upstream coders. Instead, it inserts an identity transform (specifically, an SDK-side Flatten) before the Runner Flatten. This identity transform implicitly converts the input PCollection's elements to use the same output coder as the target Runner Flatten, ensuring the correctly encoded data emitted from it.

fixes #32930
fixes #34643

The PR also fixes the flaky test of Test_preprocessor_preProcessGraph/ignoreEmptyAndIdentityTransform under https://github.com/apache/beam/actions/workflows/beam_PreCommit_Go.yml.

@shunping shunping marked this pull request as ready for review April 16, 2025 04:02
@shunping shunping requested a review from lostluck April 16, 2025 04:03
Copy link
Contributor

Assigning reviewers. If you would like to opt out of this review, comment assign to next reviewer:

R: @jrmccluskey for label go.

Available commands:

  • stop reviewer notifications - opt out of the automated review tooling
  • remind me after tests pass - tag the comment author after tests pass
  • waiting on author - shift the attention set back to the author (any comment or push by the author will return the attention set to the reviewers)

The PR bot will only process comments in the main thread (not review comments).

@shunping shunping self-assigned this Apr 16, 2025
@shunping shunping changed the title Inject SDK-side flattens while handling coder mismatch in flattens. Inject SDK-side flattens while handling input/output coder mismatch in flattens. Apr 16, 2025
…the fix.

The test is also included in the test suite of flink, samza and spark, but without
transcoding until their corresponding FRs are resolved.
@github-actions github-actions bot added python and removed python labels Apr 16, 2025
@shunping shunping force-pushed the prism-sdk-side-flatten-injection branch from 9d6ec0a to 369859b Compare April 16, 2025 17:11
@github-actions github-actions bot added python and removed python labels Apr 16, 2025
@github-actions github-actions bot added python and removed python labels Apr 16, 2025
Copy link
Contributor

@lostluck lostluck left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I like how clean this ended up. Great work.

@@ -88,8 +88,7 @@ func (h *runner) PrepareTransform(tid string, t *pipepb.PTransform, comps *pipep
}

func (h *runner) handleFlatten(tid string, t *pipepb.PTransform, comps *pipepb.Components) prepareResult {
if !h.config.SDKFlatten {
t.EnvironmentId = "" // force the flatten to be a runner transform due to configuration.
if !h.config.SDKFlatten && !strings.HasPrefix(tid, "ft_") {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I'll note that there's no user serviceable way to do these configurations at the moment, and it really was a hard binary. It would be acceptable to remove the SDKFlatten option in favour of just a single approach that biases to runner flattens, but does the SDK flatten to get around these issues.

@@ -492,6 +492,9 @@ func finalizeStage(stg *stage, comps *pipepb.Components, pipelineFacts *fusionFa
}

stg.internalCols = internal
// Sort the keys of internal producers (from stageFacts.PcolProducers)
// to ensure deterministic order for stable tests.
sort.Strings(stg.internalCols)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Good find! I thought I had everything deterministic already.

@lostluck lostluck merged commit 4ead940 into apache:master Apr 16, 2025
108 checks passed
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
2 participants