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


##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/AbstractWindmillStream.java:
##########
@@ -160,80 +179,125 @@ protected boolean isShutdown() {
   private StreamObserver<RequestT> requestObserver() {
     if (requestObserver == null) {
       throw new NullPointerException(
-          "requestObserver cannot be null. Missing a call to startStream() to 
initialize.");
+          "requestObserver cannot be null. Missing a call to start() to 
initialize stream.");
     }
 
     return requestObserver;
   }
 
   /** Send a request to the server. */
   protected final void send(RequestT request) {
-    lastSendTimeMs.set(Instant.now().getMillis());
     synchronized (this) {
+      if (isShutdown()) {
+        return;
+      }
+
       if (streamClosed.get()) {
         throw new IllegalStateException("Send called on a client closed 
stream.");
       }
 
-      requestObserver().onNext(request);
+      try {
+        lastSendTimeMs.set(Instant.now().getMillis());
+        requestObserver.onNext(request);
+      } catch (StreamObserverCancelledException e) {
+        if (isShutdown()) {
+          logger.debug("Stream was closed or shutdown during send.", e);
+          return;
+        }
+
+        requestObserver.onError(e);
+      }
+    }
+  }
+
+  @Override
+  public final void start() {
+    if (!isShutdown.get() && started.compareAndSet(false, true)) {
+      // start() should only be executed once during the lifetime of the 
stream for idempotency and
+      // when shutdown() has not been called.
+      startStream();
     }
   }
 
   /** Starts the underlying stream. */
-  protected final void startStream() {
+  private void startStream() {
     // Add the stream to the registry after it has been fully constructed.
     streamRegistry.add(this);
     while (true) {
       try {
         synchronized (this) {
+          if (isShutdown.get()) {
+            break;
+          }
           startTimeMs.set(Instant.now().getMillis());
           lastResponseTimeMs.set(0);
           streamClosed.set(false);
-          // lazily initialize the requestObserver. Gets reset whenever the 
stream is reopened.
-          requestObserver = requestObserverSupplier.get();
+          requestObserver.reset();
           onNewStream();
           if (clientClosed.get()) {
             halfClose();
           }
           return;

Review Comment:
   we want to ensure we only add the stream once so we do it outside the loop
   if it hits the bottom of the method then we exited the loop w/o returning do 
to shutdown or error and we remove it there



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