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



##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannel.java
##########
@@ -74,6 +75,8 @@
     /** The current backoff (in ms). */
     private int currentBackoff;
 
+    protected int currentBufferSize;

Review comment:
       On the second thought, you are storing the `currentBufferSize` only for 
the testing purposes? If so, couldn't you add this code to `TestInputChannel` 
instead of modifying the production code?

##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannel.java
##########
@@ -74,6 +75,8 @@
     /** The current backoff (in ms). */
     private int currentBackoff;
 
+    protected int currentBufferSize;

Review comment:
       I think changes from this file should belong to the following commit

##########
File path: 
docs/layouts/shortcodes/generated/all_taskmanager_network_section.html
##########
@@ -27,17 +27,35 @@
             <td>Boolean flag to enable/disable more detailed metrics about 
inbound/outbound network queue lengths.</td>
         </tr>
         <tr>
-            
<td><h5>taskmanager.network.memory.automatic-buffer-adjustment.period</h5></td>
-            <td style="word-wrap: break-word;">500</td>
-            <td>Integer</td>
-            <td>The minimum period of time after which the buffer size will be 
automatically adjusted to a new value if required. The low value provides a 
fast reaction to the load fluctuation but can influence the performance.</td>
+            <td><h5>taskmanager.network.memory.buffer-debloat.enabled</h5></td>
+            <td style="word-wrap: break-word;">false</td>
+            <td>Boolean</td>
+            <td>The switch of the automatic buffered debloating feature. If 
enabled the amount of in-flight data will be adjusted automatically accordingly 
to the measured throughput.</td>
         </tr>
         <tr>
             
<td><h5>taskmanager.network.memory.automatic-buffer-adjustment.samples</h5></td>
             <td style="word-wrap: break-word;">20</td>
             <td>Integer</td>
             <td>The number of the last buffer size values that will be taken 
for the correct calculation of the new one.</td>
         </tr>
+        <tr>
+            <td><h5>taskmanager.network.memory.buffer-debloat.enabled</h5></td>
+            <td style="word-wrap: break-word;">true</td>
+            <td>Boolean</td>
+            <td>The switch of the automatic buffered debloating feature. If 
enabled the amount of in-flight data will be adjusted automatically accordingly 
to the measured throughput.</td>
+        </tr>

Review comment:
       Why are there two entires for this? Was it some merge conflict? Does it 
need to be regenerated once more?




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