m-trieu commented on code in PR #32774:
URL: https://github.com/apache/beam/pull/32774#discussion_r1844326533


##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/AbstractWindmillStream.java:
##########
@@ -269,31 +312,43 @@ public final boolean awaitTermination(int time, TimeUnit 
unit) throws Interrupte
 
   @Override
   public final Instant startTime() {
-    return new Instant(startTimeMs.get());
+    return new Instant(debugMetrics.getStartTimeMs());
   }
 
   @Override
   public String backendWorkerToken() {
     return backendWorkerToken;
   }
 
+  @SuppressWarnings("GuardedBy")
   @Override
-  public void shutdown() {
-    if (isShutdown.compareAndSet(false, true)) {
-      requestObserver()
-          .onError(new WindmillStreamShutdownException("Explicit call to 
shutdown stream."));
+  public final void shutdown() {
+    // Don't lock on "this" before poisoning the request observer since 
otherwise the observer may
+    // be blocking in send().
+    requestObserver.poison();
+    synchronized (this) {
+      if (!isShutdown) {
+        isShutdown = true;
+        debugMetrics.recordShutdown();
+        shutdownInternal();
+      }
     }
   }
 
-  private void setLastError(String error) {
-    lastError.set(error);
-    lastErrorTime.set(DateTime.now());
-  }
-
-  public static class WindmillStreamShutdownException extends RuntimeException 
{
-    public WindmillStreamShutdownException(String message) {
-      super(message);
+  protected abstract void shutdownInternal();
+
+  /** Returns true if the stream was torn down and should not be restarted 
internally. */
+  private synchronized boolean maybeTearDownStream() {
+    if (requestObserver.hasReceivedPoisonPill()

Review Comment:
   done, realized we weren't calling `break` on the 
`WindmillStreamShutdownException` in `startStream()`



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