gemini-code-assist[bot] commented on code in PR #38230:
URL: https://github.com/apache/beam/pull/38230#discussion_r3118193737


##########
runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnRunner.java:
##########
@@ -1009,8 +1029,19 @@ private void prefetchOnTrigger(
       ReduceFn<K, InputT, OutputT, W>.Context renamedContext,
       final boolean isFinished,
       boolean isEndOfWindow,
-      CausedByDrain causedByDrain)
+      CombinedMetadata metadata)
       throws Exception {
+    ValueState<CombinedMetadata> metadataState = 
directContext.state().access(METADATA_TAG);
+    CombinedMetadata aggregatedMetadata = metadataState.read();
+    if (aggregatedMetadata == null) {
+      aggregatedMetadata = CombinedMetadata.createDefault();
+    }
+    CombinedMetadata fullyAggregatedMetadata =
+        CombinedMetadataCombiner.of().addInput(aggregatedMetadata, metadata);
+    final CausedByDrain aggregatedCausedByDrain = 
fullyAggregatedMetadata.causedByDrain();
+    if (isFinished) {
+      metadataState.clear();
+    }

Review Comment:
   ![high](https://www.gstatic.com/codereviewagent/high-priority.svg)
   
   In `onTrigger`, the metadata from the trigger/timer is combined with the 
persisted state, but the result is not written back to state if the window 
remains open (`isFinished == false`). This means that any metadata associated 
with the trigger (e.g., a drain signal from a timer) will be lost for 
subsequent panes of the same window. To ensure "stateful tracking" as intended, 
the aggregated metadata should be persisted if the window remains open.
   
   ```suggestion
       ValueState<CombinedMetadata> metadataState = 
directContext.state().access(METADATA_TAG);
       CombinedMetadata aggregatedMetadata = metadataState.read();
       CombinedMetadata fullyAggregatedMetadata =
           CombinedMetadataCombiner.of()
               .addInput(
                   aggregatedMetadata == null ? 
CombinedMetadata.createDefault() : aggregatedMetadata,
                   metadata);
       final CausedByDrain aggregatedCausedByDrain = 
fullyAggregatedMetadata.causedByDrain();
       if (isFinished) {
         metadataState.clear();
       } else if (!fullyAggregatedMetadata.equals(aggregatedMetadata)) {
         metadataState.write(fullyAggregatedMetadata);
       }
   ```



##########
runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnRunner.java:
##########
@@ -590,6 +595,17 @@ private void processElement(Map<W, W> windowToMergeResult, 
WindowedValue<InputT>
               value.getTimestamp(),
               StateStyle.DIRECT,
               value.causedByDrain());
+
+      ValueState<CombinedMetadata> metadataState = 
directContext.state().access(METADATA_TAG);
+      CombinedMetadata currentMetadata = metadataState.read();
+      if (currentMetadata == null) {
+        currentMetadata = CombinedMetadata.createDefault();
+      }
+      CombinedMetadata inputMetadata = 
CombinedMetadata.create(value.causedByDrain());
+      CombinedMetadata newMetadata =
+          CombinedMetadataCombiner.of().addInput(currentMetadata, 
inputMetadata);
+      metadataState.write(newMetadata);

Review Comment:
   ![high](https://www.gstatic.com/codereviewagent/high-priority.svg)
   
   This block introduces a state read and write for every single element 
processed by `ReduceFnRunner`. This is a significant performance regression, 
especially for high-throughput pipelines where `causedByDrain` is typically 
`NORMAL`. Additionally, this logic is currently executed *before* checking if 
the window is closed (line 609), meaning state is updated even for elements 
that will be dropped. 
   
   Consider optimizing this to avoid state access in the common case and moving 
it after the `isClosed` check.
   
   ```suggestion
         if (value.causedByDrain() == CausedByDrain.CAUSED_BY_DRAIN) {
           ValueState<CombinedMetadata> metadataState = 
directContext.state().access(METADATA_TAG);
           CombinedMetadata currentMetadata = metadataState.read();
           if (currentMetadata == null || currentMetadata.causedByDrain() != 
CausedByDrain.CAUSED_BY_DRAIN) {
             
metadataState.write(CombinedMetadata.create(CausedByDrain.CAUSED_BY_DRAIN));
           }
         }
   ```



##########
runners/core-java/src/main/java/org/apache/beam/runners/core/CombinedMetadata.java:
##########
@@ -0,0 +1,94 @@
+/*
+ * 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.coders.ByteArrayCoder;
+import org.apache.beam.sdk.coders.NullableCoder;
+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 {
+      if (value == null) {
+        NullableCoder.of(ByteArrayCoder.of()).encode(null, outStream);
+        return;
+      }
+      BeamFnApi.Elements.ElementMetadata.Builder builder =
+          BeamFnApi.Elements.ElementMetadata.newBuilder();
+      builder.setDrain(
+          value.causedByDrain() == CausedByDrain.CAUSED_BY_DRAIN
+              ? BeamFnApi.Elements.DrainMode.Enum.DRAINING
+              : BeamFnApi.Elements.DrainMode.Enum.NOT_DRAINING);
+
+      
NullableCoder.of(ByteArrayCoder.of()).encode(builder.build().toByteArray(), 
outStream);

Review Comment:
   ![medium](https://www.gstatic.com/codereviewagent/medium-priority.svg)
   
   The `Coder` implementation uses `NullableCoder.of(ByteArrayCoder.of())` to 
wrap the serialized proto bytes. This adds unnecessary overhead (extra bytes 
for nullability and length prefixing) since `AtomicCoder` can handle the 
serialization directly. Furthermore, `ByteArrayCoder` is redundant if you are 
already managing the byte array from the proto. Consider simplifying the coder 
to write the proto bytes directly to the `OutputStream` with a simple length 
prefix if necessary.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to