dpcollins-google commented on a change in pull request #16901:
URL: https://github.com/apache/beam/pull/16901#discussion_r813218264



##########
File path: 
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/BoundedQueueExecutor.java
##########
@@ -18,62 +18,126 @@
 package org.apache.beam.runners.dataflow.worker.util;
 
 import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.Semaphore;
 import java.util.concurrent.ThreadFactory;
 import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
+import 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.util.concurrent.Monitor;
+import 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.util.concurrent.Monitor.Guard;
 
-/** Executor that blocks on execute() if its queue is full. */
+/** An executor for executing work on windmill items. */
 @SuppressWarnings({
   "nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 })
-public class BoundedQueueExecutor extends ThreadPoolExecutor {
-  private static class ReducableSemaphore extends Semaphore {
-    ReducableSemaphore(int permits) {
-      super(permits);
-    }
+public class BoundedQueueExecutor {
+  private final ThreadPoolExecutor executor;
+  private final int maximumElementsOutstanding;
+  private final long maximumBytesOutstanding;
 
-    @Override
-    public void reducePermits(int permits) {
-      super.reducePermits(permits);
-    }
-  }
-
-  private ReducableSemaphore semaphore;
+  private final Monitor monitor = new Monitor();
+  private int elementsOutstanding = 0;
+  private long bytesOutstanding = 0;
 
   public BoundedQueueExecutor(
       int maximumPoolSize,
       long keepAliveTime,
       TimeUnit unit,
-      int maximumQueueSize,
+      int maximumElementsOutstanding,
+      long maximumBytesOutstanding,
       ThreadFactory threadFactory) {
-    super(
-        maximumPoolSize,
-        maximumPoolSize,
-        keepAliveTime,
-        unit,
-        new LinkedBlockingQueue<Runnable>(),
-        threadFactory);
-    this.semaphore = new ReducableSemaphore(maximumQueueSize);
-    allowCoreThreadTimeOut(true);
+    executor =
+        new ThreadPoolExecutor(
+            maximumPoolSize,
+            maximumPoolSize,
+            keepAliveTime,
+            unit,
+            new LinkedBlockingQueue<>(),
+            threadFactory);
+    executor.allowCoreThreadTimeOut(true);
+    this.maximumElementsOutstanding = maximumElementsOutstanding;
+    this.maximumBytesOutstanding = maximumBytesOutstanding;
   }
 
-  // Before adding a Runnable to the queue, acquire the semaphore.
-  @Override
-  public void execute(Runnable r) {
-    semaphore.acquireUninterruptibly();
-    super.execute(r);
+  // Before adding a Work to the queue, check that there are enough bytes of 
space or no other
+  // outstanding elements of work.
+  public void execute(Runnable work, long workBytes) {
+    monitor.enterWhenUninterruptibly(
+        new Guard(monitor) {
+          @Override
+          public boolean isSatisfied() {
+            return elementsOutstanding == 0
+                || (bytesAvailable() >= workBytes
+                    && elementsOutstanding < maximumElementsOutstanding);
+          }
+        });
+    Runnable workRunnable =
+        () -> {
+          try {
+            work.run();
+          } finally {
+            monitor.enter();
+            --elementsOutstanding;
+            bytesOutstanding -= workBytes;
+            monitor.leave();
+          }
+        };
+    try {
+      bytesOutstanding += workBytes;
+      ++elementsOutstanding;
+      executor.execute(workRunnable);
+    } finally {
+      monitor.leave();
+    }
   }
 
   // Forcibly add something to the queue, ignoring the length limit.
   public void forceExecute(Runnable r) {
-    semaphore.reducePermits(1);
-    super.execute(r);
+    executor.execute(r);

Review comment:
       This is now enforced.




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