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


##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/harness/WindmillStreamSender.java:
##########
@@ -126,35 +148,49 @@ private static GetWorkRequest 
withRequestBudget(GetWorkRequest request, GetWorkB
   }
 
   synchronized void start() {
-    if (!started.get()) {
+    if (isRunning.compareAndSet(false, true)) {
       checkState(!streamStarter.isShutdown(), "WindmillStreamSender has 
already been shutdown.");
-
       // Start these 3 streams in parallel since they each may perform 
blocking IO.
+      CountDownLatch waitForInitialStream = new CountDownLatch(1);
+      streamStarter.execute(() -> getWorkStreamLoop(waitForInitialStream));
       CompletableFuture.allOf(
-              CompletableFuture.runAsync(getWorkStream::start, streamStarter),
               CompletableFuture.runAsync(getDataStream::start, streamStarter),
               CompletableFuture.runAsync(commitWorkStream::start, 
streamStarter))
           .join();
+      try {
+        waitForInitialStream.await();
+      } catch (InterruptedException e) {
+        close();
+        LOG.error("GetWorkStream to {} was never able to start.", 
backendWorkerToken);
+        throw new IllegalStateException("GetWorkStream unable to start 
aborting.", e);
+      }
       workCommitter.start();
-      started.set(true);
     }
   }
 
   @Override
   public synchronized void close() {
+    isRunning.set(false);
     streamStarter.shutdownNow();
-    getWorkStream.shutdown();
     getDataStream.shutdown();
     workCommitter.stop();
     commitWorkStream.shutdown();
   }
 
   @Override
   public void setBudget(long items, long bytes) {
-    GetWorkBudget budget = 
GetWorkBudget.builder().setItems(items).setBytes(bytes).build();
-    getWorkBudget.set(budget);
-    if (started.get()) {
-      getWorkStream.setBudget(budget);
+    synchronized (activeGetWorkStream) {
+      GetWorkBudget budget = 
GetWorkBudget.builder().setItems(items).setBytes(bytes).build();
+      getWorkBudget.set(budget);
+      if (isRunning.get()) {

Review Comment:
   setBudget on the `GetWorkStream` object may try to send an extension, should 
we make the `GetWorkStream` check isStarted before doing that? or this would 
prevent that



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