HBASE-16614 Use daemon thread for netty event loop

This commit is contained in:
zhangduo 2016-09-12 16:32:03 +08:00
parent 105bfc7d7c
commit 0860bdb675
2 changed files with 7 additions and 2 deletions

View File

@ -21,6 +21,7 @@ import io.netty.channel.Channel;
import io.netty.channel.EventLoopGroup;
import io.netty.channel.nio.NioEventLoopGroup;
import io.netty.channel.socket.nio.NioSocketChannel;
import io.netty.util.concurrent.DefaultThreadFactory;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.util.Pair;
@ -32,6 +33,8 @@ import org.apache.hadoop.hbase.util.Pair;
class DefaultNettyEventLoopConfig {
public static final Pair<EventLoopGroup, Class<? extends Channel>> GROUP_AND_CHANNEL_CLASS = Pair
.<EventLoopGroup, Class<? extends Channel>> newPair(new NioEventLoopGroup(),
.<EventLoopGroup, Class<? extends Channel>> newPair(
new NioEventLoopGroup(0,
new DefaultThreadFactory("Default-IPC-NioEventLoopGroup", true, Thread.MAX_PRIORITY)),
NioSocketChannel.class);
}

View File

@ -21,6 +21,7 @@ import io.netty.channel.Channel;
import io.netty.channel.EventLoopGroup;
import io.netty.channel.nio.NioEventLoopGroup;
import io.netty.channel.socket.nio.NioSocketChannel;
import io.netty.util.concurrent.DefaultThreadFactory;
import java.io.IOException;
import java.net.SocketAddress;
@ -51,7 +52,8 @@ public class NettyRpcClient extends AbstractRpcClient<NettyRpcConnection> {
.getEventLoopConfig(conf);
if (groupAndChannelClass == null) {
// Use our own EventLoopGroup.
this.group = new NioEventLoopGroup();
this.group = new NioEventLoopGroup(0,
new DefaultThreadFactory("IPC-NioEventLoopGroup", true, Thread.MAX_PRIORITY));
this.channelClass = NioSocketChannel.class;
this.shutdownGroupWhenClose = true;
} else {