m-trieu commented on code in PR #34367: URL: https://github.com/apache/beam/pull/34367#discussion_r2023655268
########## 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(); Review Comment: moved the build out, i think setting it should still hold the lock since (from other comment) -> setting the budget on a stream can affect how many items/bytes we fetch in the header (and subsequent extension calls). wanted to simplify the stream mechanics by preventing setBudget() from being called mid stream creation. That way either we start the stream first, and then call set budget which will update the internal stream budget and send an extension (possibly) OR we set the budget first, and start the stream w/ the new budget. wdyt? -- 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