Skip to content

Fixes metadata propagation from doFn output to next parDo by introducing new interface and plumbing propagation#37851

Draft
stankiewicz wants to merge 3 commits intoapache:masterfrom
stankiewicz:metadata_propagation
Draft

Fixes metadata propagation from doFn output to next parDo by introducing new interface and plumbing propagation#37851
stankiewicz wants to merge 3 commits intoapache:masterfrom
stankiewicz:metadata_propagation

Conversation

@stankiewicz
Copy link
Contributor

When data is outputted with processContext, outputReceiver or builder, it's going through sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnOutputReceivers.java which was still using context.outputWindowedValue(element, ts, windows, pane ...) interface which we decided to abandon as adding new metadata requires changing this interface and we should use builder and windowedValue interface.

  • adds new outputWindowedValue(WV) public interface with implementations
  • add tests to cover metadata propagation across parDos in single and mulitple stages for direct runner
  • plumbs metadata propagation, where it was lost previously.

Thank you for your contribution! Follow this checklist to help us incorporate your contribution quickly and easily:

  • Mention the appropriate issue in your description (for example: addresses #123), if applicable. This will automatically add a link to the pull request in the issue. If you would like the issue to automatically close on merging the pull request, comment fixes #<ISSUE NUMBER> instead.
  • Update CHANGES.md with noteworthy changes.
  • If this contribution is large, please file an Apache Individual Contributor License Agreement.

See the Contributor Guide for more tips on how to make review process smoother.

To check the build health, please visit https://github.com/apache/beam/blob/master/.test-infra/BUILD_STATUS.md

GitHub Actions Tests Status (on master branch)

Build python source distribution and wheels
Python tests
Java tests
Go tests

See CI.md for more information about GitHub Actions CI or the workflows README to see a list of phrases to trigger workflows.

@gemini-code-assist
Copy link
Contributor

Summary of Changes

Hello, I'm Gemini Code Assist1! I'm currently reviewing this pull request and will post my feedback shortly. In the meantime, here's a summary to help you and other reviewers quickly get up to speed!

This pull request addresses a critical issue where metadata associated with WindowedValue objects was not fully propagated between DoFn outputs and subsequent ParDo operations. The existing output mechanism in DoFnOutputReceivers was deconstructing WindowedValue into individual components, leading to the loss of richer metadata like CausedByDrain, record IDs, and offsets. To resolve this, a new outputWindowedValue(WindowedValue) interface has been introduced, allowing DoFns to output the complete WindowedValue object directly. This change ensures that all relevant metadata is preserved and correctly plumbed through the Beam pipeline, improving data fidelity and enabling more robust processing, particularly in scenarios involving draining and record tracking.

Highlights

  • New outputWindowedValue(WindowedValue) Interface: Introduced new outputWindowedValue(WindowedValue) methods in DoFn.Context and its implementations across various runners and testers to allow direct output of WindowedValue objects.
  • Refactored Output Mechanism: Refactored DoFnOutputReceivers to utilize the new outputWindowedValue(WindowedValue) interface, ensuring complete metadata propagation by avoiding deconstruction of WindowedValue components.
  • Enhanced Metadata Handling: Enhanced WindowedValues and ValueInSingleWindow to correctly handle and encode CausedByDrain, record IDs, and record offsets, ensuring these critical metadata fields are preserved throughout the pipeline.
  • New Metadata Propagation Tests: Added new integration tests to validate the correct propagation of metadata, specifically CausedByDrain, across ParDo and Redistribute transforms.

🧠 New Feature in Public Preview: You can now enable Memory to help Gemini Code Assist learn from your team's feedback. This makes future code reviews more consistent and personalized to your project's style. Click here to enable Memory in your admin console.

Changelog
  • runners/core-java/src/main/java/org/apache/beam/runners/core/OutputAndTimeBoundedSplittableProcessElementInvoker.java
    • Added new outputWindowedValue methods to support direct WindowedValue output.
  • runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java
    • Implemented new outputWindowedValue methods in DoFnProcessContext and DoFnStartBundleContext to handle WindowedValue objects.
  • sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java
    • Imported WindowedValue and added abstract outputWindowedValue methods to DoFn.Context for direct WindowedValue output.
  • sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnOutputReceivers.java
    • Updated the output method to use the newly introduced outputWindowedValue(WindowedValue) interface, simplifying metadata handling.
  • sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java
    • Added new outputWindowedValue methods to correctly process and store WindowedValue objects during testing.
  • sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Redistribute.java
    • Modified the processElement method to propagate the CausedByDrain metadata using the output builder.
  • sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Reify.java
    • Updated the processElement method to accept ProcessContext and capture currentRecordId, currentRecordOffset, and causedByDrain for ValueInSingleWindow.
  • sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/SplittableParDoNaiveBounded.java
    • Imported WindowedValue, updated OutputBuilder to use the new outputWindowedValue interface, and added new outputWindowedValue methods to ProcessContext and StartBundleContext.
  • sdks/java/core/src/main/java/org/apache/beam/sdk/values/ValueInSingleWindow.java
    • Enhanced ElementMetadata encoding to include CausedByDrain status.
  • sdks/java/core/src/main/java/org/apache/beam/sdk/values/WindowedValues.java
    • Modified builder and of methods to ensure recordOffset, recordId, and causedByDrain are correctly propagated when creating WindowedValue instances.
  • sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/MetadataPropagationTest.java
    • Added a new test file containing CausedByDrainSettingDoFn and CausedByDrainExtractingDoFn to verify metadata propagation across transforms.
  • sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnApiDoFnRunner.java
    • Implemented new outputWindowedValue methods in various contexts to directly handle WindowedValue objects and removed a redundant causedByDrain override.
Activity
  • The pull request was created by stankiewicz.
  • The author provided a detailed description of the problem and the proposed solution, including a checklist for contribution.
Using Gemini Code Assist

The full guide for Gemini Code Assist can be found on our documentation page, here are some quick tips.

Invoking Gemini

You can request assistance from Gemini at any point by creating a comment using either /gemini <command> or @gemini-code-assist <command>. Below is a summary of the supported commands on the current page.

Feature Command Description
Code Review /gemini review Performs a code review for the current pull request in its current state.
Pull Request Summary /gemini summary Provides a summary of the current pull request in its current state.
Comment @gemini-code-assist Responds in comments when explicitly tagged, both in pull request comments and review comments.
Help /gemini help Displays a list of available commands.

Customization

To customize Gemini Code Assist for GitHub experience, repository maintainers can create a configuration file and/or provide a custom code review style guide (such as PEP-8 for Python) by creating and adding files to a .gemini/ folder in the base of the repository. Detailed instructions can be found here.

Limitations & Feedback

Gemini Code Assist may make mistakes. Please leave feedback on any instances where its feedback is incorrect or counter productive. You can react with 👍 and 👎 on @gemini-code-assist comments. If you're interested in giving your feedback about your experience with Gemini Code Assist for GitHub and other Google products, sign up here.

You can also get AI-powered code generation, chat, as well as code reviews directly in the IDE at no cost with the Gemini Code Assist IDE Extension.

Footnotes

  1. Review the Privacy Notices, Generative AI Prohibited Use Policy, Terms of Service, and learn how to configure Gemini Code Assist in GitHub here. Gemini can make mistakes, so double check it and use code with caution.


public abstract <T> void outputWindowedValue(TupleTag<T> tag, WindowedValue<T> windowedValue);

public abstract void outputWindowedValue(WindowedValue<OutputT> windowedValue);
Copy link
Contributor Author

Choose a reason for hiding this comment

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

@kennknowles I'm really curious if we are able to avoid that. You've spent quite a lot of time on outputbuilder work to avoid changing public interface and this method is exposed in ProcessContext so any user can use it if they want.

@stankiewicz stankiewicz force-pushed the metadata_propagation branch from 296350a to 90306f3 Compare March 13, 2026 17:04
- add tests to cover metadata propagation across parDos in single and mulitple stages for direct runner
- plumbs metadata propagation, where it was lost previously. 
spotless
@stankiewicz stankiewicz force-pushed the metadata_propagation branch from 90306f3 to 7beaf1e Compare March 13, 2026 17:04
@stankiewicz stankiewicz marked this pull request as draft March 13, 2026 17:13
@stankiewicz
Copy link
Contributor Author

few tests to fix as it looks like this approached skipped some timestamp validation.

@stankiewicz stankiewicz force-pushed the metadata_propagation branch from cb1c99c to fee5855 Compare March 13, 2026 18:58
@stankiewicz
Copy link
Contributor Author

/gemini review

@stankiewicz
Copy link
Contributor Author

Run Java_PVR_Prism_Loopback PreCommit

Copy link
Contributor

@gemini-code-assist gemini-code-assist bot left a comment

Choose a reason for hiding this comment

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

Code Review

This pull request introduces a new outputWindowedValue(WindowedValue) interface to better propagate metadata, replacing the older interface that took exploded parameters. The changes are well-implemented and plumbed through various runners and components. The new tests for metadata propagation are a great addition.

I've found a few areas for improvement:

  • There are a couple of instances of code duplication in new methods that could be refactored for better maintainability.
  • I noticed a potential correctness issue in SimpleDoFnRunner where a timestamp check is missing in newly added methods.
  • The new test class MetadataPropagationTest has a test isolation issue due to a shared static flag, which could lead to flaky tests.

Overall, this is a solid contribution that improves the SDK's metadata handling capabilities. Addressing the points above will make it even better.

@stankiewicz stankiewicz force-pushed the metadata_propagation branch from fee5855 to 55439e2 Compare March 13, 2026 20:44
@stankiewicz
Copy link
Contributor Author

/gemini review

Copy link
Contributor

@gemini-code-assist gemini-code-assist bot left a comment

Choose a reason for hiding this comment

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

Code Review

This pull request introduces a new outputWindowedValue(WindowedValue) interface to simplify metadata propagation, deprecating the older interface that required deconstructing the WindowedValue. The changes are plumbed through various DoFn runners and contexts, and new tests are added to verify the propagation of CausedByDrain metadata. The changes are well-implemented and consistent. I have a couple of minor suggestions for the new test file to improve maintainability.

…tadataPropagationTest.java

Co-authored-by: gemini-code-assist[bot] <176961590+gemini-code-assist[bot]@users.noreply.github.com>
@stankiewicz
Copy link
Contributor Author

Run Java PreCommit

@stankiewicz
Copy link
Contributor Author

Run Java_PVR_Prism_Loopback PreCommit

1 similar comment
@stankiewicz
Copy link
Contributor Author

Run Java_PVR_Prism_Loopback PreCommit

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

Projects

None yet

Development

Successfully merging this pull request may close these issues.

1 participant