-
Notifications
You must be signed in to change notification settings - Fork 4.6k
[Drain] Support extensible element metadata propagation in ReduceFnRunner #38230
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
Changes from all commits
Commits
Show all changes
4 commits
Select commit
Hold shift + click to select a range
21b033c
Refactor metadata propagation in ReduceFnRunner to support extensible…
stankiewicz 110e759
Add missing dependency on model:fn-execution to runners:core-java
stankiewicz fd571c9
document asserts due to new state added
stankiewicz 79f7db9
Fix propagation of metadata
stankiewicz File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
87 changes: 87 additions & 0 deletions
87
runners/core-java/src/main/java/org/apache/beam/runners/core/CombinedMetadata.java
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,87 @@ | ||
| /* | ||
| * Licensed to the Apache Software Foundation (ASF) under one | ||
| * or more contributor license agreements. See the NOTICE file | ||
| * distributed with this work for additional information | ||
| * regarding copyright ownership. The ASF licenses this file | ||
| * to you under the Apache License, Version 2.0 (the | ||
| * "License"); you may not use this file except in compliance | ||
| * with the License. You may obtain a copy of the License at | ||
| * | ||
| * http://www.apache.org/licenses/LICENSE-2.0 | ||
| * | ||
| * Unless required by applicable law or agreed to in writing, software | ||
| * distributed under the License is distributed on an "AS IS" BASIS, | ||
| * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
| * See the License for the specific language governing permissions and | ||
| * limitations under the License. | ||
| */ | ||
| package org.apache.beam.runners.core; | ||
|
|
||
| import com.google.auto.value.AutoValue; | ||
| import java.io.IOException; | ||
| import java.io.InputStream; | ||
| import java.io.OutputStream; | ||
| import org.apache.beam.model.fnexecution.v1.BeamFnApi; | ||
| import org.apache.beam.sdk.coders.AtomicCoder; | ||
| import org.apache.beam.sdk.values.CausedByDrain; | ||
|
|
||
| /** | ||
| * Encapsulates metadata that propagates with elements in the pipeline. | ||
| * | ||
| * <p>This metadata is sent along with elements. It currently includes fields like {@link | ||
| * CausedByDrain}, and is designed to be extensible to support future metadata fields such as | ||
| * OpenTelemetry context or CDC (Change Data Capture) kind. | ||
| * | ||
| * <p>The purpose of this class is to group targeted metadata fields together. This makes it easier | ||
| * to define combination strategies (e.g., when accumulating state in {@code ReduceFnRunner}) when | ||
| * multiple elements are merged or grouped, without having to extend method signatures or state | ||
| * handling for every new metadata field. | ||
| */ | ||
| @AutoValue | ||
| public abstract class CombinedMetadata { | ||
| public abstract CausedByDrain causedByDrain(); | ||
|
|
||
| public static CombinedMetadata create(CausedByDrain causedByDrain) { | ||
| return new AutoValue_CombinedMetadata(causedByDrain); | ||
| } | ||
|
|
||
| public static CombinedMetadata createDefault() { | ||
| return create(CausedByDrain.NORMAL); | ||
| } | ||
|
|
||
| public static class Coder extends AtomicCoder<CombinedMetadata> { | ||
| private static final Coder INSTANCE = new Coder(); | ||
|
|
||
| public static Coder of() { | ||
| return INSTANCE; | ||
| } | ||
|
|
||
| @Override | ||
| public void encode(CombinedMetadata value, OutputStream outStream) throws IOException { | ||
| BeamFnApi.Elements.ElementMetadata proto = | ||
| BeamFnApi.Elements.ElementMetadata.newBuilder() | ||
| .setDrain( | ||
| value.causedByDrain() == CausedByDrain.CAUSED_BY_DRAIN | ||
| ? BeamFnApi.Elements.DrainMode.Enum.DRAINING | ||
| : BeamFnApi.Elements.DrainMode.Enum.NOT_DRAINING) | ||
| .build(); | ||
| proto.writeDelimitedTo(outStream); | ||
| } | ||
|
stankiewicz marked this conversation as resolved.
|
||
|
|
||
| @Override | ||
| public CombinedMetadata decode(InputStream inStream) throws IOException { | ||
| BeamFnApi.Elements.ElementMetadata proto = | ||
| BeamFnApi.Elements.ElementMetadata.parseDelimitedFrom(inStream); | ||
| if (proto == null) { | ||
| return CombinedMetadata.createDefault(); | ||
| } | ||
|
|
||
| CausedByDrain causedByDrain = | ||
| proto.getDrain() == BeamFnApi.Elements.DrainMode.Enum.DRAINING | ||
| ? CausedByDrain.CAUSED_BY_DRAIN | ||
| : CausedByDrain.NORMAL; | ||
|
|
||
| return CombinedMetadata.create(causedByDrain); | ||
| } | ||
|
stankiewicz marked this conversation as resolved.
|
||
| } | ||
| } | ||
78 changes: 78 additions & 0 deletions
78
runners/core-java/src/main/java/org/apache/beam/runners/core/CombinedMetadataCombiner.java
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,78 @@ | ||
| /* | ||
| * Licensed to the Apache Software Foundation (ASF) under one | ||
| * or more contributor license agreements. See the NOTICE file | ||
| * distributed with this work for additional information | ||
| * regarding copyright ownership. The ASF licenses this file | ||
| * to you under the Apache License, Version 2.0 (the | ||
| * "License"); you may not use this file except in compliance | ||
| * with the License. You may obtain a copy of the License at | ||
| * | ||
| * http://www.apache.org/licenses/LICENSE-2.0 | ||
| * | ||
| * Unless required by applicable law or agreed to in writing, software | ||
| * distributed under the License is distributed on an "AS IS" BASIS, | ||
| * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
| * See the License for the specific language governing permissions and | ||
| * limitations under the License. | ||
| */ | ||
| package org.apache.beam.runners.core; | ||
|
|
||
| import org.apache.beam.sdk.transforms.Combine.CombineFn; | ||
| import org.apache.beam.sdk.values.CausedByDrain; | ||
|
|
||
| /** Combiner for CombinedMetadata. */ | ||
| class CombinedMetadataCombiner | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Pretty sure this is the same as |
||
| extends CombineFn<CombinedMetadata, CombinedMetadata, CombinedMetadata> { | ||
| private static final CombinedMetadataCombiner INSTANCE = new CombinedMetadataCombiner(); | ||
|
|
||
| public static CombinedMetadataCombiner of() { | ||
| return INSTANCE; | ||
| } | ||
|
|
||
| @Override | ||
| public CombinedMetadata createAccumulator() { | ||
| return CombinedMetadata.create(CausedByDrainCombiner.of().createAccumulator()); | ||
| } | ||
|
|
||
| @Override | ||
| public CombinedMetadata addInput(CombinedMetadata accumulator, CombinedMetadata input) { | ||
| return CombinedMetadata.create( | ||
| CausedByDrainCombiner.of().addInput(accumulator.causedByDrain(), input.causedByDrain())); | ||
| } | ||
|
|
||
| @Override | ||
| public CombinedMetadata mergeAccumulators(Iterable<CombinedMetadata> accumulators) { | ||
| CombinedMetadata result = createAccumulator(); | ||
| for (CombinedMetadata accum : accumulators) { | ||
| result = addInput(result, accum); | ||
| } | ||
| return result; | ||
| } | ||
|
|
||
| @Override | ||
| public CombinedMetadata extractOutput(CombinedMetadata accumulator) { | ||
| return accumulator; | ||
| } | ||
|
|
||
| /** Combiner for CausedByDrain metadata. */ | ||
| static class CausedByDrainCombiner implements MetadataCombiner<CausedByDrain> { | ||
| private static final CausedByDrainCombiner INSTANCE = new CausedByDrainCombiner(); | ||
|
|
||
| public static CausedByDrainCombiner of() { | ||
| return INSTANCE; | ||
| } | ||
|
|
||
| @Override | ||
| public CausedByDrain createAccumulator() { | ||
| return CausedByDrain.NORMAL; | ||
| } | ||
|
|
||
| @Override | ||
| public CausedByDrain addInput(CausedByDrain current, CausedByDrain input) { | ||
| if (current == CausedByDrain.CAUSED_BY_DRAIN || input == CausedByDrain.CAUSED_BY_DRAIN) { | ||
| return CausedByDrain.CAUSED_BY_DRAIN; | ||
| } | ||
| return CausedByDrain.NORMAL; | ||
| } | ||
| } | ||
| } | ||
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
25 changes: 25 additions & 0 deletions
25
runners/core-java/src/main/java/org/apache/beam/runners/core/MetadataCombiner.java
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,25 @@ | ||
| /* | ||
| * Licensed to the Apache Software Foundation (ASF) under one | ||
| * or more contributor license agreements. See the NOTICE file | ||
| * distributed with this work for additional information | ||
| * regarding copyright ownership. The ASF licenses this file | ||
| * to you under the Apache License, Version 2.0 (the | ||
| * "License"); you may not use this file except in compliance | ||
| * with the License. You may obtain a copy of the License at | ||
| * | ||
| * http://www.apache.org/licenses/LICENSE-2.0 | ||
| * | ||
| * Unless required by applicable law or agreed to in writing, software | ||
| * distributed under the License is distributed on an "AS IS" BASIS, | ||
| * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
| * See the License for the specific language governing permissions and | ||
| * limitations under the License. | ||
| */ | ||
| package org.apache.beam.runners.core; | ||
|
|
||
| /** Interface for combining pipeline metadata. */ | ||
| interface MetadataCombiner<T> { | ||
| T createAccumulator(); | ||
|
|
||
| T addInput(T accumulator, T input); | ||
| } |
Oops, something went wrong.
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Is this implemention the same as using
ProtoCoder.of(ElementMetadata.class)?beam/sdks/java/extensions/protobuf/src/main/java/org/apache/beam/sdk/extensions/protobuf/ProtoCoder.java
Line 116 in 16609ed
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
it's almost the same, but as discussed, don't want to bring extension dependency.