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


##########
pinot-core/src/main/java/org/apache/pinot/core/transport/DirectOOMHandler.java:
##########
@@ -0,0 +1,100 @@
+/**
+ * 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.pinot.core.transport;
+
+import io.netty.channel.Channel;
+import io.netty.channel.ChannelHandlerContext;
+import io.netty.channel.ChannelInboundHandlerAdapter;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.pinot.spi.exception.QueryCancelledException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+// Handling netty direct memory OOM. In this case there is a great chance that 
multiple channels are receiving
+// large data tables from servers concurrently. We want to close all channels 
to servers to proactively release
+// the direct memory, because the execution of netty threads can deadlock in 
allocating direct memory, in which case
+// no one will reach channelRead0.
+public class DirectOOMHandler extends ChannelInboundHandlerAdapter {
+  private static final Logger LOGGER = 
LoggerFactory.getLogger(DataTableHandler.class);
+  private static final AtomicBoolean DIRECT_OOM_SHUTTING_DOWN = new 
AtomicBoolean(false);
+  private final QueryRouter _queryRouter;
+  private final ServerRoutingInstance _serverRoutingInstance;
+  private final ConcurrentHashMap<ServerRoutingInstance, 
ServerChannels.ServerChannel> _serverToChannelMap;
+
+  public void setSilentShutDown() {
+    _silentShutDown = true;
+  }
+
+  private volatile boolean _silentShutDown = false;
+
+  public DirectOOMHandler(QueryRouter queryRouter, ServerRoutingInstance 
serverRoutingInstance,
+      ConcurrentHashMap<ServerRoutingInstance, ServerChannels.ServerChannel> 
serverToChannelMap) {
+    _queryRouter = queryRouter;
+    _serverRoutingInstance = serverRoutingInstance;
+    _serverToChannelMap = serverToChannelMap;
+  }
+
+  @Override
+  public void channelInactive(ChannelHandlerContext ctx) {
+    // if we are shutting down channels due to direct memory OOM, we short 
circuit the channel inactive
+    if (_silentShutDown) {
+      return;
+    }
+    ctx.fireChannelInactive();
+  }
+
+  @Override
+  public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) 
throws Exception {
+    // catch direct memory oom here
+    if (cause instanceof OutOfMemoryError && 
cause.getMessage().contains("Direct buffer")) {
+      // only one thread can get here and do the shutdown
+      if (DIRECT_OOM_SHUTTING_DOWN.compareAndSet(false, true)) {
+        try {
+          LOGGER.error("Closing ALL channels to servers, as we are running out 
of direct memory "
+              + "while receiving response from {}", _serverRoutingInstance, 
cause);
+          // close all channels to servers
+          ctx.channel().close();
+          _serverToChannelMap.keySet().forEach(serverRoutingInstance -> {
+            Channel channel = 
_serverToChannelMap.get(serverRoutingInstance)._channel;
+            if (channel != null) {
+              DirectOOMHandler directOOMHandler = 
channel.pipeline().get(DirectOOMHandler.class);
+              if (directOOMHandler != null) {
+                directOOMHandler.setSilentShutDown();
+              }
+              channel.close();
+            }
+            _serverToChannelMap.remove(serverRoutingInstance);
+          });
+          _queryRouter.markServerDown(_serverRoutingInstance,

Review Comment:
   In the context of AdaptiveServerSelection:
   If this error is hit, it looks like this codepath will not decrement the 
numInProgressQueries for all servers? 
   Can you please validate that? Looking at the code it looks like we might 
have to invoke the following function inside MarkServerDown.
   ```
             
_serverRoutingStatsManager.recordStatsUponResponseArrival(_requestId, 
entry.getKey().getInstanceId(),
                 _timeoutMs);
   ```
   



##########
pinot-core/src/main/java/org/apache/pinot/core/transport/DirectOOMHandler.java:
##########
@@ -0,0 +1,100 @@
+/**
+ * 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.pinot.core.transport;
+
+import io.netty.channel.Channel;
+import io.netty.channel.ChannelHandlerContext;
+import io.netty.channel.ChannelInboundHandlerAdapter;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.pinot.spi.exception.QueryCancelledException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+// Handling netty direct memory OOM. In this case there is a great chance that 
multiple channels are receiving
+// large data tables from servers concurrently. We want to close all channels 
to servers to proactively release
+// the direct memory, because the execution of netty threads can deadlock in 
allocating direct memory, in which case
+// no one will reach channelRead0.
+public class DirectOOMHandler extends ChannelInboundHandlerAdapter {
+  private static final Logger LOGGER = 
LoggerFactory.getLogger(DataTableHandler.class);
+  private static final AtomicBoolean DIRECT_OOM_SHUTTING_DOWN = new 
AtomicBoolean(false);
+  private final QueryRouter _queryRouter;
+  private final ServerRoutingInstance _serverRoutingInstance;
+  private final ConcurrentHashMap<ServerRoutingInstance, 
ServerChannels.ServerChannel> _serverToChannelMap;
+
+  public void setSilentShutDown() {
+    _silentShutDown = true;
+  }
+
+  private volatile boolean _silentShutDown = false;
+
+  public DirectOOMHandler(QueryRouter queryRouter, ServerRoutingInstance 
serverRoutingInstance,
+      ConcurrentHashMap<ServerRoutingInstance, ServerChannels.ServerChannel> 
serverToChannelMap) {
+    _queryRouter = queryRouter;
+    _serverRoutingInstance = serverRoutingInstance;
+    _serverToChannelMap = serverToChannelMap;
+  }
+
+  @Override
+  public void channelInactive(ChannelHandlerContext ctx) {
+    // if we are shutting down channels due to direct memory OOM, we short 
circuit the channel inactive
+    if (_silentShutDown) {
+      return;
+    }
+    ctx.fireChannelInactive();
+  }
+
+  @Override
+  public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) 
throws Exception {
+    // catch direct memory oom here
+    if (cause instanceof OutOfMemoryError && 
cause.getMessage().contains("Direct buffer")) {
+      // only one thread can get here and do the shutdown
+      if (DIRECT_OOM_SHUTTING_DOWN.compareAndSet(false, true)) {
+        try {
+          LOGGER.error("Closing ALL channels to servers, as we are running out 
of direct memory "
+              + "while receiving response from {}", _serverRoutingInstance, 
cause);
+          // close all channels to servers
+          ctx.channel().close();
+          _serverToChannelMap.keySet().forEach(serverRoutingInstance -> {
+            Channel channel = 
_serverToChannelMap.get(serverRoutingInstance)._channel;

Review Comment:
   Would suggest checking if ServerChannel is not null before this.
   
   IMO, would be better to do the following:
   1. Have a getter in ServerChannel to get the DirectOOMHandler.
   2. Have a method in ServerChannel to close the channel.
   
   This way we could keep all the ServerChannel handling within the class



##########
pinot-core/src/main/java/org/apache/pinot/core/transport/DirectOOMHandler.java:
##########
@@ -0,0 +1,100 @@
+/**
+ * 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.pinot.core.transport;
+
+import io.netty.channel.Channel;
+import io.netty.channel.ChannelHandlerContext;
+import io.netty.channel.ChannelInboundHandlerAdapter;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.pinot.spi.exception.QueryCancelledException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+// Handling netty direct memory OOM. In this case there is a great chance that 
multiple channels are receiving
+// large data tables from servers concurrently. We want to close all channels 
to servers to proactively release
+// the direct memory, because the execution of netty threads can deadlock in 
allocating direct memory, in which case
+// no one will reach channelRead0.
+public class DirectOOMHandler extends ChannelInboundHandlerAdapter {
+  private static final Logger LOGGER = 
LoggerFactory.getLogger(DataTableHandler.class);
+  private static final AtomicBoolean DIRECT_OOM_SHUTTING_DOWN = new 
AtomicBoolean(false);
+  private final QueryRouter _queryRouter;
+  private final ServerRoutingInstance _serverRoutingInstance;
+  private final ConcurrentHashMap<ServerRoutingInstance, 
ServerChannels.ServerChannel> _serverToChannelMap;
+
+  public void setSilentShutDown() {
+    _silentShutDown = true;
+  }
+
+  private volatile boolean _silentShutDown = false;
+
+  public DirectOOMHandler(QueryRouter queryRouter, ServerRoutingInstance 
serverRoutingInstance,
+      ConcurrentHashMap<ServerRoutingInstance, ServerChannels.ServerChannel> 
serverToChannelMap) {
+    _queryRouter = queryRouter;
+    _serverRoutingInstance = serverRoutingInstance;
+    _serverToChannelMap = serverToChannelMap;
+  }
+
+  @Override
+  public void channelInactive(ChannelHandlerContext ctx) {
+    // if we are shutting down channels due to direct memory OOM, we short 
circuit the channel inactive
+    if (_silentShutDown) {
+      return;
+    }
+    ctx.fireChannelInactive();
+  }
+
+  @Override
+  public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) 
throws Exception {
+    // catch direct memory oom here
+    if (cause instanceof OutOfMemoryError && 
cause.getMessage().contains("Direct buffer")) {
+      // only one thread can get here and do the shutdown
+      if (DIRECT_OOM_SHUTTING_DOWN.compareAndSet(false, true)) {

Review Comment:
   I might have a basic question to make sure I understand this correctly:
   
   - Each channel will have it's own instance (object) of DirectOOMHandler.
   - DIRECT_OOM_SHUTTING_DOWN offers protection only when multiple threads are 
trying to execute the below code within a single instance.
   - If there are multiple instances (from multiple channels) of 
DirectOOMHandler trying to close each other's channels, would this interleaving 
lead to any problems? 
   
   
   Eg: ServerChannel_1's DirectOOMHandler trying to close ServerChannel_2. At 
the same time, ServerChannel_2's DirectOOMHandler trying to close 
ServerChannel_1.
   
   
   
   
   
   
   



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