arunpandianp commented on code in PR #32774:
URL: https://github.com/apache/beam/pull/32774#discussion_r1816257623
##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetDataStream.java:
##########
@@ -342,62 +394,91 @@ 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 (isShutdown()) {
+ handleShutdown(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 (isShutdown()) {
+ handleShutdown(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) {
+ LOG.error("Error occurred sending batch.", e);
Review Comment:
This might log on every shutdown. We could remove this and rely on logging
unexpected exceptions upstream
##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetDataStream.java:
##########
@@ -302,38 +324,68 @@ public void appendSpecificHtml(PrintWriter writer) {
}
private <ResponseT> ResponseT issueRequest(QueuedRequest request,
ParseFn<ResponseT> parseFn) {
- while (true) {
+ while (!isShutdown()) {
request.resetResponseStream();
try {
queueRequestAndWait(request);
return parseFn.parse(request.getResponseStream());
- } catch (CancellationException e) {
- // Retry issuing the request since the response stream was cancelled.
- continue;
+ } catch (AppendableInputStream.InvalidInputStreamStateException |
CancellationException e) {
+ handleShutdown(request, e);
+ if (!(e instanceof CancellationException)) {
+ throw e;
+ }
} catch (IOException e) {
LOG.error("Parsing GetData response failed: ", e);
- continue;
} catch (InterruptedException e) {
Thread.currentThread().interrupt();
+ handleShutdown(request, e);
throw new RuntimeException(e);
} finally {
pending.remove(request.id());
}
}
+
+ throw new WindmillStreamShutdownException(
+ "Cannot send request=[" + request + "] on closed stream.");
+ }
+
+ private void handleShutdown(QueuedRequest request, Throwable... causes) {
+ if (isShutdown()) {
+ WindmillStreamShutdownException shutdownException =
+ new WindmillStreamShutdownException(
+ "Cannot send request=[" + request + "] on closed stream.");
+
+ for (Throwable cause : causes) {
+ shutdownException.addSuppressed(cause);
+ }
+
+ throw shutdownException;
+ }
+ }
+
+ private void handleShutdown(QueuedBatch batch) {
+ if (isShutdown()) {
+ throw new WindmillStreamShutdownException(
+ "Stream was closed when attempting to send " + batch.requestsCount()
+ " requests.");
+ }
}
private void queueRequestAndWait(QueuedRequest request) throws
InterruptedException {
QueuedBatch batch;
boolean responsibleForSend = false;
- CountDownLatch waitForSendLatch = null;
- synchronized (batches) {
+ @Nullable QueuedBatch prevBatch = null;
+ synchronized (shutdownLock) {
+ if (isShutdown()) {
+ handleShutdown(request);
Review Comment:
```suggestion
throw shutdownException(request);
```
throwing here would be more readable, than relying on handleShutdown to
throw.
--
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]