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


##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/AbstractWindmillStream.java:
##########
@@ -153,113 +163,172 @@ private static long debugDuration(long nowMs, long 
startMs) {
   protected abstract void startThrottleTimer();
 
   /** Reflects that {@link #shutdown()} was explicitly called. */
-  protected boolean isShutdown() {
-    return isShutdown.get();
-  }
-
-  private StreamObserver<RequestT> requestObserver() {
-    if (requestObserver == null) {
-      throw new NullPointerException(
-          "requestObserver cannot be null. Missing a call to startStream() to 
initialize.");
+  protected boolean hasReceivedShutdownSignal() {
+    synchronized (shutdownLock) {
+      return isShutdown;
     }
-
-    return requestObserver;
   }
 
   /** Send a request to the server. */
   protected final void send(RequestT request) {
-    lastSendTimeMs.set(Instant.now().getMillis());
     synchronized (this) {
-      if (streamClosed.get()) {
+      if (hasReceivedShutdownSignal()) {
+        return;
+      }
+
+      if (streamClosed) {
+        // TODO(m-trieu): throw a more specific exception here (i.e 
StreamClosedException)
         throw new IllegalStateException("Send called on a client closed 
stream.");
       }
 
-      requestObserver().onNext(request);
+      try {
+        verify(!Thread.holdsLock(shutdownLock), "shutdownLock should not be 
held during send.");
+        debugMetrics.recordSend();
+        requestObserver.onNext(request);
+      } catch (StreamObserverCancelledException e) {
+        if (hasReceivedShutdownSignal()) {
+          logger.debug("Stream was shutdown during send.", e);
+          return;
+        }
+
+        requestObserver.onError(e);
+      }
+    }
+  }
+
+  @Override
+  public final void start() {
+    boolean shouldStartStream = false;
+    synchronized (shutdownLock) {
+      if (!isShutdown && !started) {
+        started = true;
+        shouldStartStream = true;
+      }
+    }
+
+    if (shouldStartStream) {
+      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) {
-          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();
+          if (hasReceivedShutdownSignal()) {

Review Comment:
   done



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