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


##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetDataStream.java:
##########
@@ -342,62 +401,86 @@ 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 (this) {
+        if (isShutdown) {
+          throw shutdownExceptionFor(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) throws WindmillStreamShutdownException {
+    try {
+      sendBatch(batch);
+      synchronized (this) {
+        if (isShutdown) {
+          throw shutdownExceptionFor(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) throws 
WindmillStreamShutdownException {
+    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) {
+      if (isShutdown) {
+        throw shutdownExceptionFor(batch);
+      }
+
+      for (QueuedRequest request : batch.requestsReadOnly()) {
         // Map#put returns null if there was no previous mapping for the key, 
meaning we have not
         // seen it before.
-        verify(pending.put(request.id(), request.getResponseStream()) == null);
+        verify(
+            pending.put(request.id(), request.getResponseStream()) == null,
+            "Request already sent.");
       }
-      try {
-        send(batchedRequest);
-      } catch (IllegalStateException e) {
+
+      if (!trySend(batch.asGetDataRequest())) {
         // The stream broke before this call went through; onNewStream will 
retry the fetch.
-        LOG.warn("GetData stream broke before call started.", e);
+        LOG.warn("GetData stream broke before call started.");
       }
     }
   }
 
-  @SuppressWarnings("argument")
-  private StreamingGetDataRequest flushToBatch(List<QueuedRequest> requests) {
-    // Put all global data requests first because there is only a single 
repeated field for
-    // request ids and the initial ids correspond to global data requests if 
they are present.
-    requests.sort(QueuedRequest.globalRequestsFirst());
-    StreamingGetDataRequest.Builder builder = 
StreamingGetDataRequest.newBuilder();
-    for (QueuedRequest request : requests) {
-      request.addToStreamingGetDataRequest(builder);
-    }
-    return builder.build();
+  private synchronized void verify(boolean condition, String message) {

Review Comment:
   done



##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcCommitWorkStream.java:
##########
@@ -144,41 +149,57 @@ protected boolean hasPendingRequests() {
   }
 
   @Override
-  public void sendHealthCheck() {
+  public void sendHealthCheck() throws WindmillStreamShutdownException {
     if (hasPendingRequests()) {
       StreamingCommitWorkRequest.Builder builder = 
StreamingCommitWorkRequest.newBuilder();
       builder.addCommitChunkBuilder().setRequestId(HEARTBEAT_REQUEST_ID);
-      send(builder.build());
+      trySend(builder.build());
     }
   }
 
   @Override
   protected void onResponse(StreamingCommitResponse response) {
     commitWorkThrottleTimer.stop();
 
-    RuntimeException finalException = null;
+    CommitCompletionException failures = new CommitCompletionException();
     for (int i = 0; i < response.getRequestIdCount(); ++i) {

Review Comment:
   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