[ 
https://issues.apache.org/jira/browse/BEAM-7948?focusedWorklogId=345098&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-345098
 ]

ASF GitHub Bot logged work on BEAM-7948:
----------------------------------------

                Author: ASF GitHub Bot
            Created on: 18/Nov/19 07:40
            Start Date: 18/Nov/19 07:40
    Worklog Time Spent: 10m 
      Work Description: sunjincheng121 commented on pull request #9949: 
[BEAM-7948] Add time-based cache threshold support in the Java data s…
URL: https://github.com/apache/beam/pull/9949#discussion_r347233168
 
 

 ##########
 File path: 
sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/data/BeamFnDataTimeBasedBufferingOutboundObserver.java
 ##########
 @@ -0,0 +1,109 @@
+/*
+ * 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.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.AtomicReference;
+import org.apache.beam.model.fnexecution.v1.BeamFnApi;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.vendor.grpc.v1p21p0.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;
+
+/**
+ * A buffering outbound {@link FnDataReceiver} with both size-based buffer and 
time-based buffer
+ * enabled for the Beam Fn Data API.
+ */
+public class BeamFnDataTimeBasedBufferingOutboundObserver<T>
+    extends BeamFnDataSizeBasedBufferingOutboundObserver<T> {
+
+  private final Object lock;
+  private final ScheduledFuture<?> flushFuture;
+  @VisibleForTesting final AtomicReference<IOException> flushException;
+
+  BeamFnDataTimeBasedBufferingOutboundObserver(
+      int sizeLimit,
+      long timeLimit,
+      LogicalEndpoint outputLocation,
+      Coder<T> coder,
+      StreamObserver<BeamFnApi.Elements> outboundObserver) {
+    super(sizeLimit, outputLocation, coder, outboundObserver);
+    this.lock = new Object();
+    this.flushFuture =
+        Executors.newSingleThreadScheduledExecutor(
+                new ThreadFactoryBuilder()
+                    .setDaemon(true)
+                    .setNameFormat("DataBufferOutboundFlusher-thread")
+                    .build())
+            .scheduleAtFixedRate(this::periodicFlush, timeLimit, timeLimit, 
TimeUnit.MILLISECONDS);
+    this.flushException = new AtomicReference<>(null);
+  }
+
+  @Override
+  public void close() throws Exception {
+    checkFlushThreadException();
+    synchronized (lock) {
+      flushFuture.cancel(true);
+      try {
+        flushFuture.get();
+      } catch (CancellationException | ExecutionException | 
InterruptedException exn) {
+        // expected
+      }
+    }
+    super.close();
+  }
+
+  @Override
+  public void flush() throws IOException {
+    synchronized (lock) {
+      super.flush();
+    }
+  }
+
+  @Override
+  public void accept(T t) throws IOException {
+    checkFlushThreadException();
+    super.accept(t);
+  }
+
+  private void periodicFlush() {
 
 Review comment:
   It's not possible to use the flush method directly as it throws an 
IOException. It's a good idea to use `flushFuture.isDone()` to check the status 
of the flush thread. Will be Updated. :)
 
----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


Issue Time Tracking
-------------------

    Worklog Id:     (was: 345098)
    Time Spent: 1h 40m  (was: 1.5h)

> Add time-based cache threshold support in the Java data service
> ---------------------------------------------------------------
>
>                 Key: BEAM-7948
>                 URL: https://issues.apache.org/jira/browse/BEAM-7948
>             Project: Beam
>          Issue Type: Sub-task
>          Components: java-fn-execution
>            Reporter: sunjincheng
>            Assignee: sunjincheng
>            Priority: Major
>          Time Spent: 1h 40m
>  Remaining Estimate: 0h
>
> Currently only size-based cache threshold is supported in data service. It 
> should also support the time-based cache threshold. This is very important, 
> especially for streaming jobs which are sensitive to the delay.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to