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 2021/10/25 12:27:29 UTC

[GitHub] [flink] pnowojski commented on a change in pull request #17473: [FLINK-24470][runtime] EMA calculation move to BufferDebloater

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



##########
File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/bufferdebloat/BufferDebloater.java
##########
@@ -94,10 +97,10 @@ public void recalculateBufferSize(long currentThroughput) {
             return;
         }
 
-        lastBufferSize = newSize;
         for (IndexedInputGate inputGate : inputGates) {
             inputGate.announceBufferSize(newSize);
         }
+        lastBufferSize = newSize;

Review comment:
       Is this change relevant?

##########
File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/bufferdebloat/BufferSizeEMA.java
##########
@@ -0,0 +1,58 @@
+/*
+ * 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.streaming.runtime.tasks.bufferdebloat;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+
+/** Implementation of 'Exponential moving average' algorithm. */
+public class BufferSizeEMA {
+    private final int maxBufferSize;
+    private final int minBufferSize;
+    /** EMA algorithm specific constant which responsible for speed of reaction. */
+    private final double alpha;
+
+    private int lastBufferSize;
+
+    public BufferSizeEMA(int maxBufferSize, int minBufferSize, long numberOfSamples) {
+        this.maxBufferSize = maxBufferSize;
+        this.minBufferSize = minBufferSize;
+        alpha = 2.0 / (numberOfSamples + 1);
+        this.lastBufferSize = maxBufferSize;
+    }
+
+    /**
+     * Calculating the buffer size over total possible buffers size and number of buffers in use.
+     *
+     * @param totalBufferSizeInBytes Total buffers size.
+     * @param totalBuffers Total number of buffers in use.
+     * @return Throughput calculated according to implemented algorithm.
+     */
+    public int calculateBufferSize(long totalBufferSizeInBytes, int totalBuffers) {
+        checkArgument(totalBufferSizeInBytes >= 0, "Size of buffer should be non negative");
+        checkArgument(totalBuffers > 0, "Number of buffers should be positive");
+
+        // Limit the possible buffer size by twice of current buffer size in order to have the same
+        // changes speed for increasing and decreasing the size.

Review comment:
       Can you elaborate on this explanation? Describe what was the problem with sudden throughput spike
   ?

##########
File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/bufferdebloat/BufferSizeEMA.java
##########
@@ -0,0 +1,58 @@
+/*
+ * 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.streaming.runtime.tasks.bufferdebloat;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+
+/** Implementation of 'Exponential moving average' algorithm. */
+public class BufferSizeEMA {
+    private final int maxBufferSize;
+    private final int minBufferSize;
+    /** EMA algorithm specific constant which responsible for speed of reaction. */
+    private final double alpha;
+
+    private int lastBufferSize;
+
+    public BufferSizeEMA(int maxBufferSize, int minBufferSize, long numberOfSamples) {
+        this.maxBufferSize = maxBufferSize;
+        this.minBufferSize = minBufferSize;
+        alpha = 2.0 / (numberOfSamples + 1);
+        this.lastBufferSize = maxBufferSize;
+    }
+
+    /**
+     * Calculating the buffer size over total possible buffers size and number of buffers in use.
+     *
+     * @param totalBufferSizeInBytes Total buffers size.
+     * @param totalBuffers Total number of buffers in use.
+     * @return Throughput calculated according to implemented algorithm.
+     */
+    public int calculateBufferSize(long totalBufferSizeInBytes, int totalBuffers) {
+        checkArgument(totalBufferSizeInBytes >= 0, "Size of buffer should be non negative");
+        checkArgument(totalBuffers > 0, "Number of buffers should be positive");
+
+        // Limit the possible buffer size by twice of current buffer size in order to have the same
+        // changes speed for increasing and decreasing the size.
+        long desirableBufferSize =
+                Math.min(totalBufferSizeInBytes / totalBuffers, 2L * lastBufferSize);

Review comment:
       Shouldn't this be
   ```
   Math.min(totalBufferSizeInBytes / totalBuffers, 2L * maxBufferSize);
   ```
   ?
   Otherwise won't growing from  `minBufferSize` to `maxBufferSize` be actually very slow compared to shrinking from `max` to `min`?




-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org