m-trieu commented on code in PR #32774: URL: https://github.com/apache/beam/pull/32774#discussion_r1830060117
########## runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetDataStream.java: ########## @@ -342,62 +391,88 @@ private void queueRequestAndWait(QueuedRequest request) throws InterruptedExcept batch.addRequest(request); } if (responsibleForSend) { - if (waitForSendLatch == null) { + if (prevBatch == null) { // If there was not a previous batch wait a little while to improve // batching. - Thread.sleep(1); + sleeper.sleep(1); } else { - waitForSendLatch.await(); + prevBatch.waitForSendOrFailNotification(); } // Finalize the batch so that no additional requests will be added. Leave the batch in the // queue so that a subsequent batch will wait for its completion. - synchronized (batches) { - verify(batch == batches.peekFirst()); + synchronized (shutdownLock) { + if (hasReceivedShutdownSignal()) { + throw shutdownException(batch); + } + + verify(batch == batches.peekFirst(), "GetDataStream request batch removed before send()."); batch.markFinalized(); } - sendBatch(batch.requests()); - synchronized (batches) { - verify(batch == batches.pollFirst()); + trySendBatch(batch); + } else { + // Wait for this batch to be sent before parsing the response. + batch.waitForSendOrFailNotification(); + } + } + + void trySendBatch(QueuedBatch batch) { + try { + sendBatch(batch); + synchronized (shutdownLock) { + if (hasReceivedShutdownSignal()) { + throw shutdownException(batch); + } + + verify( + batch == batches.pollFirst(), + "Sent GetDataStream request batch removed before send() was complete."); } // Notify all waiters with requests in this batch as well as the sender // of the next batch (if one exists). - batch.countDown(); - } else { - // Wait for this batch to be sent before parsing the response. - batch.await(); + batch.notifySent(); + } catch (Exception e) { + // Free waiters if the send() failed. + batch.notifyFailed(); + // Propagate the exception to the calling thread. + throw e; } } - @SuppressWarnings("NullableProblems") - private void sendBatch(List<QueuedRequest> requests) { - StreamingGetDataRequest batchedRequest = flushToBatch(requests); + private void sendBatch(QueuedBatch batch) { + if (batch.isEmpty()) { + return; + } + + // Synchronization of pending inserts is necessary with send to ensure duplicates are not + // sent on stream reconnect. synchronized (this) { - // Synchronization of pending inserts is necessary with send to ensure duplicates are not - // sent on stream reconnect. - for (QueuedRequest request : requests) { + synchronized (shutdownLock) { + // shutdown() clears pending, once the stream is shutdown, prevent values from being added Review Comment: whoops that was supposed to be in done w/in the lock 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: github-unsubscr...@beam.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org