[BEAM-7233, BEAM-6868] Create an implementation for a bundle finalizer that stores bundle finalization requests in memory. (#10074)

This simplifies the interface for bundle finalization handlers to only need to know about the bundle id.
diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/BundleFinalizationHandler.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/BundleFinalizationHandler.java
index 849663b..646cdf9 100644
--- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/BundleFinalizationHandler.java
+++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/BundleFinalizationHandler.java
@@ -17,8 +17,6 @@
  */
 package org.apache.beam.runners.fnexecution.control;
 
-import org.apache.beam.model.fnexecution.v1.BeamFnApi.ProcessBundleResponse;
-
 /**
  * A handler for the runner when a finalization request has been received.
  *
@@ -29,5 +27,12 @@
  * details.
  */
 public interface BundleFinalizationHandler {
-  void requestsFinalization(ProcessBundleResponse response);
+  /**
+   * This callback is invoked whenever an inflight bundle that is being processed requests
+   * finalization.
+   *
+   * <p>The runner is responsible for invoking bundle finalization when the output of the bundle has
+   * been durably persisted.
+   */
+  void requestsFinalization(String bundleId);
 }
diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/BundleFinalizationHandlers.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/BundleFinalizationHandlers.java
new file mode 100644
index 0000000..8a7a4e8
--- /dev/null
+++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/BundleFinalizationHandlers.java
@@ -0,0 +1,65 @@
+/*
+ * 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.fnexecution.control;
+
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.beam.model.fnexecution.v1.BeamFnApi.FinalizeBundleRequest;
+import org.apache.beam.model.fnexecution.v1.BeamFnApi.InstructionRequest;
+
+/** Utility methods for creating {@link BundleFinalizationHandler}s. */
+public class BundleFinalizationHandlers {
+
+  /**
+   * A bundle finalizer that stores all bundle finalization requests in memory. After the runner
+   * durably persists the output, the runner is responsible for invoking {@link
+   * InMemoryFinalizer#finalizeAllOutstandingBundles()}.
+   */
+  public static InMemoryFinalizer inMemoryFinalizer(InstructionRequestHandler fnApiControlClient) {
+    return new InMemoryFinalizer(fnApiControlClient);
+  }
+
+  /** See {@link #inMemoryFinalizer(InstructionRequestHandler)} for details. */
+  public static class InMemoryFinalizer implements BundleFinalizationHandler {
+    private final InstructionRequestHandler fnApiControlClient;
+    private final List<String> bundleIds;
+
+    private InMemoryFinalizer(InstructionRequestHandler fnApiControlClient) {
+      this.fnApiControlClient = fnApiControlClient;
+      this.bundleIds = new ArrayList<>();
+    }
+
+    /** All finalization requests will be sent without waiting for the responses. */
+    public synchronized void finalizeAllOutstandingBundles() {
+      for (String bundleId : bundleIds) {
+        InstructionRequest request =
+            InstructionRequest.newBuilder()
+                .setFinalizeBundle(
+                    FinalizeBundleRequest.newBuilder().setInstructionId(bundleId).build())
+                .build();
+        fnApiControlClient.handle(request);
+      }
+      bundleIds.clear();
+    }
+
+    @Override
+    public synchronized void requestsFinalization(String bundleId) {
+      bundleIds.add(bundleId);
+    }
+  }
+}
diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/SdkHarnessClient.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/SdkHarnessClient.java
index 86232dd..2799e58 100644
--- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/SdkHarnessClient.java
+++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/SdkHarnessClient.java
@@ -146,7 +146,7 @@
                     "The %s does not have a registered bundle checkpoint handler.",
                     ActiveBundle.class.getSimpleName()));
           },
-          request -> {
+          bundleId -> {
             throw new UnsupportedOperationException(
                 String.format(
                     "The %s does not have a registered bundle finalization handler.",
@@ -313,7 +313,7 @@
             checkpointHandler.onCheckpoint(completedResponse);
           }
           if (completedResponse.getRequiresFinalization()) {
-            finalizationHandler.requestsFinalization(completedResponse);
+            finalizationHandler.requestsFinalization(bundleId);
           }
         } else {
           // TODO: [BEAM-3962] Handle aborting the bundle being processed.
diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/BundleFinalizationHandlersTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/BundleFinalizationHandlersTest.java
new file mode 100644
index 0000000..9494104
--- /dev/null
+++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/BundleFinalizationHandlersTest.java
@@ -0,0 +1,56 @@
+/*
+ * 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.fnexecution.control;
+
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.verifyZeroInteractions;
+
+import org.apache.beam.model.fnexecution.v1.BeamFnApi.FinalizeBundleRequest;
+import org.apache.beam.model.fnexecution.v1.BeamFnApi.InstructionRequest;
+import org.apache.beam.runners.fnexecution.control.BundleFinalizationHandlers.InMemoryFinalizer;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/** Tests for {@link BundleFinalizationHandlers}. */
+@RunWith(JUnit4.class)
+public class BundleFinalizationHandlersTest {
+  @Test
+  public void testInMemoryFinalizer() {
+    InstructionRequestHandler mockHandler = mock(InstructionRequestHandler.class);
+    InMemoryFinalizer finalizer = BundleFinalizationHandlers.inMemoryFinalizer(mockHandler);
+
+    finalizer.finalizeAllOutstandingBundles();
+    verifyZeroInteractions(mockHandler);
+
+    finalizer.requestsFinalization("A");
+    finalizer.requestsFinalization("B");
+    verifyZeroInteractions(mockHandler);
+
+    finalizer.finalizeAllOutstandingBundles();
+    verify(mockHandler).handle(requestFor("A"));
+    verify(mockHandler).handle(requestFor("B"));
+  }
+
+  private static InstructionRequest requestFor(String bundleId) {
+    return InstructionRequest.newBuilder()
+        .setFinalizeBundle(FinalizeBundleRequest.newBuilder().setInstructionId(bundleId).build())
+        .build();
+  }
+}
diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/SdkHarnessClientTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/SdkHarnessClientTest.java
index 089c8d1..ef0b2ac 100644
--- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/SdkHarnessClientTest.java
+++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/SdkHarnessClientTest.java
@@ -627,8 +627,6 @@
 
   @Test
   public void testBundleCheckpointCallback() throws Exception {
-    Exception testException = new Exception();
-
     InboundDataClient mockOutputReceiver = mock(InboundDataClient.class);
     CloseableFnDataReceiver mockInputSender = mock(CloseableFnDataReceiver.class);
 
@@ -658,8 +656,6 @@
         ProcessBundleResponse.newBuilder()
             .addResidualRoots(DelayedBundleApplication.getDefaultInstance())
             .build();
-    ArrayList<ProcessBundleResponse> checkpoints = new ArrayList<>();
-
     try (ActiveBundle activeBundle =
         processor.newBundle(
             ImmutableMap.of(SDK_GRPC_WRITE_TRANSFORM, mockRemoteOutputReceiver),
@@ -680,8 +676,6 @@
 
   @Test
   public void testBundleFinalizationCallback() throws Exception {
-    Exception testException = new Exception();
-
     InboundDataClient mockOutputReceiver = mock(InboundDataClient.class);
     CloseableFnDataReceiver mockInputSender = mock(CloseableFnDataReceiver.class);
 
@@ -709,8 +703,7 @@
 
     ProcessBundleResponse response =
         ProcessBundleResponse.newBuilder().setRequiresFinalization(true).build();
-    ArrayList<ProcessBundleResponse> checkpoints = new ArrayList<>();
-
+    String bundleId;
     try (ActiveBundle activeBundle =
         processor.newBundle(
             ImmutableMap.of(SDK_GRPC_WRITE_TRANSFORM, mockRemoteOutputReceiver),
@@ -720,12 +713,13 @@
             mockProgressHandler,
             mockCheckpointHandler,
             mockFinalizationHandler)) {
+      bundleId = activeBundle.getId();
       processBundleResponseFuture.complete(
           InstructionResponse.newBuilder().setProcessBundle(response).build());
     }
 
     verify(mockProgressHandler).onCompleted(response);
-    verify(mockFinalizationHandler).requestsFinalization(response);
+    verify(mockFinalizationHandler).requestsFinalization(bundleId);
     verifyZeroInteractions(mockCheckpointHandler);
   }