scwhittle commented on code in PR #34283:
URL: https://github.com/apache/beam/pull/34283#discussion_r2011714194


##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/AbstractWindmillStream.java:
##########
@@ -236,13 +239,34 @@ protected final void executeSafely(Runnable runnable) {
     }
   }
 
-  public final synchronized void maybeSendHealthCheck(Instant 
lastSendThreshold) {
-    if (!clientClosed && debugMetrics.getLastSendTimeMs() < 
lastSendThreshold.getMillis()) {
-      try {
-        sendHealthCheck();
-      } catch (Exception e) {
-        logger.debug("Received exception sending health check.", e);
-      }
+  /**
+   * Schedule an application level keep-alive health check to be sent on the 
stream.
+   *
+   * @implNote This is sent asynchronously via an executor to minimize 
blocking. Messages are sent
+   *     serially. If we recently sent a message before we attempt to schedule 
the health check, the
+   *     stream has been restarted/closed, there is an active health check 
that hasn't completed due
+   *     to flow control/pushback or there was a more recent send by the time 
we enter the
+   *     synchronized block, we skip the attempt to send scheduled the health 
check.
+   */
+  public final void maybeSendHealthCheck(Instant lastSendThreshold) {
+    if (debugMetrics.getLastSendTimeMs() < lastSendThreshold.getMillis() && 
!isHealthCheckActive) {

Review Comment:
   maybe it should be !isHealthCheckScheduled and then set it to true here 
before putting on the executor?
   
   if the executor gets backlogged we could still end up queuing a lot of 
things on the executor since currently we don't mark active until it schedules



##########
runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/AbstractWindmillStreamTest.java:
##########
@@ -110,12 +79,75 @@ public void setMessageCompression(boolean b) {}
     // Sleep a bit to give sendExecutor time to execute the send().
     Uninterruptibles.sleepUninterruptibly(100, TimeUnit.MILLISECONDS);
 
-    sendBlocker.countDown();
+    callStreamObserver.unblockSend();
     
assertThat(sendFuture.get()).isInstanceOf(WindmillStreamShutdownException.class);
   }
 
+  @Test
+  public void testMaybeSendHealthCheck() throws InterruptedException, 
ExecutionException {
+    TestCallStreamObserver callStreamObserver = new TestCallStreamObserver();
+    Function<StreamObserver<Integer>, StreamObserver<Integer>> clientFactory =
+        ignored -> callStreamObserver;
+
+    TestStream testStream = newStream(clientFactory);
+    testStream.start();
+    ExecutorService sendExecutor = Executors.newSingleThreadExecutor();
+    Instant reportingThreshold = Instant.now().minus(Duration.millis(1));
+    Future<?> sendFuture =

Review Comment:
   I don't think you need to schedule this on a executor because 
maybeSendHealthCheck is no longer blocking.



##########
runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/AbstractWindmillStreamTest.java:
##########
@@ -110,12 +79,75 @@ public void setMessageCompression(boolean b) {}
     // Sleep a bit to give sendExecutor time to execute the send().
     Uninterruptibles.sleepUninterruptibly(100, TimeUnit.MILLISECONDS);
 
-    sendBlocker.countDown();
+    callStreamObserver.unblockSend();
     
assertThat(sendFuture.get()).isInstanceOf(WindmillStreamShutdownException.class);
   }
 
+  @Test
+  public void testMaybeSendHealthCheck() throws InterruptedException, 
ExecutionException {
+    TestCallStreamObserver callStreamObserver = new TestCallStreamObserver();
+    Function<StreamObserver<Integer>, StreamObserver<Integer>> clientFactory =
+        ignored -> callStreamObserver;
+
+    TestStream testStream = newStream(clientFactory);
+    testStream.start();
+    ExecutorService sendExecutor = Executors.newSingleThreadExecutor();
+    Instant reportingThreshold = Instant.now().minus(Duration.millis(1));
+    Future<?> sendFuture =
+        sendExecutor.submit(() -> 
testStream.maybeSendHealthCheck(reportingThreshold));
+
+    // Sleep a bit to give sendExecutor time to execute the send().
+    Uninterruptibles.sleepUninterruptibly(100, TimeUnit.MILLISECONDS);
+
+    callStreamObserver.unblockSend();
+
+    // Make sure the future completes.
+    sendFuture.get();
+
+    assertThat(testStream.numHealthChecks.get()).isEqualTo(1);
+    testStream.shutdown();
+  }
+
+  @Test
+  public void testMaybeSendHealthCheck_doesNotSendIfLastSendLessThanThreshold()
+      throws ExecutionException, InterruptedException {
+    TestCallStreamObserver callStreamObserver = new TestCallStreamObserver();
+    Function<StreamObserver<Integer>, StreamObserver<Integer>> clientFactory =
+        ignored -> callStreamObserver;
+
+    TestStream testStream = newStream(clientFactory);
+    testStream.start();
+    ExecutorService sendExecutor = Executors.newSingleThreadExecutor();
+    Future<?> sendFuture =
+        sendExecutor.submit(
+            () -> {
+              try {
+                testStream.trySend(1);
+              } catch (WindmillStreamShutdownException e) {
+                throw new RuntimeException(e);
+              }
+
+              // Sleep a bit to give sendExecutor time to execute the send().
+              Uninterruptibles.sleepUninterruptibly(100, 
TimeUnit.MILLISECONDS);
+
+              // Set a really long reporting threshold.
+              Instant reportingThreshold = 
Instant.now().minus(Duration.standardHours(1));
+              // Should not send health checks since we just sent the above 
message.
+              testStream.maybeSendHealthCheck(reportingThreshold);

Review Comment:
   might need to sleep or verify the internal executor for the stream has 
nothing queud before you check no health-checks were sent



##########
runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/AbstractWindmillStreamTest.java:
##########
@@ -110,12 +79,75 @@ public void setMessageCompression(boolean b) {}
     // Sleep a bit to give sendExecutor time to execute the send().
     Uninterruptibles.sleepUninterruptibly(100, TimeUnit.MILLISECONDS);
 
-    sendBlocker.countDown();
+    callStreamObserver.unblockSend();
     
assertThat(sendFuture.get()).isInstanceOf(WindmillStreamShutdownException.class);
   }
 
+  @Test
+  public void testMaybeSendHealthCheck() throws InterruptedException, 
ExecutionException {
+    TestCallStreamObserver callStreamObserver = new TestCallStreamObserver();
+    Function<StreamObserver<Integer>, StreamObserver<Integer>> clientFactory =
+        ignored -> callStreamObserver;
+
+    TestStream testStream = newStream(clientFactory);
+    testStream.start();
+    ExecutorService sendExecutor = Executors.newSingleThreadExecutor();
+    Instant reportingThreshold = Instant.now().minus(Duration.millis(1));
+    Future<?> sendFuture =
+        sendExecutor.submit(() -> 
testStream.maybeSendHealthCheck(reportingThreshold));
+
+    // Sleep a bit to give sendExecutor time to execute the send().
+    Uninterruptibles.sleepUninterruptibly(100, TimeUnit.MILLISECONDS);
+
+    callStreamObserver.unblockSend();
+
+    // Make sure the future completes.
+    sendFuture.get();
+
+    assertThat(testStream.numHealthChecks.get()).isEqualTo(1);

Review Comment:
   not sure if this is going to be flaky since lambda queued by trySend could 
possibly not have actually executed yet. Seems like it could be safer to loop 
while it is not yet 1.



##########
runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/AbstractWindmillStreamTest.java:
##########
@@ -110,12 +79,75 @@ public void setMessageCompression(boolean b) {}
     // Sleep a bit to give sendExecutor time to execute the send().
     Uninterruptibles.sleepUninterruptibly(100, TimeUnit.MILLISECONDS);
 
-    sendBlocker.countDown();
+    callStreamObserver.unblockSend();
     
assertThat(sendFuture.get()).isInstanceOf(WindmillStreamShutdownException.class);
   }
 
+  @Test
+  public void testMaybeSendHealthCheck() throws InterruptedException, 
ExecutionException {
+    TestCallStreamObserver callStreamObserver = new TestCallStreamObserver();
+    Function<StreamObserver<Integer>, StreamObserver<Integer>> clientFactory =
+        ignored -> callStreamObserver;
+
+    TestStream testStream = newStream(clientFactory);
+    testStream.start();
+    ExecutorService sendExecutor = Executors.newSingleThreadExecutor();
+    Instant reportingThreshold = Instant.now().minus(Duration.millis(1));
+    Future<?> sendFuture =
+        sendExecutor.submit(() -> 
testStream.maybeSendHealthCheck(reportingThreshold));
+
+    // Sleep a bit to give sendExecutor time to execute the send().
+    Uninterruptibles.sleepUninterruptibly(100, TimeUnit.MILLISECONDS);
+
+    callStreamObserver.unblockSend();
+
+    // Make sure the future completes.
+    sendFuture.get();
+
+    assertThat(testStream.numHealthChecks.get()).isEqualTo(1);
+    testStream.shutdown();
+  }
+
+  @Test
+  public void testMaybeSendHealthCheck_doesNotSendIfLastSendLessThanThreshold()
+      throws ExecutionException, InterruptedException {
+    TestCallStreamObserver callStreamObserver = new TestCallStreamObserver();
+    Function<StreamObserver<Integer>, StreamObserver<Integer>> clientFactory =
+        ignored -> callStreamObserver;
+
+    TestStream testStream = newStream(clientFactory);
+    testStream.start();
+    ExecutorService sendExecutor = Executors.newSingleThreadExecutor();
+    Future<?> sendFuture =
+        sendExecutor.submit(
+            () -> {
+              try {
+                testStream.trySend(1);
+              } catch (WindmillStreamShutdownException e) {
+                throw new RuntimeException(e);
+              }
+
+              // Sleep a bit to give sendExecutor time to execute the send().
+              Uninterruptibles.sleepUninterruptibly(100, 
TimeUnit.MILLISECONDS);
+
+              // Set a really long reporting threshold.
+              Instant reportingThreshold = 
Instant.now().minus(Duration.standardHours(1));
+              // Should not send health checks since we just sent the above 
message.
+              testStream.maybeSendHealthCheck(reportingThreshold);
+              testStream.maybeSendHealthCheck(reportingThreshold);
+            });
+
+    callStreamObserver.unblockSend();

Review Comment:
   can we just start the test with it unblocked? Seems like you could just 
remove the sendFuture then and just do everything from the main test thread.



-- 
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: github-unsubscr...@beam.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org

Reply via email to