pnowojski commented on a change in pull request #17229:
URL: https://github.com/apache/flink/pull/17229#discussion_r746647988



##########
File path: 
flink-dstl/flink-dstl-dfs/src/main/java/org/apache/flink/changelog/fs/UploadThrottle.java
##########
@@ -0,0 +1,91 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.changelog.fs;
+
+import org.apache.flink.runtime.io.AvailabilityProvider;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.concurrent.GuardedBy;
+import javax.annotation.concurrent.ThreadSafe;
+
+import java.util.concurrent.CompletableFuture;
+
+/** Helper class to throttle upload requests when the in-flight data size 
limit is exceeded. */
+@ThreadSafe

Review comment:
       Wouldn't this be solved by simplifying this class to a dumb accounting 
class of reserved/seized bytes?
   
   So in `BatchingStateChangeUploader` instead of current calls:
   ```
   uploadThrottle.seizeCapacity(size);
   synchronized (scheduled) {
      foo();
   }
   ```
   ```
   (...)
   uploadThrottle.releaseCapacity(preComputedTaskSize);
   (...)
   ```
   We would have:
   ```
   synchronized (scheduled) {
      while (!uploadThrottle.seizeCapacity(size)) { //return true if 
successfully seized
        scheduled.wait();
      }
      foo();
   }
   ```
   ```
   (...)
   CompletableFuture<?> toNotify = null;
   synchronized (scheduled) {
     if (uploadThrottle.releaseCapacity(preComputedTaskSize)) { // return true 
if released enough to unblock
       toNotify = availabilityHelper.getUnavailableToResetAvailable(); // note 
AvailabilityHelper class in the master
     }
     scheduled.notifyAll();
   }
   if (toNotify != null) {
     toNotify.complete(null);
   }
   (...)
   ```
   
   Code would move a bit between `UploadThrottle` and 
`BatchingStateChangeUploader`, but we would loose the lock without adding 
overall complexity, and as me and @curcur are arguing, we would actually 
simplify and optimise the code with fewer locks.




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