lukecwik commented on a change in pull request #16439:
URL: https://github.com/apache/beam/pull/16439#discussion_r799069555



##########
File path: 
sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/data/BeamFnDataOutboundAggregator.java
##########
@@ -0,0 +1,372 @@
+/*
+ * 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.sdk.fn.data;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.LongAdder;
+import java.util.function.Supplier;
+import org.apache.beam.model.fnexecution.v1.BeamFnApi;
+import org.apache.beam.model.fnexecution.v1.BeamFnApi.Elements;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.options.ExperimentalOptions;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.vendor.grpc.v1p43p2.com.google.protobuf.ByteString;
+import org.apache.beam.vendor.grpc.v1p43p2.io.grpc.stub.StreamObserver;
+import 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
+import 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * An outbound data buffering aggregator with size-based buffer and time-based 
buffer if
+ * corresponding options are set.
+ *
+ * <p>The default size-based buffer threshold can be overridden by specifying 
the experiment {@code
+ * data_buffer_size_limit=<bytes>}
+ *
+ * <p>The default time-based buffer threshold can be overridden by specifying 
the experiment {@code
+ * data_buffer_time_limit_ms=<milliseconds>}
+ */
+@SuppressWarnings({
+  "nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
+})
+public class BeamFnDataOutboundAggregator {
+
+  public static final String DATA_BUFFER_SIZE_LIMIT = 
"data_buffer_size_limit=";
+  public static final int DEFAULT_BUFFER_LIMIT_BYTES = 1_000_000;
+  public static final String DATA_BUFFER_TIME_LIMIT_MS = 
"data_buffer_time_limit_ms=";
+  public static final long DEFAULT_BUFFER_LIMIT_TIME_MS = -1L;
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(BeamFnDataOutboundAggregator.class);
+  private final int sizeLimit;
+  private final long timeLimit;
+  private final Supplier<String> processBundleRequestIdSupplier;
+  private final Map<String, Receiver<?>> outputDataReceivers;
+  private final Map<TimerEndpoint, Receiver<?>> outputTimersReceivers;
+  private final StreamObserver<Elements> outboundObserver;
+  @VisibleForTesting ScheduledFuture<?> flushFuture;
+  private final LongAdder totalByteCounter;
+  private final Object flushLock;
+
+  public BeamFnDataOutboundAggregator(
+      PipelineOptions options,
+      Supplier<String> processBundleRequestIdSupplier,
+      StreamObserver<BeamFnApi.Elements> outboundObserver) {
+    this.sizeLimit = getSizeLimit(options);
+    this.timeLimit = getTimeLimit(options);
+    this.outputDataReceivers = new HashMap<>();
+    this.outputTimersReceivers = new HashMap<>();
+    this.outboundObserver = outboundObserver;
+    this.processBundleRequestIdSupplier = processBundleRequestIdSupplier;
+    this.totalByteCounter = new LongAdder();
+    this.flushLock = new Object();
+  }
+
+  /** Starts the flushing daemon thread if data_buffer_time_limit_ms is set. */
+  public void startFlushThread() {
+    if (timeLimit > 0 && this.flushFuture == null) {
+      this.flushFuture =
+          Executors.newSingleThreadScheduledExecutor(
+                  new ThreadFactoryBuilder()
+                      .setDaemon(true)
+                      .setNameFormat("DataBufferOutboundFlusher-thread")
+                      .build())
+              .scheduleAtFixedRate(
+                  this::periodicFlush, timeLimit, timeLimit, 
TimeUnit.MILLISECONDS);
+    }
+  }
+
+  /**
+   * Register the outbound data logical endpoint, returns the FnDataReceiver 
for processing the
+   * endpoint's outbound data.
+   */
+  public <T> FnDataReceiver<T> registerOutputDataLocation(String pTransformId, 
Coder<T> coder) {
+    if (outputDataReceivers.containsKey(pTransformId)) {
+      throw new IllegalStateException(
+          "Outbound data endpoint already registered for " + pTransformId);
+    }
+    Receiver<T> receiver = new Receiver<>(coder);
+    if (timeLimit > 0) {
+      outputDataReceivers.put(pTransformId, receiver);
+      return data -> {
+        checkFlushThreadException();
+        synchronized (flushLock) {
+          receiver.accept(data);
+          if (totalByteCounter.longValue() > sizeLimit) {
+            flush();
+          }

Review comment:
       in `accept` now.
   ```suggestion
   ```

##########
File path: 
sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/data/BeamFnDataOutboundAggregator.java
##########
@@ -0,0 +1,372 @@
+/*
+ * 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.sdk.fn.data;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.LongAdder;
+import java.util.function.Supplier;
+import org.apache.beam.model.fnexecution.v1.BeamFnApi;
+import org.apache.beam.model.fnexecution.v1.BeamFnApi.Elements;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.options.ExperimentalOptions;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.vendor.grpc.v1p43p2.com.google.protobuf.ByteString;
+import org.apache.beam.vendor.grpc.v1p43p2.io.grpc.stub.StreamObserver;
+import 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
+import 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * An outbound data buffering aggregator with size-based buffer and time-based 
buffer if
+ * corresponding options are set.
+ *
+ * <p>The default size-based buffer threshold can be overridden by specifying 
the experiment {@code
+ * data_buffer_size_limit=<bytes>}
+ *
+ * <p>The default time-based buffer threshold can be overridden by specifying 
the experiment {@code
+ * data_buffer_time_limit_ms=<milliseconds>}
+ */
+@SuppressWarnings({
+  "nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
+})
+public class BeamFnDataOutboundAggregator {
+
+  public static final String DATA_BUFFER_SIZE_LIMIT = 
"data_buffer_size_limit=";
+  public static final int DEFAULT_BUFFER_LIMIT_BYTES = 1_000_000;
+  public static final String DATA_BUFFER_TIME_LIMIT_MS = 
"data_buffer_time_limit_ms=";
+  public static final long DEFAULT_BUFFER_LIMIT_TIME_MS = -1L;
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(BeamFnDataOutboundAggregator.class);
+  private final int sizeLimit;
+  private final long timeLimit;
+  private final Supplier<String> processBundleRequestIdSupplier;
+  private final Map<String, Receiver<?>> outputDataReceivers;
+  private final Map<TimerEndpoint, Receiver<?>> outputTimersReceivers;
+  private final StreamObserver<Elements> outboundObserver;
+  @VisibleForTesting ScheduledFuture<?> flushFuture;
+  private final LongAdder totalByteCounter;
+  private final Object flushLock;
+
+  public BeamFnDataOutboundAggregator(
+      PipelineOptions options,
+      Supplier<String> processBundleRequestIdSupplier,
+      StreamObserver<BeamFnApi.Elements> outboundObserver) {
+    this.sizeLimit = getSizeLimit(options);
+    this.timeLimit = getTimeLimit(options);
+    this.outputDataReceivers = new HashMap<>();
+    this.outputTimersReceivers = new HashMap<>();
+    this.outboundObserver = outboundObserver;
+    this.processBundleRequestIdSupplier = processBundleRequestIdSupplier;
+    this.totalByteCounter = new LongAdder();
+    this.flushLock = new Object();
+  }
+
+  /** Starts the flushing daemon thread if data_buffer_time_limit_ms is set. */
+  public void startFlushThread() {
+    if (timeLimit > 0 && this.flushFuture == null) {
+      this.flushFuture =
+          Executors.newSingleThreadScheduledExecutor(
+                  new ThreadFactoryBuilder()
+                      .setDaemon(true)
+                      .setNameFormat("DataBufferOutboundFlusher-thread")
+                      .build())
+              .scheduleAtFixedRate(
+                  this::periodicFlush, timeLimit, timeLimit, 
TimeUnit.MILLISECONDS);
+    }
+  }
+
+  /**
+   * Register the outbound data logical endpoint, returns the FnDataReceiver 
for processing the
+   * endpoint's outbound data.
+   */
+  public <T> FnDataReceiver<T> registerOutputDataLocation(String pTransformId, 
Coder<T> coder) {
+    if (outputDataReceivers.containsKey(pTransformId)) {
+      throw new IllegalStateException(
+          "Outbound data endpoint already registered for " + pTransformId);
+    }
+    Receiver<T> receiver = new Receiver<>(coder);
+    if (timeLimit > 0) {
+      outputDataReceivers.put(pTransformId, receiver);
+      return data -> {
+        checkFlushThreadException();
+        synchronized (flushLock) {
+          receiver.accept(data);
+          if (totalByteCounter.longValue() > sizeLimit) {
+            flush();
+          }
+        }
+      };
+    }
+    outputDataReceivers.put(pTransformId, receiver);
+    return receiver;
+  }
+
+  /**
+   * Register the outbound timers logical endpoint, returns the FnDataReceiver 
for processing the
+   * endpoint's outbound timers data.
+   */
+  public <T> FnDataReceiver<T> registerOutputTimersLocation(
+      String pTransformId, String timerFamilyId, Coder<T> coder) {
+    TimerEndpoint timerKey = new TimerEndpoint(pTransformId, timerFamilyId);
+    if (outputTimersReceivers.containsKey(timerKey)) {
+      throw new IllegalStateException(
+          "Outbound timers endpoint already registered for " + timerKey);
+    }
+    Receiver<T> receiver = new Receiver<>(coder);
+    if (timeLimit > 0) {
+      outputTimersReceivers.put(timerKey, receiver);
+      return timers -> {
+        checkFlushThreadException();
+        synchronized (flushLock) {
+          receiver.accept(timers);
+        }
+      };
+    }
+    outputTimersReceivers.put(timerKey, receiver);
+    return receiver;
+  }
+
+  public void flush() throws IOException {
+    Elements.Builder elements = convertBufferForTransmission();
+    if (elements.getDataCount() > 0 || elements.getTimersCount() > 0) {
+      outboundObserver.onNext(elements.build());
+    }
+  }
+
+  /**
+   * Closes the streams for all registered outbound endpoints. Should be 
called at the end of each
+   * bundle.
+   */
+  public void sendBufferedDataAndFinishOutboundStreams() {
+    if (totalByteCounter.longValue() == 0
+        && outputTimersReceivers.isEmpty()
+        && outputDataReceivers.isEmpty()) {
+      return;
+    }
+    Elements.Builder bufferedElements = convertBufferForTransmission();
+    LOG.debug(
+        "Closing streams for outbound endpoints {} and {}.",
+        outputDataReceivers.keySet(),
+        outputTimersReceivers.keySet());
+    LOG.debug(
+        "Sent outbound data : {}, outbound timers : {}.",
+        outputDataReceivers,
+        outputTimersReceivers);
+    for (String pTransformID : outputDataReceivers.keySet()) {
+      bufferedElements
+          .addDataBuilder()
+          .setInstructionId(processBundleRequestIdSupplier.get())
+          .setTransformId(pTransformID)
+          .setIsLast(true);
+    }
+    for (TimerEndpoint timerKey : outputTimersReceivers.keySet()) {
+      bufferedElements
+          .addTimersBuilder()
+          .setInstructionId(processBundleRequestIdSupplier.get())
+          .setTransformId(timerKey.pTransformId)
+          .setTimerFamilyId(timerKey.timerFamilyId)
+          .setIsLast(true);
+    }
+    outboundObserver.onNext(bufferedElements.build());
+    outputDataReceivers.values().forEach(Receiver::reset);
+    outputTimersReceivers.values().forEach(Receiver::reset);
+  }
+
+  private Elements.Builder convertBufferForTransmission() {
+    Elements.Builder bufferedElements = Elements.newBuilder();
+    for (Map.Entry<String, Receiver<?>> entry : 
outputDataReceivers.entrySet()) {
+      if (entry.getValue().getOutput().size() == 0) {
+        continue;
+      }
+      ByteString bytes = entry.getValue().getOutput().toByteString();
+      bufferedElements
+          .addDataBuilder()
+          .setInstructionId(processBundleRequestIdSupplier.get())
+          .setTransformId(entry.getKey())
+          .setData(bytes);
+      entry.getValue().reset();
+    }
+    for (Map.Entry<TimerEndpoint, Receiver<?>> entry : 
outputTimersReceivers.entrySet()) {
+      if (entry.getValue().getOutput().size() == 0) {
+        continue;
+      }
+      ByteString bytes = entry.getValue().getOutput().toByteString();
+      bufferedElements
+          .addTimersBuilder()
+          .setInstructionId(processBundleRequestIdSupplier.get())
+          .setTransformId(entry.getKey().pTransformId)
+          .setTimerFamilyId(entry.getKey().timerFamilyId)
+          .setTimers(bytes);
+      entry.getValue().reset();
+    }
+    totalByteCounter.reset();

Review comment:
       why are we resetting the totalByteCounter here?

##########
File path: 
sdks/java/harness/src/main/java/org/apache/beam/fn/harness/BeamFnDataWriteRunner.java
##########
@@ -66,93 +60,42 @@
   }
 
   /** A factory for {@link BeamFnDataWriteRunner}s. */
-  static class Factory<InputT> implements 
PTransformRunnerFactory<BeamFnDataWriteRunner<InputT>> {
+  static class Factory<InputT> implements 
PTransformRunnerFactory<BeamFnDataWriteRunner> {
 
     @Override
-    public BeamFnDataWriteRunner<InputT> createRunnerForPTransform(Context 
context)
-        throws IOException {
-
-      BeamFnDataWriteRunner<InputT> runner =
-          new BeamFnDataWriteRunner<>(
-              context.getBundleCacheSupplier(),
-              context.getPTransformId(),
-              context.getPTransform(),
-              context.getProcessBundleInstructionIdSupplier(),
-              context.getCoders(),
-              context.getBeamFnDataClient(),
-              context.getBeamFnStateClient());
-      context.addStartBundleFunction(runner::registerForOutput);
+    public BeamFnDataWriteRunner createRunnerForPTransform(Context context) 
throws IOException {
+
+      RemoteGrpcPort port = 
RemoteGrpcPortWrite.fromPTransform(context.getPTransform()).getPort();
+      RehydratedComponents components =
+          RehydratedComponents.forComponents(
+              
Components.newBuilder().putAllCoders(context.getCoders()).build());
+      Coder<WindowedValue<InputT>> coder =
+          (Coder<WindowedValue<InputT>>)
+              CoderTranslation.fromProto(
+                  context.getCoders().get(port.getCoderId()),
+                  components,
+                  new StateBackedIterableTranslationContext() {
+                    @Override
+                    public Supplier<Cache<?, ?>> getCache() {
+                      return context.getBundleCacheSupplier();
+                    }
+
+                    @Override
+                    public BeamFnStateClient getStateClient() {
+                      return context.getBeamFnStateClient();
+                    }
+
+                    @Override
+                    public Supplier<String> getCurrentInstructionId() {
+                      return context.getProcessBundleInstructionIdSupplier();
+                    }
+                  });
       context.addPCollectionConsumer(
           getOnlyElement(context.getPTransform().getInputsMap().values()),
-          (FnDataReceiver) (FnDataReceiver<WindowedValue<InputT>>) 
runner::consume,
-          ((WindowedValueCoder<InputT>) runner.coder).getValueCoder());
+          context.addOutgoingDataEndpoint(port.getApiServiceDescriptor(), 
coder),
+          ((WindowedValueCoder<InputT>) coder).getValueCoder());
 
-      context.addFinishBundleFunction(runner::close);
-      return runner;
+      return new BeamFnDataWriteRunner();
     }
   }
-
-  private final Endpoints.ApiServiceDescriptor apiServiceDescriptor;

Review comment:
       This is sweet that we can now delete this whole implementation.

##########
File path: 
sdks/java/harness/src/test/java/org/apache/beam/fn/harness/FnApiDoFnRunnerTest.java
##########
@@ -3729,6 +3771,7 @@ public void testDoFnSkewAllowed() throws Exception {
               
.pCollections(pProto.getComponentsOrBuilder().getPcollectionsMap())
               .coders(pProto.getComponents().getCodersMap())
               
.windowingStrategies(pProto.getComponents().getWindowingStrategiesMap())
+              .outboundAggregators(new HashMap<>())

Review comment:
       Isn't this the default in PTransformRunnerFactoryTestContext and 
unnecessary?
   ```suggestion
   ```

##########
File path: 
sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/data/BeamFnDataOutboundAggregator.java
##########
@@ -0,0 +1,372 @@
+/*
+ * 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.sdk.fn.data;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.LongAdder;
+import java.util.function.Supplier;
+import org.apache.beam.model.fnexecution.v1.BeamFnApi;
+import org.apache.beam.model.fnexecution.v1.BeamFnApi.Elements;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.options.ExperimentalOptions;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.vendor.grpc.v1p43p2.com.google.protobuf.ByteString;
+import org.apache.beam.vendor.grpc.v1p43p2.io.grpc.stub.StreamObserver;
+import 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
+import 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * An outbound data buffering aggregator with size-based buffer and time-based 
buffer if
+ * corresponding options are set.
+ *
+ * <p>The default size-based buffer threshold can be overridden by specifying 
the experiment {@code
+ * data_buffer_size_limit=<bytes>}
+ *
+ * <p>The default time-based buffer threshold can be overridden by specifying 
the experiment {@code
+ * data_buffer_time_limit_ms=<milliseconds>}
+ */
+@SuppressWarnings({
+  "nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
+})
+public class BeamFnDataOutboundAggregator {
+
+  public static final String DATA_BUFFER_SIZE_LIMIT = 
"data_buffer_size_limit=";
+  public static final int DEFAULT_BUFFER_LIMIT_BYTES = 1_000_000;
+  public static final String DATA_BUFFER_TIME_LIMIT_MS = 
"data_buffer_time_limit_ms=";
+  public static final long DEFAULT_BUFFER_LIMIT_TIME_MS = -1L;
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(BeamFnDataOutboundAggregator.class);
+  private final int sizeLimit;
+  private final long timeLimit;
+  private final Supplier<String> processBundleRequestIdSupplier;
+  private final Map<String, Receiver<?>> outputDataReceivers;
+  private final Map<TimerEndpoint, Receiver<?>> outputTimersReceivers;
+  private final StreamObserver<Elements> outboundObserver;
+  @VisibleForTesting ScheduledFuture<?> flushFuture;
+  private final LongAdder totalByteCounter;
+  private final Object flushLock;
+
+  public BeamFnDataOutboundAggregator(
+      PipelineOptions options,
+      Supplier<String> processBundleRequestIdSupplier,
+      StreamObserver<BeamFnApi.Elements> outboundObserver) {
+    this.sizeLimit = getSizeLimit(options);
+    this.timeLimit = getTimeLimit(options);
+    this.outputDataReceivers = new HashMap<>();
+    this.outputTimersReceivers = new HashMap<>();
+    this.outboundObserver = outboundObserver;
+    this.processBundleRequestIdSupplier = processBundleRequestIdSupplier;
+    this.totalByteCounter = new LongAdder();
+    this.flushLock = new Object();
+  }
+
+  /** Starts the flushing daemon thread if data_buffer_time_limit_ms is set. */
+  public void startFlushThread() {
+    if (timeLimit > 0 && this.flushFuture == null) {
+      this.flushFuture =
+          Executors.newSingleThreadScheduledExecutor(
+                  new ThreadFactoryBuilder()
+                      .setDaemon(true)
+                      .setNameFormat("DataBufferOutboundFlusher-thread")
+                      .build())
+              .scheduleAtFixedRate(
+                  this::periodicFlush, timeLimit, timeLimit, 
TimeUnit.MILLISECONDS);
+    }
+  }
+
+  /**
+   * Register the outbound data logical endpoint, returns the FnDataReceiver 
for processing the
+   * endpoint's outbound data.
+   */
+  public <T> FnDataReceiver<T> registerOutputDataLocation(String pTransformId, 
Coder<T> coder) {
+    if (outputDataReceivers.containsKey(pTransformId)) {
+      throw new IllegalStateException(
+          "Outbound data endpoint already registered for " + pTransformId);
+    }
+    Receiver<T> receiver = new Receiver<>(coder);
+    if (timeLimit > 0) {
+      outputDataReceivers.put(pTransformId, receiver);
+      return data -> {
+        checkFlushThreadException();
+        synchronized (flushLock) {
+          receiver.accept(data);
+          if (totalByteCounter.longValue() > sizeLimit) {
+            flush();
+          }
+        }
+      };
+    }
+    outputDataReceivers.put(pTransformId, receiver);
+    return receiver;
+  }
+
+  /**
+   * Register the outbound timers logical endpoint, returns the FnDataReceiver 
for processing the
+   * endpoint's outbound timers data.
+   */
+  public <T> FnDataReceiver<T> registerOutputTimersLocation(
+      String pTransformId, String timerFamilyId, Coder<T> coder) {
+    TimerEndpoint timerKey = new TimerEndpoint(pTransformId, timerFamilyId);
+    if (outputTimersReceivers.containsKey(timerKey)) {
+      throw new IllegalStateException(
+          "Outbound timers endpoint already registered for " + timerKey);
+    }
+    Receiver<T> receiver = new Receiver<>(coder);
+    if (timeLimit > 0) {
+      outputTimersReceivers.put(timerKey, receiver);
+      return timers -> {
+        checkFlushThreadException();
+        synchronized (flushLock) {
+          receiver.accept(timers);
+        }
+      };
+    }
+    outputTimersReceivers.put(timerKey, receiver);
+    return receiver;
+  }
+
+  public void flush() throws IOException {
+    Elements.Builder elements = convertBufferForTransmission();
+    if (elements.getDataCount() > 0 || elements.getTimersCount() > 0) {
+      outboundObserver.onNext(elements.build());
+    }
+  }
+
+  /**
+   * Closes the streams for all registered outbound endpoints. Should be 
called at the end of each
+   * bundle.
+   */
+  public void sendBufferedDataAndFinishOutboundStreams() {
+    if (totalByteCounter.longValue() == 0
+        && outputTimersReceivers.isEmpty()
+        && outputDataReceivers.isEmpty()) {

Review comment:
       I don't think the byte count will ever be non zero if nothing is 
registered.
   ```suggestion
       if (outputTimersReceivers.isEmpty()
           && outputDataReceivers.isEmpty()) {
   ```

##########
File path: 
sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/data/BeamFnDataOutboundAggregator.java
##########
@@ -0,0 +1,372 @@
+/*
+ * 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.sdk.fn.data;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.LongAdder;
+import java.util.function.Supplier;
+import org.apache.beam.model.fnexecution.v1.BeamFnApi;
+import org.apache.beam.model.fnexecution.v1.BeamFnApi.Elements;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.options.ExperimentalOptions;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.vendor.grpc.v1p43p2.com.google.protobuf.ByteString;
+import org.apache.beam.vendor.grpc.v1p43p2.io.grpc.stub.StreamObserver;
+import 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
+import 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * An outbound data buffering aggregator with size-based buffer and time-based 
buffer if
+ * corresponding options are set.
+ *
+ * <p>The default size-based buffer threshold can be overridden by specifying 
the experiment {@code
+ * data_buffer_size_limit=<bytes>}
+ *
+ * <p>The default time-based buffer threshold can be overridden by specifying 
the experiment {@code
+ * data_buffer_time_limit_ms=<milliseconds>}
+ */
+@SuppressWarnings({
+  "nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
+})
+public class BeamFnDataOutboundAggregator {
+
+  public static final String DATA_BUFFER_SIZE_LIMIT = 
"data_buffer_size_limit=";
+  public static final int DEFAULT_BUFFER_LIMIT_BYTES = 1_000_000;
+  public static final String DATA_BUFFER_TIME_LIMIT_MS = 
"data_buffer_time_limit_ms=";
+  public static final long DEFAULT_BUFFER_LIMIT_TIME_MS = -1L;
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(BeamFnDataOutboundAggregator.class);
+  private final int sizeLimit;
+  private final long timeLimit;
+  private final Supplier<String> processBundleRequestIdSupplier;
+  private final Map<String, Receiver<?>> outputDataReceivers;
+  private final Map<TimerEndpoint, Receiver<?>> outputTimersReceivers;
+  private final StreamObserver<Elements> outboundObserver;
+  @VisibleForTesting ScheduledFuture<?> flushFuture;
+  private final LongAdder totalByteCounter;
+  private final Object flushLock;
+
+  public BeamFnDataOutboundAggregator(
+      PipelineOptions options,
+      Supplier<String> processBundleRequestIdSupplier,
+      StreamObserver<BeamFnApi.Elements> outboundObserver) {
+    this.sizeLimit = getSizeLimit(options);
+    this.timeLimit = getTimeLimit(options);
+    this.outputDataReceivers = new HashMap<>();
+    this.outputTimersReceivers = new HashMap<>();
+    this.outboundObserver = outboundObserver;
+    this.processBundleRequestIdSupplier = processBundleRequestIdSupplier;
+    this.totalByteCounter = new LongAdder();
+    this.flushLock = new Object();
+  }
+
+  /** Starts the flushing daemon thread if data_buffer_time_limit_ms is set. */
+  public void startFlushThread() {
+    if (timeLimit > 0 && this.flushFuture == null) {
+      this.flushFuture =
+          Executors.newSingleThreadScheduledExecutor(
+                  new ThreadFactoryBuilder()
+                      .setDaemon(true)
+                      .setNameFormat("DataBufferOutboundFlusher-thread")
+                      .build())
+              .scheduleAtFixedRate(
+                  this::periodicFlush, timeLimit, timeLimit, 
TimeUnit.MILLISECONDS);
+    }
+  }
+
+  /**
+   * Register the outbound data logical endpoint, returns the FnDataReceiver 
for processing the
+   * endpoint's outbound data.
+   */
+  public <T> FnDataReceiver<T> registerOutputDataLocation(String pTransformId, 
Coder<T> coder) {
+    if (outputDataReceivers.containsKey(pTransformId)) {
+      throw new IllegalStateException(
+          "Outbound data endpoint already registered for " + pTransformId);
+    }
+    Receiver<T> receiver = new Receiver<>(coder);
+    if (timeLimit > 0) {
+      outputDataReceivers.put(pTransformId, receiver);
+      return data -> {
+        checkFlushThreadException();
+        synchronized (flushLock) {
+          receiver.accept(data);
+          if (totalByteCounter.longValue() > sizeLimit) {
+            flush();
+          }
+        }
+      };
+    }
+    outputDataReceivers.put(pTransformId, receiver);
+    return receiver;
+  }
+
+  /**
+   * Register the outbound timers logical endpoint, returns the FnDataReceiver 
for processing the
+   * endpoint's outbound timers data.
+   */
+  public <T> FnDataReceiver<T> registerOutputTimersLocation(
+      String pTransformId, String timerFamilyId, Coder<T> coder) {
+    TimerEndpoint timerKey = new TimerEndpoint(pTransformId, timerFamilyId);
+    if (outputTimersReceivers.containsKey(timerKey)) {
+      throw new IllegalStateException(
+          "Outbound timers endpoint already registered for " + timerKey);
+    }
+    Receiver<T> receiver = new Receiver<>(coder);
+    if (timeLimit > 0) {
+      outputTimersReceivers.put(timerKey, receiver);
+      return timers -> {
+        checkFlushThreadException();
+        synchronized (flushLock) {
+          receiver.accept(timers);
+        }
+      };
+    }
+    outputTimersReceivers.put(timerKey, receiver);
+    return receiver;
+  }
+
+  public void flush() throws IOException {
+    Elements.Builder elements = convertBufferForTransmission();
+    if (elements.getDataCount() > 0 || elements.getTimersCount() > 0) {
+      outboundObserver.onNext(elements.build());
+    }
+  }
+
+  /**
+   * Closes the streams for all registered outbound endpoints. Should be 
called at the end of each
+   * bundle.
+   */
+  public void sendBufferedDataAndFinishOutboundStreams() {
+    if (totalByteCounter.longValue() == 0
+        && outputTimersReceivers.isEmpty()
+        && outputDataReceivers.isEmpty()) {
+      return;
+    }
+    Elements.Builder bufferedElements = convertBufferForTransmission();
+    LOG.debug(
+        "Closing streams for outbound endpoints {} and {}.",
+        outputDataReceivers.keySet(),
+        outputTimersReceivers.keySet());
+    LOG.debug(
+        "Sent outbound data : {}, outbound timers : {}.",
+        outputDataReceivers,
+        outputTimersReceivers);
+    for (String pTransformID : outputDataReceivers.keySet()) {
+      bufferedElements
+          .addDataBuilder()
+          .setInstructionId(processBundleRequestIdSupplier.get())
+          .setTransformId(pTransformID)
+          .setIsLast(true);
+    }
+    for (TimerEndpoint timerKey : outputTimersReceivers.keySet()) {
+      bufferedElements
+          .addTimersBuilder()
+          .setInstructionId(processBundleRequestIdSupplier.get())
+          .setTransformId(timerKey.pTransformId)
+          .setTimerFamilyId(timerKey.timerFamilyId)
+          .setIsLast(true);
+    }
+    outboundObserver.onNext(bufferedElements.build());
+    outputDataReceivers.values().forEach(Receiver::reset);
+    outputTimersReceivers.values().forEach(Receiver::reset);
+  }
+
+  private Elements.Builder convertBufferForTransmission() {
+    Elements.Builder bufferedElements = Elements.newBuilder();
+    for (Map.Entry<String, Receiver<?>> entry : 
outputDataReceivers.entrySet()) {
+      if (entry.getValue().getOutput().size() == 0) {
+        continue;
+      }
+      ByteString bytes = entry.getValue().getOutput().toByteString();
+      bufferedElements
+          .addDataBuilder()
+          .setInstructionId(processBundleRequestIdSupplier.get())
+          .setTransformId(entry.getKey())
+          .setData(bytes);
+      entry.getValue().reset();
+    }
+    for (Map.Entry<TimerEndpoint, Receiver<?>> entry : 
outputTimersReceivers.entrySet()) {
+      if (entry.getValue().getOutput().size() == 0) {
+        continue;
+      }
+      ByteString bytes = entry.getValue().getOutput().toByteString();
+      bufferedElements
+          .addTimersBuilder()
+          .setInstructionId(processBundleRequestIdSupplier.get())
+          .setTransformId(entry.getKey().pTransformId)
+          .setTimerFamilyId(entry.getKey().timerFamilyId)
+          .setTimers(bytes);
+      entry.getValue().reset();
+    }
+    totalByteCounter.reset();
+    return bufferedElements;
+  }
+
+  private void periodicFlush() {
+    try {
+      synchronized (flushLock) {
+        flush();
+      }
+    } catch (Throwable t) {
+      throw new RuntimeException(t);
+    }
+  }
+
+  /** Check if the flush thread failed with an exception. */
+  private void checkFlushThreadException() throws IOException {
+    if (timeLimit > 0 && flushFuture.isDone()) {
+      try {
+        flushFuture.get();
+        throw new IOException("Periodic flushing thread finished 
unexpectedly.");
+      } catch (ExecutionException ee) {
+        unwrapExecutionException(ee);
+      } catch (CancellationException ce) {
+        throw new IOException(ce);
+      } catch (InterruptedException ie) {
+        Thread.currentThread().interrupt();
+        throw new IOException(ie);
+      }
+    }
+  }
+
+  private void unwrapExecutionException(ExecutionException ee) throws 
IOException {
+    // the cause is always RuntimeException
+    RuntimeException re = (RuntimeException) ee.getCause();
+    if (re.getCause() instanceof IOException) {
+      throw (IOException) re.getCause();
+    } else {
+      throw new IOException(re.getCause());
+    }
+  }
+
+  private static int getSizeLimit(PipelineOptions options) {
+    List<String> experiments = 
options.as(ExperimentalOptions.class).getExperiments();
+    for (String experiment : experiments == null ? 
Collections.<String>emptyList() : experiments) {
+      if (experiment.startsWith(DATA_BUFFER_SIZE_LIMIT)) {
+        return 
Integer.parseInt(experiment.substring(DATA_BUFFER_SIZE_LIMIT.length()));
+      }
+    }
+    return DEFAULT_BUFFER_LIMIT_BYTES;
+  }
+
+  private static long getTimeLimit(PipelineOptions options) {
+    List<String> experiments = 
options.as(ExperimentalOptions.class).getExperiments();
+    for (String experiment : experiments == null ? 
Collections.<String>emptyList() : experiments) {
+      if (experiment.startsWith(DATA_BUFFER_TIME_LIMIT_MS)) {
+        return 
Long.parseLong(experiment.substring(DATA_BUFFER_TIME_LIMIT_MS.length()));
+      }
+    }
+    return DEFAULT_BUFFER_LIMIT_TIME_MS;
+  }
+
+  private class Receiver<T> implements FnDataReceiver<T> {
+    private final ByteString.Output output;
+    private final Coder<T> coder;
+    private long byteCount;
+    private long elementCount;
+
+    public Receiver(Coder<T> coder) {
+      this.output = ByteString.newOutput();
+      this.coder = coder;
+      this.byteCount = 0L;
+    }
+
+    @Override
+    public void accept(T input) throws Exception {
+      int size = output.size();
+      coder.encode(input, output);
+      if (output.size() - size == 0) {
+        output.write(0);
+      }
+      final long delta = (long) output.size() - size;
+      totalByteCounter.add(delta);
+      byteCount += delta;
+      elementCount += 1;
+      if (totalByteCounter.longValue() > sizeLimit) {
+        flush();
+      }
+    }
+
+    public ByteString.Output getOutput() {
+      return output;
+    }
+
+    public void reset() {
+      this.byteCount = 0L;
+      this.elementCount = 0L;
+      this.output.reset();
+    }
+
+    @Override
+    public String toString() {
+      return String.format("Byte size: %s, Element count: %s", byteCount, 
elementCount);
+    }
+  }
+
+  private static class TimerEndpoint {
+
+    private final String pTransformId;
+    private final String timerFamilyId;
+
+    public TimerEndpoint(String pTransformId, String timerFamilyId) {
+      this.pTransformId = pTransformId;
+      this.timerFamilyId = timerFamilyId;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) {
+        return true;
+      }
+      if (!(o instanceof TimerEndpoint)) {
+        return false;
+      }
+      TimerEndpoint that = (TimerEndpoint) o;
+      return pTransformId.equals(that.pTransformId) && 
timerFamilyId.equals(that.timerFamilyId);
+    }
+
+    @Override
+    public int hashCode() {
+      return Objects.hash(pTransformId, timerFamilyId);
+    }
+
+    @Override
+    public String toString() {
+      return "TimerEndpoint{"
+          + "pTransformId='"
+          + pTransformId
+          + '\''
+          + ", timerFamilyId='"
+          + timerFamilyId
+          + '\''
+          + '}';

Review comment:
       So it looks better in the debug log.
   ```suggestion
         return "pTransformId: " + pTransformId + " timerFamilyId: " + 
timerFamilyId;
   ```

##########
File path: 
sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/data/BeamFnDataOutboundAggregator.java
##########
@@ -0,0 +1,372 @@
+/*
+ * 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.sdk.fn.data;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.LongAdder;
+import java.util.function.Supplier;
+import org.apache.beam.model.fnexecution.v1.BeamFnApi;
+import org.apache.beam.model.fnexecution.v1.BeamFnApi.Elements;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.options.ExperimentalOptions;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.vendor.grpc.v1p43p2.com.google.protobuf.ByteString;
+import org.apache.beam.vendor.grpc.v1p43p2.io.grpc.stub.StreamObserver;
+import 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
+import 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * An outbound data buffering aggregator with size-based buffer and time-based 
buffer if
+ * corresponding options are set.
+ *
+ * <p>The default size-based buffer threshold can be overridden by specifying 
the experiment {@code
+ * data_buffer_size_limit=<bytes>}
+ *
+ * <p>The default time-based buffer threshold can be overridden by specifying 
the experiment {@code
+ * data_buffer_time_limit_ms=<milliseconds>}
+ */
+@SuppressWarnings({
+  "nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
+})
+public class BeamFnDataOutboundAggregator {
+
+  public static final String DATA_BUFFER_SIZE_LIMIT = 
"data_buffer_size_limit=";
+  public static final int DEFAULT_BUFFER_LIMIT_BYTES = 1_000_000;
+  public static final String DATA_BUFFER_TIME_LIMIT_MS = 
"data_buffer_time_limit_ms=";
+  public static final long DEFAULT_BUFFER_LIMIT_TIME_MS = -1L;
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(BeamFnDataOutboundAggregator.class);
+  private final int sizeLimit;
+  private final long timeLimit;
+  private final Supplier<String> processBundleRequestIdSupplier;
+  private final Map<String, Receiver<?>> outputDataReceivers;
+  private final Map<TimerEndpoint, Receiver<?>> outputTimersReceivers;
+  private final StreamObserver<Elements> outboundObserver;
+  @VisibleForTesting ScheduledFuture<?> flushFuture;
+  private final LongAdder totalByteCounter;
+  private final Object flushLock;
+
+  public BeamFnDataOutboundAggregator(
+      PipelineOptions options,
+      Supplier<String> processBundleRequestIdSupplier,
+      StreamObserver<BeamFnApi.Elements> outboundObserver) {
+    this.sizeLimit = getSizeLimit(options);
+    this.timeLimit = getTimeLimit(options);
+    this.outputDataReceivers = new HashMap<>();
+    this.outputTimersReceivers = new HashMap<>();
+    this.outboundObserver = outboundObserver;
+    this.processBundleRequestIdSupplier = processBundleRequestIdSupplier;
+    this.totalByteCounter = new LongAdder();
+    this.flushLock = new Object();
+  }
+
+  /** Starts the flushing daemon thread if data_buffer_time_limit_ms is set. */
+  public void startFlushThread() {
+    if (timeLimit > 0 && this.flushFuture == null) {
+      this.flushFuture =
+          Executors.newSingleThreadScheduledExecutor(
+                  new ThreadFactoryBuilder()
+                      .setDaemon(true)
+                      .setNameFormat("DataBufferOutboundFlusher-thread")
+                      .build())
+              .scheduleAtFixedRate(
+                  this::periodicFlush, timeLimit, timeLimit, 
TimeUnit.MILLISECONDS);
+    }
+  }
+
+  /**
+   * Register the outbound data logical endpoint, returns the FnDataReceiver 
for processing the
+   * endpoint's outbound data.
+   */
+  public <T> FnDataReceiver<T> registerOutputDataLocation(String pTransformId, 
Coder<T> coder) {
+    if (outputDataReceivers.containsKey(pTransformId)) {
+      throw new IllegalStateException(
+          "Outbound data endpoint already registered for " + pTransformId);
+    }
+    Receiver<T> receiver = new Receiver<>(coder);
+    if (timeLimit > 0) {
+      outputDataReceivers.put(pTransformId, receiver);
+      return data -> {
+        checkFlushThreadException();
+        synchronized (flushLock) {
+          receiver.accept(data);
+          if (totalByteCounter.longValue() > sizeLimit) {
+            flush();
+          }
+        }
+      };
+    }
+    outputDataReceivers.put(pTransformId, receiver);
+    return receiver;
+  }
+
+  /**
+   * Register the outbound timers logical endpoint, returns the FnDataReceiver 
for processing the
+   * endpoint's outbound timers data.
+   */
+  public <T> FnDataReceiver<T> registerOutputTimersLocation(
+      String pTransformId, String timerFamilyId, Coder<T> coder) {
+    TimerEndpoint timerKey = new TimerEndpoint(pTransformId, timerFamilyId);
+    if (outputTimersReceivers.containsKey(timerKey)) {
+      throw new IllegalStateException(
+          "Outbound timers endpoint already registered for " + timerKey);
+    }
+    Receiver<T> receiver = new Receiver<>(coder);
+    if (timeLimit > 0) {
+      outputTimersReceivers.put(timerKey, receiver);
+      return timers -> {
+        checkFlushThreadException();
+        synchronized (flushLock) {
+          receiver.accept(timers);
+        }
+      };
+    }
+    outputTimersReceivers.put(timerKey, receiver);
+    return receiver;
+  }
+
+  public void flush() throws IOException {
+    Elements.Builder elements = convertBufferForTransmission();
+    if (elements.getDataCount() > 0 || elements.getTimersCount() > 0) {
+      outboundObserver.onNext(elements.build());
+    }
+  }
+
+  /**
+   * Closes the streams for all registered outbound endpoints. Should be 
called at the end of each
+   * bundle.
+   */
+  public void sendBufferedDataAndFinishOutboundStreams() {
+    if (totalByteCounter.longValue() == 0
+        && outputTimersReceivers.isEmpty()
+        && outputDataReceivers.isEmpty()) {
+      return;
+    }
+    Elements.Builder bufferedElements = convertBufferForTransmission();
+    LOG.debug(
+        "Closing streams for outbound endpoints {} and {}.",
+        outputDataReceivers.keySet(),
+        outputTimersReceivers.keySet());
+    LOG.debug(
+        "Sent outbound data : {}, outbound timers : {}.",
+        outputDataReceivers,
+        outputTimersReceivers);
+    for (String pTransformID : outputDataReceivers.keySet()) {
+      bufferedElements
+          .addDataBuilder()
+          .setInstructionId(processBundleRequestIdSupplier.get())
+          .setTransformId(pTransformID)
+          .setIsLast(true);
+    }
+    for (TimerEndpoint timerKey : outputTimersReceivers.keySet()) {
+      bufferedElements
+          .addTimersBuilder()
+          .setInstructionId(processBundleRequestIdSupplier.get())
+          .setTransformId(timerKey.pTransformId)
+          .setTimerFamilyId(timerKey.timerFamilyId)
+          .setIsLast(true);
+    }
+    outboundObserver.onNext(bufferedElements.build());
+    outputDataReceivers.values().forEach(Receiver::reset);
+    outputTimersReceivers.values().forEach(Receiver::reset);

Review comment:
       Note that this is a lot slower executing when compared to:
   ```
   for (Receiver receiver : outputDataReceivers.values()) {
     receiver.reset();
   }
   ```
   
   Also, you can merge this with the iteration above using an entryset and add 
the data/timer and reset the receiver in the loop above.

##########
File path: 
sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/data/BeamFnDataOutboundAggregatorTest.java
##########
@@ -0,0 +1,371 @@
+/*
+ * 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.sdk.fn.data;
+
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.empty;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.function.Consumer;
+import org.apache.beam.model.fnexecution.v1.BeamFnApi;
+import org.apache.beam.model.fnexecution.v1.BeamFnApi.Elements;
+import org.apache.beam.sdk.coders.ByteArrayCoder;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.LengthPrefixCoder;
+import org.apache.beam.sdk.fn.test.TestStreams;
+import org.apache.beam.sdk.options.ExperimentalOptions;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.vendor.grpc.v1p43p2.com.google.protobuf.ByteString;
+import org.hamcrest.Matchers;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
+/** Tests for {@link BeamFnDataOutboundAggregator}. */
+@RunWith(Parameterized.class)
+public class BeamFnDataOutboundAggregatorTest {
+
+  private static final LogicalEndpoint DATA_OUTPUT_LOCATION = 
LogicalEndpoint.data("777L", "555L");
+  private static final LogicalEndpoint TIMER_OUTPUT_LOCATION =
+      LogicalEndpoint.timer("999L", "333L", "111L");
+  private static final Coder<byte[]> CODER = 
LengthPrefixCoder.of(ByteArrayCoder.of());
+
+  @Parameters
+  public static Collection<LogicalEndpoint> data() {
+    return Arrays.asList(DATA_OUTPUT_LOCATION, TIMER_OUTPUT_LOCATION);
+  }
+
+  private final LogicalEndpoint endpoint;
+
+  public BeamFnDataOutboundAggregatorTest(LogicalEndpoint endpoint) {
+    this.endpoint = endpoint;
+  }
+
+  @Test
+  public void testWithDefaultBuffer() throws Exception {
+    final List<Elements> values = new ArrayList<>();
+    final AtomicBoolean onCompletedWasCalled = new AtomicBoolean();
+    BeamFnDataOutboundAggregator aggregator =
+        new BeamFnDataOutboundAggregator(
+            PipelineOptionsFactory.create(),
+            endpoint::getInstructionId,
+            TestStreams.<Elements>withOnNext(values::add)
+                .withOnCompleted(() -> onCompletedWasCalled.set(true))
+                .build());
+
+    // Test that nothing is emitted till the default buffer size is surpassed.
+    FnDataReceiver<byte[]> dataReceiver = registerOutputLocation(aggregator, 
endpoint, CODER);
+    aggregator.startFlushThread();
+    dataReceiver.accept(new 
byte[BeamFnDataOutboundAggregator.DEFAULT_BUFFER_LIMIT_BYTES - 50]);
+    assertThat(values, empty());
+
+    // Test that when we cross the buffer, we emit.
+    dataReceiver.accept(new byte[50]);
+    assertEquals(
+        messageWithData(
+            new byte[BeamFnDataOutboundAggregator.DEFAULT_BUFFER_LIMIT_BYTES - 
50], new byte[50]),
+        values.get(0));
+
+    // Test that nothing is emitted till the default buffer size is surpassed 
after a reset
+    dataReceiver.accept(new 
byte[BeamFnDataOutboundAggregator.DEFAULT_BUFFER_LIMIT_BYTES - 50]);
+    assertEquals(1, values.size());
+
+    // Test that when we cross the buffer, we emit.
+    dataReceiver.accept(new byte[50]);
+    assertEquals(
+        messageWithData(
+            new byte[BeamFnDataOutboundAggregator.DEFAULT_BUFFER_LIMIT_BYTES - 
50], new byte[50]),
+        values.get(1));
+
+    // Test that when we close with an empty buffer we only have one end of 
stream
+    aggregator.sendBufferedDataAndFinishOutboundStreams();
+
+    assertEquals(endMessage(), values.get(2));
+
+    // Test that we can close twice.
+    aggregator.sendBufferedDataAndFinishOutboundStreams();

Review comment:
       ```suggestion
       aggregator.sendBufferedDataAndFinishOutboundStreams();
       assertEquals(endMessage(), values.get(2));
   
       // Test that we can close twice.
       aggregator.sendBufferedDataAndFinishOutboundStreams();
       assertEquals(endMessage(), values.get(2));
   ```

##########
File path: 
sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/data/BeamFnDataOutboundAggregator.java
##########
@@ -0,0 +1,372 @@
+/*
+ * 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.sdk.fn.data;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.LongAdder;
+import java.util.function.Supplier;
+import org.apache.beam.model.fnexecution.v1.BeamFnApi;
+import org.apache.beam.model.fnexecution.v1.BeamFnApi.Elements;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.options.ExperimentalOptions;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.vendor.grpc.v1p43p2.com.google.protobuf.ByteString;
+import org.apache.beam.vendor.grpc.v1p43p2.io.grpc.stub.StreamObserver;
+import 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
+import 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * An outbound data buffering aggregator with size-based buffer and time-based 
buffer if
+ * corresponding options are set.
+ *
+ * <p>The default size-based buffer threshold can be overridden by specifying 
the experiment {@code
+ * data_buffer_size_limit=<bytes>}
+ *
+ * <p>The default time-based buffer threshold can be overridden by specifying 
the experiment {@code
+ * data_buffer_time_limit_ms=<milliseconds>}
+ */
+@SuppressWarnings({
+  "nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
+})
+public class BeamFnDataOutboundAggregator {
+
+  public static final String DATA_BUFFER_SIZE_LIMIT = 
"data_buffer_size_limit=";
+  public static final int DEFAULT_BUFFER_LIMIT_BYTES = 1_000_000;
+  public static final String DATA_BUFFER_TIME_LIMIT_MS = 
"data_buffer_time_limit_ms=";
+  public static final long DEFAULT_BUFFER_LIMIT_TIME_MS = -1L;
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(BeamFnDataOutboundAggregator.class);
+  private final int sizeLimit;
+  private final long timeLimit;
+  private final Supplier<String> processBundleRequestIdSupplier;
+  private final Map<String, Receiver<?>> outputDataReceivers;
+  private final Map<TimerEndpoint, Receiver<?>> outputTimersReceivers;
+  private final StreamObserver<Elements> outboundObserver;
+  @VisibleForTesting ScheduledFuture<?> flushFuture;
+  private final LongAdder totalByteCounter;
+  private final Object flushLock;
+
+  public BeamFnDataOutboundAggregator(
+      PipelineOptions options,
+      Supplier<String> processBundleRequestIdSupplier,
+      StreamObserver<BeamFnApi.Elements> outboundObserver) {
+    this.sizeLimit = getSizeLimit(options);
+    this.timeLimit = getTimeLimit(options);
+    this.outputDataReceivers = new HashMap<>();
+    this.outputTimersReceivers = new HashMap<>();
+    this.outboundObserver = outboundObserver;
+    this.processBundleRequestIdSupplier = processBundleRequestIdSupplier;
+    this.totalByteCounter = new LongAdder();
+    this.flushLock = new Object();
+  }
+
+  /** Starts the flushing daemon thread if data_buffer_time_limit_ms is set. */
+  public void startFlushThread() {
+    if (timeLimit > 0 && this.flushFuture == null) {
+      this.flushFuture =
+          Executors.newSingleThreadScheduledExecutor(
+                  new ThreadFactoryBuilder()
+                      .setDaemon(true)
+                      .setNameFormat("DataBufferOutboundFlusher-thread")
+                      .build())
+              .scheduleAtFixedRate(
+                  this::periodicFlush, timeLimit, timeLimit, 
TimeUnit.MILLISECONDS);
+    }
+  }
+
+  /**
+   * Register the outbound data logical endpoint, returns the FnDataReceiver 
for processing the
+   * endpoint's outbound data.
+   */
+  public <T> FnDataReceiver<T> registerOutputDataLocation(String pTransformId, 
Coder<T> coder) {
+    if (outputDataReceivers.containsKey(pTransformId)) {
+      throw new IllegalStateException(
+          "Outbound data endpoint already registered for " + pTransformId);
+    }
+    Receiver<T> receiver = new Receiver<>(coder);
+    if (timeLimit > 0) {
+      outputDataReceivers.put(pTransformId, receiver);
+      return data -> {
+        checkFlushThreadException();
+        synchronized (flushLock) {
+          receiver.accept(data);
+          if (totalByteCounter.longValue() > sizeLimit) {
+            flush();
+          }
+        }
+      };
+    }
+    outputDataReceivers.put(pTransformId, receiver);
+    return receiver;
+  }
+
+  /**
+   * Register the outbound timers logical endpoint, returns the FnDataReceiver 
for processing the
+   * endpoint's outbound timers data.
+   */
+  public <T> FnDataReceiver<T> registerOutputTimersLocation(
+      String pTransformId, String timerFamilyId, Coder<T> coder) {
+    TimerEndpoint timerKey = new TimerEndpoint(pTransformId, timerFamilyId);
+    if (outputTimersReceivers.containsKey(timerKey)) {
+      throw new IllegalStateException(
+          "Outbound timers endpoint already registered for " + timerKey);
+    }
+    Receiver<T> receiver = new Receiver<>(coder);
+    if (timeLimit > 0) {
+      outputTimersReceivers.put(timerKey, receiver);
+      return timers -> {
+        checkFlushThreadException();
+        synchronized (flushLock) {
+          receiver.accept(timers);
+        }
+      };
+    }
+    outputTimersReceivers.put(timerKey, receiver);
+    return receiver;
+  }
+
+  public void flush() throws IOException {
+    Elements.Builder elements = convertBufferForTransmission();
+    if (elements.getDataCount() > 0 || elements.getTimersCount() > 0) {
+      outboundObserver.onNext(elements.build());
+    }
+  }
+
+  /**
+   * Closes the streams for all registered outbound endpoints. Should be 
called at the end of each
+   * bundle.
+   */
+  public void sendBufferedDataAndFinishOutboundStreams() {
+    if (totalByteCounter.longValue() == 0
+        && outputTimersReceivers.isEmpty()
+        && outputDataReceivers.isEmpty()) {
+      return;
+    }
+    Elements.Builder bufferedElements = convertBufferForTransmission();
+    LOG.debug(
+        "Closing streams for outbound endpoints {} and {}.",
+        outputDataReceivers.keySet(),
+        outputTimersReceivers.keySet());
+    LOG.debug(
+        "Sent outbound data : {}, outbound timers : {}.",
+        outputDataReceivers,
+        outputTimersReceivers);
+    for (String pTransformID : outputDataReceivers.keySet()) {
+      bufferedElements
+          .addDataBuilder()
+          .setInstructionId(processBundleRequestIdSupplier.get())
+          .setTransformId(pTransformID)
+          .setIsLast(true);
+    }
+    for (TimerEndpoint timerKey : outputTimersReceivers.keySet()) {
+      bufferedElements
+          .addTimersBuilder()
+          .setInstructionId(processBundleRequestIdSupplier.get())
+          .setTransformId(timerKey.pTransformId)
+          .setTimerFamilyId(timerKey.timerFamilyId)
+          .setIsLast(true);
+    }
+    outboundObserver.onNext(bufferedElements.build());
+    outputDataReceivers.values().forEach(Receiver::reset);
+    outputTimersReceivers.values().forEach(Receiver::reset);
+  }
+
+  private Elements.Builder convertBufferForTransmission() {
+    Elements.Builder bufferedElements = Elements.newBuilder();
+    for (Map.Entry<String, Receiver<?>> entry : 
outputDataReceivers.entrySet()) {
+      if (entry.getValue().getOutput().size() == 0) {
+        continue;
+      }
+      ByteString bytes = entry.getValue().getOutput().toByteString();
+      bufferedElements
+          .addDataBuilder()
+          .setInstructionId(processBundleRequestIdSupplier.get())
+          .setTransformId(entry.getKey())
+          .setData(bytes);
+      entry.getValue().reset();

Review comment:
       You want to reset the ByteString.Output and not the receiver since we 
want to keep the element and byte counts.

##########
File path: 
sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/data/BeamFnDataOutboundAggregator.java
##########
@@ -0,0 +1,372 @@
+/*
+ * 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.sdk.fn.data;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.LongAdder;
+import java.util.function.Supplier;
+import org.apache.beam.model.fnexecution.v1.BeamFnApi;
+import org.apache.beam.model.fnexecution.v1.BeamFnApi.Elements;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.options.ExperimentalOptions;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.vendor.grpc.v1p43p2.com.google.protobuf.ByteString;
+import org.apache.beam.vendor.grpc.v1p43p2.io.grpc.stub.StreamObserver;
+import 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
+import 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * An outbound data buffering aggregator with size-based buffer and time-based 
buffer if
+ * corresponding options are set.
+ *
+ * <p>The default size-based buffer threshold can be overridden by specifying 
the experiment {@code
+ * data_buffer_size_limit=<bytes>}
+ *
+ * <p>The default time-based buffer threshold can be overridden by specifying 
the experiment {@code
+ * data_buffer_time_limit_ms=<milliseconds>}
+ */
+@SuppressWarnings({
+  "nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
+})
+public class BeamFnDataOutboundAggregator {
+
+  public static final String DATA_BUFFER_SIZE_LIMIT = 
"data_buffer_size_limit=";
+  public static final int DEFAULT_BUFFER_LIMIT_BYTES = 1_000_000;
+  public static final String DATA_BUFFER_TIME_LIMIT_MS = 
"data_buffer_time_limit_ms=";
+  public static final long DEFAULT_BUFFER_LIMIT_TIME_MS = -1L;
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(BeamFnDataOutboundAggregator.class);
+  private final int sizeLimit;
+  private final long timeLimit;
+  private final Supplier<String> processBundleRequestIdSupplier;
+  private final Map<String, Receiver<?>> outputDataReceivers;
+  private final Map<TimerEndpoint, Receiver<?>> outputTimersReceivers;
+  private final StreamObserver<Elements> outboundObserver;
+  @VisibleForTesting ScheduledFuture<?> flushFuture;
+  private final LongAdder totalByteCounter;
+  private final Object flushLock;
+
+  public BeamFnDataOutboundAggregator(
+      PipelineOptions options,
+      Supplier<String> processBundleRequestIdSupplier,
+      StreamObserver<BeamFnApi.Elements> outboundObserver) {
+    this.sizeLimit = getSizeLimit(options);
+    this.timeLimit = getTimeLimit(options);
+    this.outputDataReceivers = new HashMap<>();
+    this.outputTimersReceivers = new HashMap<>();
+    this.outboundObserver = outboundObserver;
+    this.processBundleRequestIdSupplier = processBundleRequestIdSupplier;
+    this.totalByteCounter = new LongAdder();
+    this.flushLock = new Object();
+  }
+
+  /** Starts the flushing daemon thread if data_buffer_time_limit_ms is set. */
+  public void startFlushThread() {
+    if (timeLimit > 0 && this.flushFuture == null) {
+      this.flushFuture =
+          Executors.newSingleThreadScheduledExecutor(
+                  new ThreadFactoryBuilder()
+                      .setDaemon(true)
+                      .setNameFormat("DataBufferOutboundFlusher-thread")
+                      .build())
+              .scheduleAtFixedRate(
+                  this::periodicFlush, timeLimit, timeLimit, 
TimeUnit.MILLISECONDS);
+    }
+  }
+
+  /**
+   * Register the outbound data logical endpoint, returns the FnDataReceiver 
for processing the
+   * endpoint's outbound data.
+   */
+  public <T> FnDataReceiver<T> registerOutputDataLocation(String pTransformId, 
Coder<T> coder) {
+    if (outputDataReceivers.containsKey(pTransformId)) {
+      throw new IllegalStateException(
+          "Outbound data endpoint already registered for " + pTransformId);
+    }
+    Receiver<T> receiver = new Receiver<>(coder);
+    if (timeLimit > 0) {
+      outputDataReceivers.put(pTransformId, receiver);
+      return data -> {
+        checkFlushThreadException();
+        synchronized (flushLock) {
+          receiver.accept(data);
+          if (totalByteCounter.longValue() > sizeLimit) {
+            flush();
+          }
+        }
+      };
+    }
+    outputDataReceivers.put(pTransformId, receiver);
+    return receiver;
+  }
+
+  /**
+   * Register the outbound timers logical endpoint, returns the FnDataReceiver 
for processing the
+   * endpoint's outbound timers data.
+   */
+  public <T> FnDataReceiver<T> registerOutputTimersLocation(
+      String pTransformId, String timerFamilyId, Coder<T> coder) {
+    TimerEndpoint timerKey = new TimerEndpoint(pTransformId, timerFamilyId);
+    if (outputTimersReceivers.containsKey(timerKey)) {
+      throw new IllegalStateException(
+          "Outbound timers endpoint already registered for " + timerKey);
+    }
+    Receiver<T> receiver = new Receiver<>(coder);
+    if (timeLimit > 0) {
+      outputTimersReceivers.put(timerKey, receiver);
+      return timers -> {
+        checkFlushThreadException();
+        synchronized (flushLock) {
+          receiver.accept(timers);
+        }
+      };
+    }
+    outputTimersReceivers.put(timerKey, receiver);
+    return receiver;
+  }
+
+  public void flush() throws IOException {
+    Elements.Builder elements = convertBufferForTransmission();
+    if (elements.getDataCount() > 0 || elements.getTimersCount() > 0) {
+      outboundObserver.onNext(elements.build());
+    }
+  }
+
+  /**
+   * Closes the streams for all registered outbound endpoints. Should be 
called at the end of each
+   * bundle.
+   */
+  public void sendBufferedDataAndFinishOutboundStreams() {
+    if (totalByteCounter.longValue() == 0
+        && outputTimersReceivers.isEmpty()
+        && outputDataReceivers.isEmpty()) {
+      return;
+    }
+    Elements.Builder bufferedElements = convertBufferForTransmission();
+    LOG.debug(
+        "Closing streams for outbound endpoints {} and {}.",
+        outputDataReceivers.keySet(),
+        outputTimersReceivers.keySet());
+    LOG.debug(
+        "Sent outbound data : {}, outbound timers : {}.",
+        outputDataReceivers,
+        outputTimersReceivers);
+    for (String pTransformID : outputDataReceivers.keySet()) {
+      bufferedElements
+          .addDataBuilder()
+          .setInstructionId(processBundleRequestIdSupplier.get())
+          .setTransformId(pTransformID)
+          .setIsLast(true);
+    }
+    for (TimerEndpoint timerKey : outputTimersReceivers.keySet()) {
+      bufferedElements
+          .addTimersBuilder()
+          .setInstructionId(processBundleRequestIdSupplier.get())
+          .setTransformId(timerKey.pTransformId)
+          .setTimerFamilyId(timerKey.timerFamilyId)
+          .setIsLast(true);
+    }
+    outboundObserver.onNext(bufferedElements.build());
+    outputDataReceivers.values().forEach(Receiver::reset);
+    outputTimersReceivers.values().forEach(Receiver::reset);
+  }
+
+  private Elements.Builder convertBufferForTransmission() {
+    Elements.Builder bufferedElements = Elements.newBuilder();
+    for (Map.Entry<String, Receiver<?>> entry : 
outputDataReceivers.entrySet()) {
+      if (entry.getValue().getOutput().size() == 0) {
+        continue;
+      }
+      ByteString bytes = entry.getValue().getOutput().toByteString();
+      bufferedElements
+          .addDataBuilder()
+          .setInstructionId(processBundleRequestIdSupplier.get())
+          .setTransformId(entry.getKey())
+          .setData(bytes);
+      entry.getValue().reset();
+    }
+    for (Map.Entry<TimerEndpoint, Receiver<?>> entry : 
outputTimersReceivers.entrySet()) {
+      if (entry.getValue().getOutput().size() == 0) {
+        continue;
+      }
+      ByteString bytes = entry.getValue().getOutput().toByteString();
+      bufferedElements
+          .addTimersBuilder()
+          .setInstructionId(processBundleRequestIdSupplier.get())
+          .setTransformId(entry.getKey().pTransformId)
+          .setTimerFamilyId(entry.getKey().timerFamilyId)
+          .setTimers(bytes);
+      entry.getValue().reset();

Review comment:
       You want to reset the ByteString.Output and not the receiver since we 
want to keep the element and byte counts.

##########
File path: 
sdks/java/harness/src/test/java/org/apache/beam/fn/harness/FnApiDoFnRunnerTest.java
##########
@@ -3540,6 +3580,7 @@ public void 
testProcessElementForWindowedTruncateAndSizeRestriction() throws Exc
               
.pCollections(pProto.getComponentsOrBuilder().getPcollectionsMap())
               .coders(pProto.getComponents().getCodersMap())
               
.windowingStrategies(pProto.getComponents().getWindowingStrategiesMap())
+              .outboundAggregators(new HashMap<>())

Review comment:
       Isn't this the default in PTransformRunnerFactoryTestContext and 
unnecessary?
   ```suggestion
   ```

##########
File path: 
sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/data/BeamFnDataOutboundAggregator.java
##########
@@ -0,0 +1,372 @@
+/*
+ * 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.sdk.fn.data;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.LongAdder;
+import java.util.function.Supplier;
+import org.apache.beam.model.fnexecution.v1.BeamFnApi;
+import org.apache.beam.model.fnexecution.v1.BeamFnApi.Elements;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.options.ExperimentalOptions;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.vendor.grpc.v1p43p2.com.google.protobuf.ByteString;
+import org.apache.beam.vendor.grpc.v1p43p2.io.grpc.stub.StreamObserver;
+import 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
+import 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * An outbound data buffering aggregator with size-based buffer and time-based 
buffer if
+ * corresponding options are set.
+ *
+ * <p>The default size-based buffer threshold can be overridden by specifying 
the experiment {@code
+ * data_buffer_size_limit=<bytes>}
+ *
+ * <p>The default time-based buffer threshold can be overridden by specifying 
the experiment {@code
+ * data_buffer_time_limit_ms=<milliseconds>}
+ */
+@SuppressWarnings({
+  "nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
+})
+public class BeamFnDataOutboundAggregator {
+
+  public static final String DATA_BUFFER_SIZE_LIMIT = 
"data_buffer_size_limit=";
+  public static final int DEFAULT_BUFFER_LIMIT_BYTES = 1_000_000;
+  public static final String DATA_BUFFER_TIME_LIMIT_MS = 
"data_buffer_time_limit_ms=";
+  public static final long DEFAULT_BUFFER_LIMIT_TIME_MS = -1L;
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(BeamFnDataOutboundAggregator.class);
+  private final int sizeLimit;
+  private final long timeLimit;
+  private final Supplier<String> processBundleRequestIdSupplier;
+  private final Map<String, Receiver<?>> outputDataReceivers;
+  private final Map<TimerEndpoint, Receiver<?>> outputTimersReceivers;
+  private final StreamObserver<Elements> outboundObserver;
+  @VisibleForTesting ScheduledFuture<?> flushFuture;
+  private final LongAdder totalByteCounter;
+  private final Object flushLock;
+
+  public BeamFnDataOutboundAggregator(
+      PipelineOptions options,
+      Supplier<String> processBundleRequestIdSupplier,
+      StreamObserver<BeamFnApi.Elements> outboundObserver) {
+    this.sizeLimit = getSizeLimit(options);
+    this.timeLimit = getTimeLimit(options);
+    this.outputDataReceivers = new HashMap<>();
+    this.outputTimersReceivers = new HashMap<>();
+    this.outboundObserver = outboundObserver;
+    this.processBundleRequestIdSupplier = processBundleRequestIdSupplier;
+    this.totalByteCounter = new LongAdder();
+    this.flushLock = new Object();
+  }
+
+  /** Starts the flushing daemon thread if data_buffer_time_limit_ms is set. */
+  public void startFlushThread() {
+    if (timeLimit > 0 && this.flushFuture == null) {
+      this.flushFuture =
+          Executors.newSingleThreadScheduledExecutor(
+                  new ThreadFactoryBuilder()
+                      .setDaemon(true)
+                      .setNameFormat("DataBufferOutboundFlusher-thread")
+                      .build())
+              .scheduleAtFixedRate(
+                  this::periodicFlush, timeLimit, timeLimit, 
TimeUnit.MILLISECONDS);
+    }
+  }
+
+  /**
+   * Register the outbound data logical endpoint, returns the FnDataReceiver 
for processing the
+   * endpoint's outbound data.
+   */
+  public <T> FnDataReceiver<T> registerOutputDataLocation(String pTransformId, 
Coder<T> coder) {
+    if (outputDataReceivers.containsKey(pTransformId)) {
+      throw new IllegalStateException(
+          "Outbound data endpoint already registered for " + pTransformId);
+    }
+    Receiver<T> receiver = new Receiver<>(coder);
+    if (timeLimit > 0) {
+      outputDataReceivers.put(pTransformId, receiver);
+      return data -> {
+        checkFlushThreadException();
+        synchronized (flushLock) {
+          receiver.accept(data);
+          if (totalByteCounter.longValue() > sizeLimit) {
+            flush();
+          }
+        }
+      };
+    }
+    outputDataReceivers.put(pTransformId, receiver);
+    return receiver;
+  }
+
+  /**
+   * Register the outbound timers logical endpoint, returns the FnDataReceiver 
for processing the
+   * endpoint's outbound timers data.
+   */
+  public <T> FnDataReceiver<T> registerOutputTimersLocation(
+      String pTransformId, String timerFamilyId, Coder<T> coder) {
+    TimerEndpoint timerKey = new TimerEndpoint(pTransformId, timerFamilyId);
+    if (outputTimersReceivers.containsKey(timerKey)) {
+      throw new IllegalStateException(
+          "Outbound timers endpoint already registered for " + timerKey);
+    }
+    Receiver<T> receiver = new Receiver<>(coder);
+    if (timeLimit > 0) {
+      outputTimersReceivers.put(timerKey, receiver);
+      return timers -> {
+        checkFlushThreadException();
+        synchronized (flushLock) {
+          receiver.accept(timers);
+        }
+      };
+    }
+    outputTimersReceivers.put(timerKey, receiver);
+    return receiver;
+  }
+
+  public void flush() throws IOException {
+    Elements.Builder elements = convertBufferForTransmission();
+    if (elements.getDataCount() > 0 || elements.getTimersCount() > 0) {
+      outboundObserver.onNext(elements.build());
+    }
+  }
+
+  /**
+   * Closes the streams for all registered outbound endpoints. Should be 
called at the end of each
+   * bundle.
+   */
+  public void sendBufferedDataAndFinishOutboundStreams() {
+    if (totalByteCounter.longValue() == 0
+        && outputTimersReceivers.isEmpty()
+        && outputDataReceivers.isEmpty()) {
+      return;
+    }
+    Elements.Builder bufferedElements = convertBufferForTransmission();
+    LOG.debug(
+        "Closing streams for outbound endpoints {} and {}.",
+        outputDataReceivers.keySet(),
+        outputTimersReceivers.keySet());
+    LOG.debug(
+        "Sent outbound data : {}, outbound timers : {}.",
+        outputDataReceivers,
+        outputTimersReceivers);

Review comment:
       ```suggestion
       LOG.debug(
           "Closing streams for instruction {} and outbound data {} and timers 
{} having sent {} bytes.",
           processBundleRequestIdSupplier.get(),
           outputDataReceivers,
           outputTimersReceivers,
           totalByteCounter.longValue());
   ```

##########
File path: 
sdks/java/harness/src/test/java/org/apache/beam/fn/harness/FnApiDoFnRunnerTest.java
##########
@@ -3670,6 +3711,7 @@ public void testDoFnSkewNotAllowed() throws Exception {
               
.pCollections(pProto.getComponentsOrBuilder().getPcollectionsMap())
               .coders(pProto.getComponents().getCodersMap())
               
.windowingStrategies(pProto.getComponents().getWindowingStrategiesMap())
+              .outboundAggregators(new HashMap<>())

Review comment:
       Isn't this the default in PTransformRunnerFactoryTestContext and 
unnecessary?
   ```suggestion
   ```

##########
File path: 
sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnApiDoFnRunner.java
##########
@@ -238,7 +234,7 @@
 
   private final String mainInputId;
   private final FnApiStateAccessor<?> stateAccessor;
-  private Map<String, CloseableFnDataReceiver<Timer<Object>>> 
outboundTimerReceivers;
+  private Map<String, FnDataReceiver<?>> outboundTimerReceivers;

Review comment:
       ```suggestion
     private final Map<String, FnDataReceiver<?>> outboundTimerReceivers;
   ```

##########
File path: 
sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/ProcessBundleHandler.java
##########
@@ -445,6 +472,10 @@ public BundleFinalizer getBundleFinalizer() {
                 throw new RuntimeException(e);
               }
             });
+    for (BeamFnDataOutboundAggregator aggregator :
+        bundleProcessor.getOutboundAggregators().values()) {
+      aggregator.startFlushThread();
+    }

Review comment:
       ```suggestion
       for (BeamFnDataOutboundAggregator aggregator :
           bundleProcessor.getOutboundAggregators().values()) {
         aggregator.startFlushThread();
       }
   ```
   
   Please move this to `BundleProcessor#finish`.
   
   You should also add a BeamFnDataOutboundAggregator#discard method that 
cleans up the internal thread that it created and invoke it from 
`BundleProcessor#discard`.




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