arunpandianp commented on code in PR #35523:
URL: https://github.com/apache/beam/pull/35523#discussion_r2204045126


##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/AbstractWindmillStream.java:
##########
@@ -472,17 +505,74 @@ private void clearPhysicalStreamForDebug() {
     currentPhysicalStreamForDebug.set(null);
   }
 
+  private void onHalfClosePhysicalStreamTimeout(PhysicalStreamHandler handler) 
{
+    synchronized (this) {
+      if (currentPhysicalStream != handler || clientClosed || isShutdown) {
+        return;
+      }
+      try {
+        handler.streamDebugMetrics.recordHalfClose();
+        closingPhysicalStreams.add(handler);
+        currentPhysicalStream = null;
+        halfCloseFuture = null; // This is the currently running future.
+        clearPhysicalStreamForDebug();
+        requestObserver.onCompleted();
+      } catch (Exception e) {
+        // XXX figure out
+      }
+      try {
+        @NonNull PhysicalStreamHandler streamHandler = newResponseHandler();

Review Comment:
   do we need this logic to recreate streams here? If so, can we call 
startStream()? 
   
   I think `requestObserver.onCompleted();` will recreate the streams and we 
don't need to recreate streams here.



##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/AbstractWindmillStream.java:
##########
@@ -122,13 +140,23 @@ protected AbstractWindmillStream(
       StreamObserverFactory streamObserverFactory,
       Set<AbstractWindmillStream<?, ?>> streamRegistry,
       int logEveryNStreamFailures,
-      String backendWorkerToken) {
+      String backendWorkerToken,
+      Duration halfClosePhysicalStreamAfter) {
+    checkArgument(!halfClosePhysicalStreamAfter.isNegative());
     this.backendWorkerToken = backendWorkerToken;
     this.physicalStreamFactory =
         (StreamObserver<ResponseT> observer) -> 
streamObserverFactory.from(clientFactory, observer);
     this.physicalStreamDeadlineSeconds = 
streamObserverFactory.getDeadlineSeconds();
+    if (!halfClosePhysicalStreamAfter.isZero()
+        && 
halfClosePhysicalStreamAfter.compareTo(Duration.ofSeconds(physicalStreamDeadlineSeconds))
+            >= 0) {
+      logger.debug("Not attempting to half-close cleanly as stream deadline is 
shorter.");
+      halfClosePhysicalStreamAfter = Duration.ZERO;
+    }
+    this.halfClosePhysicalStreamAfter = halfClosePhysicalStreamAfter;
+    this.closingPhysicalStreams = new HashSet<>();

Review Comment:
   will IdentityHashMap be a better candidate.



##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/AbstractWindmillStream.java:
##########
@@ -472,17 +505,74 @@ private void clearPhysicalStreamForDebug() {
     currentPhysicalStreamForDebug.set(null);
   }
 
+  private void onHalfClosePhysicalStreamTimeout(PhysicalStreamHandler handler) 
{
+    synchronized (this) {
+      if (currentPhysicalStream != handler || clientClosed || isShutdown) {
+        return;
+      }
+      try {
+        handler.streamDebugMetrics.recordHalfClose();
+        closingPhysicalStreams.add(handler);
+        currentPhysicalStream = null;
+        halfCloseFuture = null; // This is the currently running future.
+        clearPhysicalStreamForDebug();
+        requestObserver.onCompleted();
+      } catch (Exception e) {
+        // XXX figure out
+      }
+      try {
+        @NonNull PhysicalStreamHandler streamHandler = newResponseHandler();
+        streamHandler.streamDebugMetrics.recordStart();
+        currentPhysicalStream = streamHandler;
+        currentPhysicalStreamForDebug.set(currentPhysicalStream);
+        requestObserver.reset(physicalStreamFactory.apply(new 
ResponseObserver(streamHandler)));
+        onFlushPending(true);
+      } catch (Exception e) {
+        // XXX figure out
+      }
+    }
+  }
+
   private void onPhysicalStreamCompletion(Status status, PhysicalStreamHandler 
handler) {
     synchronized (this) {
+      final boolean wasActiveStream = currentPhysicalStream == handler;
       if (currentPhysicalStream == handler) {
         clearPhysicalStreamForDebug();
         currentPhysicalStream = null;
+        if (halfCloseFuture != null) {
+          halfCloseFuture.cancel(false);
+          halfCloseFuture = null;

Review Comment:
   these steps are repeated in a few places, consider moving them to a function 
like `clearPhysicalStream()` 



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