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:
       redundant since in `accept` now.
   ```suggestion
   ```




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