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


##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/AbstractWindmillStream.java:
##########
@@ -153,113 +147,150 @@ 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();
+  protected synchronized boolean hasReceivedShutdownSignal() {
+    return isShutdown;
   }
 
-  private StreamObserver<RequestT> requestObserver() {
-    if (requestObserver == null) {
-      throw new NullPointerException(
-          "requestObserver cannot be null. Missing a call to startStream() to 
initialize.");
-    }
-
-    return requestObserver;
+  /** Send a request to the server. */
+  protected final synchronized void send(RequestT request)
+      throws StreamClosedException, WindmillStreamShutdownException {
+    debugMetrics.recordSend();
+    requestObserver.onNext(request);
   }
 
-  /** Send a request to the server. */
-  protected final void send(RequestT request) {
-    lastSendTimeMs.set(Instant.now().getMillis());
+  @Override
+  public final void start() {
+    boolean shouldStartStream = false;
     synchronized (this) {
-      if (streamClosed.get()) {
-        throw new IllegalStateException("Send called on a client closed 
stream.");
+      if (!isShutdown && !started) {
+        started = true;
+        shouldStartStream = true;
       }
+    }
 
-      requestObserver().onNext(request);
+    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();
+          debugMetrics.recordStart();
+          requestObserver.reset();
           onNewStream();
-          if (clientClosed.get()) {
+          if (clientClosed) {
             halfClose();
           }
           return;
         }
+      } catch (WindmillStreamShutdownException e) {
+        logger.debug("Stream was shutdown while creating new stream.", e);
       } catch (Exception e) {
-        LOG.error("Failed to create new stream, retrying: ", e);
+        logger.error("Failed to create new stream, retrying: ", e);
         try {
           long sleep = backoff.nextBackOffMillis();
-          sleepUntil.set(Instant.now().getMillis() + sleep);
-          Thread.sleep(sleep);
-        } catch (InterruptedException | IOException i) {
+          debugMetrics.recordSleep(sleep);
+          sleeper.sleep(sleep);
+        } catch (InterruptedException ie) {
+          Thread.currentThread().interrupt();
+          logger.info(
+              "Interrupted during {} creation backoff. The stream will not be 
created.",

Review Comment:
   we shutdown the executor in maybeTeardownStream() so this would clean up any 
dangling tasks in the executor
   lets shutdown the stream
   
   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