forked from apache/flink
-
Notifications
You must be signed in to change notification settings - Fork 12
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[FLINK-24470][streaming] Added EMA calculation for buffer size inside…
… BufferDebloater
- Loading branch information
Showing
5 changed files
with
192 additions
and
12 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
68 changes: 68 additions & 0 deletions
68
...a/src/main/java/org/apache/flink/streaming/runtime/tasks/bufferdebloat/BufferSizeEMA.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,68 @@ | ||
/* | ||
* 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 | ||
* | ||
* https://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"); | ||
|
||
// Since the result value is always limited by max buffer size while the instant value is | ||
// potentially unlimited. It can lead to an instant change from min to max value in case | ||
// when the instant value is significantly larger than the possible max value. | ||
// The solution is to limit the instant buffer size by twice of current buffer size in order | ||
// to have the same growth and shrink speeds. for example if the instant value is equal to 0 | ||
// and the current value is 16000 we can decrease it at maximum by 1600(suppose alfa=0.1) . | ||
// The idea is to allow increase and decrease size by the same number. So if the instant | ||
// value would be large(for example 100000) it will be possible to increase the current | ||
// value by 1600(the same as decreasing) because the limit will be 2 * currentValue = 32000. | ||
// Example of change speed: | ||
// growing = 32768, 29647, 26823, 24268, 21956, 19864 | ||
// shrinking = 19864, 21755, 23826, 26095, 28580, 31301, 32768 | ||
long desirableBufferSize = | ||
Math.min(totalBufferSizeInBytes / totalBuffers, 2L * lastBufferSize); | ||
|
||
lastBufferSize += alpha * (desirableBufferSize - lastBufferSize); | ||
return lastBufferSize = Math.max(minBufferSize, Math.min(lastBufferSize, maxBufferSize)); | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
91 changes: 91 additions & 0 deletions
91
...c/test/java/org/apache/flink/streaming/runtime/tasks/bufferdebloat/BufferSizeEMATest.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -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 | ||
* | ||
* https://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 org.apache.flink.util.TestLogger; | ||
|
||
import org.junit.Test; | ||
|
||
import static org.hamcrest.MatcherAssert.assertThat; | ||
import static org.hamcrest.Matchers.is; | ||
|
||
/** Test for {@link BufferSizeEMA}. */ | ||
public class BufferSizeEMATest extends TestLogger { | ||
|
||
@Test | ||
public void testCalculationBufferSize() { | ||
BufferSizeEMA calculator = new BufferSizeEMA(200, 10, 3); | ||
|
||
// The result value seeks to the bottom limit but it will take a while until it reaches it. | ||
assertThat(calculator.calculateBufferSize(111, 13), is(104)); | ||
assertThat(calculator.calculateBufferSize(107, 7), is(59)); | ||
assertThat(calculator.calculateBufferSize(107, 7), is(37)); | ||
assertThat(calculator.calculateBufferSize(107, 7), is(26)); | ||
assertThat(calculator.calculateBufferSize(107, 7), is(20)); | ||
assertThat(calculator.calculateBufferSize(107, 7), is(17)); | ||
assertThat(calculator.calculateBufferSize(107, 13), is(12)); | ||
assertThat(calculator.calculateBufferSize(107, 13), is(10)); | ||
|
||
// Upgrade | ||
assertThat(calculator.calculateBufferSize(333, 1), is(15)); | ||
assertThat(calculator.calculateBufferSize(333, 1), is(22)); | ||
assertThat(calculator.calculateBufferSize(333, 1), is(33)); | ||
assertThat(calculator.calculateBufferSize(333, 1), is(49)); | ||
assertThat(calculator.calculateBufferSize(333, 1), is(73)); | ||
assertThat(calculator.calculateBufferSize(333, 1), is(109)); | ||
} | ||
|
||
@Test | ||
public void testSizeGreaterThanMaxSize() { | ||
BufferSizeEMA calculator = new BufferSizeEMA(200, 10, 3); | ||
|
||
// Decrease value to less than max. | ||
assertThat(calculator.calculateBufferSize(0, 1), is(100)); | ||
|
||
// Impossible to exceed maximum. | ||
assertThat(calculator.calculateBufferSize(1000, 1), is(150)); | ||
assertThat(calculator.calculateBufferSize(1000, 1), is(200)); | ||
assertThat(calculator.calculateBufferSize(1000, 1), is(200)); | ||
} | ||
|
||
@Test | ||
public void testSizeLessThanMinSize() { | ||
BufferSizeEMA calculator = new BufferSizeEMA(200, 10, 3); | ||
|
||
// Impossible to less than min. | ||
assertThat(calculator.calculateBufferSize(0, 1), is(100)); | ||
assertThat(calculator.calculateBufferSize(0, 1), is(50)); | ||
assertThat(calculator.calculateBufferSize(0, 1), is(25)); | ||
assertThat(calculator.calculateBufferSize(0, 1), is(12)); | ||
assertThat(calculator.calculateBufferSize(0, 1), is(10)); | ||
assertThat(calculator.calculateBufferSize(0, 1), is(10)); | ||
} | ||
|
||
@Test(expected = IllegalArgumentException.class) | ||
public void testNegativeTotalSize() { | ||
BufferSizeEMA calculator = new BufferSizeEMA(100, 200, 2); | ||
calculator.calculateBufferSize(-1, 1); | ||
} | ||
|
||
@Test(expected = IllegalArgumentException.class) | ||
public void testZeroBuffers() { | ||
BufferSizeEMA calculator = new BufferSizeEMA(100, 200, 2); | ||
calculator.calculateBufferSize(1, 0); | ||
} | ||
} |