You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2020/02/21 11:59:32 UTC

[GitHub] [flink-statefun] igalshilman commented on a change in pull request #29: [FLINK-16063] Add simple backpressure

igalshilman commented on a change in pull request #29:  [FLINK-16063] Add simple backpressure
URL: https://github.com/apache/flink-statefun/pull/29#discussion_r382544542
 
 

 ##########
 File path: statefun-flink/statefun-flink-core/src/main/java/org/apache/flink/statefun/flink/core/backpressure/ThresholdBackPressureValve.java
 ##########
 @@ -0,0 +1,76 @@
+/*
+ * 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.statefun.flink.core.backpressure;
+
+import it.unimi.dsi.fastutil.objects.ObjectOpenHashMap;
+import java.util.Objects;
+import org.apache.flink.statefun.sdk.Address;
+
+/** A simple Threshold based {@link BackPressureValve}. */
+public final class ThresholdBackPressureValve implements BackPressureValve {
+  private final int maximumPendingAsynchronousOperations;
+
+  /**
+   * a set of address that had explicitly requested to stop processing any new inputs (via {@link
+   * AsyncWaiter#awaitAsyncOperationComplete()}. Note that this is a set implemented on top of a
+   * map, and the value (Boolean) has no meaning.
+   */
+  private final ObjectOpenHashMap<Address, Boolean> blockedAddressSet =
+      new ObjectOpenHashMap<>(1024);
+
+  private int pendingAsynchronousOperationsCount;
+
+  /**
+   * Constructs a ThresholdBackPressureValve.
+   *
+   * @param maximumPendingAsynchronousOperations the total allowed async operations to be inflight
+   *     per StreamTask, or {@code -1} to disable back pressure.
+   */
+  public ThresholdBackPressureValve(int maximumPendingAsynchronousOperations) {
+    this.maximumPendingAsynchronousOperations = maximumPendingAsynchronousOperations;
+  }
+
+  public boolean shouldBackPressure() {
+    return totalPendingAsyncOperationsAtCapacity() || hasBlockedAddress();
+  }
+
+  public void blockAddress(Address address) {
 
 Review comment:
   Hi @tzulitai, your understanding is correct, If a function would request to block without actually registering any async operations then it would remain blocked. 
   This is an acceptable behaviour since this is our internal API to be used by `HttpFunction` and `GrpcFunction`.
   
   If we would like it to be a part of the SDK then we would have to make sure that we track every async operation registered per each address.
   
   Let me update the JavaDoc of the `AsyncWaiter` with your observation.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services