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



##########
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:
       `seizeCapacity` needs to wait for signal of availability **and** release 
the lock while waiting.
   For the former, `availabilityFuture`  could be used.
   For the latter, I don't see any simple way to achieve it with external 
synchronization.
   
   Another issue with external synchronization is that it wouldn't be possible 
to complete the availability future outside of the syncrhonized block (at least 
without adding some more complexity).
   
   WDYT?
   
   Also in my opinion it's easier to reason about this class when it's 
thread-safe.

##########
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
+class UploadThrottle implements AvailabilityProvider {
+    private static final Logger LOG = 
LoggerFactory.getLogger(UploadThrottle.class);
+
+    private final Object lock = new Object();
+    private final long maxBytesInFlight;
+
+    @GuardedBy("lock")
+    private long inFlightBytesCounter = 0;
+
+    @GuardedBy("lock")
+    private CompletableFuture<?> availabilityFuture = 
AvailabilityProvider.AVAILABLE;
+
+    UploadThrottle(long maxBytesInFlight) {
+        this.maxBytesInFlight = maxBytesInFlight;
+    }
+
+    /**
+     * Seize <b>bytes</b> capacity, waiting if needed. Called by the Task 
thread.
+     *
+     * @throws InterruptedException
+     */
+    public void seizeCapacity(long bytes) throws InterruptedException {
+        synchronized (lock) {
+            while (!hasCapacity()) {
+                LOG.info("In flight data size threshold exceeded: {}", 
maxBytesInFlight);
+                lock.wait();
+            }
+            inFlightBytesCounter += bytes;
+            if (!hasCapacity() && isAvailable()) {
+                availabilityFuture = new CompletableFuture<>();
+            }
+        }
+    }
+
+    /**
+     * Release capacity, signalling waiting threads, if any. Called by {@link
+     * BatchingStateChangeUploader} (IO thread).
+     */
+    public void releaseCapacity(long bytes) {
+        synchronized (lock) {
+            inFlightBytesCounter -= bytes;
+            if (hasCapacity() && !isAvailable()) {
+                availabilityFuture.complete(null);

Review comment:
       Good point!

##########
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
+class UploadThrottle implements AvailabilityProvider {
+    private static final Logger LOG = 
LoggerFactory.getLogger(UploadThrottle.class);
+
+    private final Object lock = new Object();
+    private final long maxBytesInFlight;
+
+    @GuardedBy("lock")
+    private long inFlightBytesCounter = 0;
+
+    @GuardedBy("lock")
+    private CompletableFuture<?> availabilityFuture = 
AvailabilityProvider.AVAILABLE;
+
+    UploadThrottle(long maxBytesInFlight) {
+        this.maxBytesInFlight = maxBytesInFlight;
+    }
+
+    /**
+     * Seize <b>bytes</b> capacity, waiting if needed. Called by the Task 
thread.
+     *
+     * @throws InterruptedException
+     */
+    public void seizeCapacity(long bytes) throws InterruptedException {
+        synchronized (lock) {
+            while (!hasCapacity()) {
+                LOG.info("In flight data size threshold exceeded: {}", 
maxBytesInFlight);
+                lock.wait();
+            }

Review comment:
       As discussed offline, I'll remove the use of 
`backPressuredTimePerSecond` in this PR, so any waiting on changelog will be 
reported as busy (which is consistent with other state backends currently).
   
   The mertric then should be addressed in FLINK-24402.

##########
File path: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java
##########
@@ -387,6 +390,13 @@ protected StreamTask(
 
         this.stateBackend = createStateBackend();
         this.checkpointStorage = createCheckpointStorage(stateBackend);
+        this.changelogWriterAvailabilityProvider =
+                environment.getTaskStateManager().getStateChangelogStorage() 
== null
+                        ? () -> AvailabilityProvider.AVAILABLE // todo: 
benchmark
+                        : environment
+                                .getTaskStateManager()
+                                .getStateChangelogStorage()
+                                .getAvailabilityProvider();

Review comment:
       After thinking a bit more about `getStateBackendAvailabilityProvider`, 
it doesn't seem the right API.
   
   Inside a task, there can be multiple operators and therefore multiple state 
backends.
   In general (without sharing the underlying writers), each can have its own 
availability provider.
   So such API (`getTaskStateManager().getStateBackendAvailabilityProvider()`) 
wouldn't make sense.
   
   WDYT?




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