gortiz commented on code in PR #11496:
URL: https://github.com/apache/pinot/pull/11496#discussion_r1319597575


##########
pinot-core/src/main/java/org/apache/pinot/core/transport/DataTableHandler.java:
##########
@@ -83,5 +98,12 @@ protected void channelRead0(ChannelHandlerContext ctx, 
ByteBuf msg) {
   public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) {
     LOGGER.error("Caught exception while handling response from server: {}", 
_serverRoutingInstance, cause);
     
_brokerMetrics.addMeteredGlobalValue(BrokerMeter.RESPONSE_FETCH_EXCEPTIONS, 1);
+    if (cause instanceof java.lang.OutOfMemoryError) {

Review Comment:
   I don't like the solution where the broker sends the max allowed bytes to 
the server. I mean, it is fine. It would also be fine to have a static limit in 
the server.
   
   But that would not be a complete solution because the problem will still be 
there in case we have a lot of servers responding. The broker should protect 
itself against that issue by not reading the whole payload in one single 
buffer. Instead we can read the message in a fixed size buffer, read there the 
number of bytes that have been sent and only if that number is acceptable, read 
the rest of the data. Whether the size is acceptable can be a static decision 
(and therefore it would be affected by the number of servers) or can be 
calculated at runtime (taking into account the amount of direct memory 
consumed).
   
   That is the kind of work where Netty low level control really shines.
   
   I assume this approach would require more time to be implemented, so I'm 
fine with adding a catch on OOM, but then I strongly would recommend to do not 
add it here but in a new specific Channel. There we can explain why we are 
adding that channel and clearly indicate it is a patch before we have time to 
add a better solution. It would also make it clear that the OOM we want to 
catch thrown when DataTableHandler is deserializing the buffer but before that.
   
   Also, remember that `exceptionCaught` is being called, but it is deprecated. 
We should implement the correct method instead (see 
[https://netty.io/4.1/api/io/netty/channel/ChannelHandler.html#exceptionCaught-io.netty.channel.ChannelHandlerContext-java.lang.Throwable-](https://netty.io/4.1/api/io/netty/channel/ChannelHandler.html#exceptionCaught-io.netty.channel.ChannelHandlerContext-java.lang.Throwable-))



-- 
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: commits-unsubscr...@pinot.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org
For additional commands, e-mail: commits-h...@pinot.apache.org

Reply via email to