Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
1 change: 1 addition & 0 deletions runners/core-java/build.gradle
Original file line number Diff line number Diff line change
Expand Up @@ -42,6 +42,7 @@ dependencies {
implementation project(path: ":model:pipeline", configuration: "shadow")
implementation project(path: ":sdks:java:core", configuration: "shadow")
implementation project(path: ":model:job-management", configuration: "shadow")
implementation project(path: ":model:fn-execution", configuration: "shadow")
implementation library.java.vendored_guava_32_1_2_jre
implementation library.java.joda_time
implementation library.java.vendored_grpc_1_69_0
Expand Down
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);
Copy link
Copy Markdown
Member

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) ?

public static <T extends Message> ProtoCoder<T> of(Class<T> protoMessageClass) {

Copy link
Copy Markdown
Contributor Author

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.

}
Comment thread
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);
}
Comment thread
stankiewicz marked this conversation as resolved.
}
}
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
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

Pretty sure this is the same as BinaryCombineFn default implementations. But since I hate inheritance, I'm happy to have it spelled out here.

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;
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -145,7 +145,13 @@ public <K, InputT> Iterable<WindowedValue<InputT>> filter(
} else {
nonLateElements.add(
WindowedValues.of(
element.getValue(), element.getTimestamp(), window, element.getPaneInfo()));
element.getValue(),
element.getTimestamp(),
window,
element.getPaneInfo(),
element.getRecordId(),
element.getRecordOffset(),
element.causedByDrain()));
}
}
}
Expand Down
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);
}
Loading
Loading