akalash commented on a change in pull request #17473: URL: https://github.com/apache/flink/pull/17473#discussion_r735707775
########## 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: Not actually. The idea is to have a symmetric for the current value. 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) . my idea was to allow increase and decrease by the same number. So according to my implementation 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. But if we keep the maximum value constant(2 * maxBufferSize) this means that we decrease the value relatively slow but increase value pretty fast especially when it is small(which is not so bad for some number of cases). I have actually tried both options. They all work the same in most cases but my current solution is a little less sensitive to spikes when buffer size is small which I decided the most useful case. Example of value change(the difference for the small value even large): decrease = 32768, 29647, 26823, 24268, 21956, 19864 increase (2* currentValue) = 19864, 21755, 23826, 26095, 28580, 31301, 32768 increase (2* maxValue) = 19864, 24213, 31708, 32768 -- 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]
