未验证 提交 34a04aca 编写于 作者: C CalvinKirs 提交者: GitHub

[Improvement][remote] improvement netty eventLoopGroup (#3580)

* [Improvement][remote] improvement netty eventLoopGroup
If you are running on linux you can use EpollEventLoopGroup and so get better performance, less GC and have more advanced features that are only available on linux.

* reformat code

* utility classes should not have public constructors

* reformat code

* reformat code

* reformat code

* add test

* code style

* add test

* add test

* fix test error

* add test

* test

* test

* add test config

* add test config
上级 93660f4d
...@@ -18,10 +18,17 @@ ...@@ -18,10 +18,17 @@
package org.apache.dolphinscheduler.remote; package org.apache.dolphinscheduler.remote;
import io.netty.bootstrap.Bootstrap; import io.netty.bootstrap.Bootstrap;
import io.netty.channel.*; import io.netty.channel.Channel;
import io.netty.channel.ChannelFuture;
import io.netty.channel.ChannelFutureListener;
import io.netty.channel.ChannelInitializer;
import io.netty.channel.ChannelOption;
import io.netty.channel.EventLoopGroup;
import io.netty.channel.epoll.EpollEventLoopGroup;
import io.netty.channel.nio.NioEventLoopGroup; import io.netty.channel.nio.NioEventLoopGroup;
import io.netty.channel.socket.SocketChannel; import io.netty.channel.socket.SocketChannel;
import io.netty.channel.socket.nio.NioSocketChannel; import io.netty.channel.socket.nio.NioSocketChannel;
import org.apache.dolphinscheduler.remote.codec.NettyDecoder; import org.apache.dolphinscheduler.remote.codec.NettyDecoder;
import org.apache.dolphinscheduler.remote.codec.NettyEncoder; import org.apache.dolphinscheduler.remote.codec.NettyEncoder;
import org.apache.dolphinscheduler.remote.command.Command; import org.apache.dolphinscheduler.remote.command.Command;
...@@ -38,6 +45,8 @@ import org.apache.dolphinscheduler.remote.processor.NettyRequestProcessor; ...@@ -38,6 +45,8 @@ import org.apache.dolphinscheduler.remote.processor.NettyRequestProcessor;
import org.apache.dolphinscheduler.remote.utils.Host; import org.apache.dolphinscheduler.remote.utils.Host;
import org.apache.dolphinscheduler.remote.utils.CallerThreadExecutePolicy; import org.apache.dolphinscheduler.remote.utils.CallerThreadExecutePolicy;
import org.apache.dolphinscheduler.remote.utils.NamedThreadFactory; import org.apache.dolphinscheduler.remote.utils.NamedThreadFactory;
import org.apache.dolphinscheduler.remote.utils.NettyUtils;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
...@@ -47,7 +56,7 @@ import java.util.concurrent.atomic.AtomicBoolean; ...@@ -47,7 +56,7 @@ import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicInteger;
/** /**
* remoting netty client * remoting netty client
*/ */
public class NettyRemotingClient { public class NettyRemotingClient {
...@@ -59,7 +68,7 @@ public class NettyRemotingClient { ...@@ -59,7 +68,7 @@ public class NettyRemotingClient {
private final Bootstrap bootstrap = new Bootstrap(); private final Bootstrap bootstrap = new Bootstrap();
/** /**
* encoder * encoder
*/ */
private final NettyEncoder encoder = new NettyEncoder(); private final NettyEncoder encoder = new NettyEncoder();
...@@ -69,57 +78,69 @@ public class NettyRemotingClient { ...@@ -69,57 +78,69 @@ public class NettyRemotingClient {
private final ConcurrentHashMap<Host, Channel> channels = new ConcurrentHashMap(128); private final ConcurrentHashMap<Host, Channel> channels = new ConcurrentHashMap(128);
/** /**
* started flag * started flag
*/ */
private final AtomicBoolean isStarted = new AtomicBoolean(false); private final AtomicBoolean isStarted = new AtomicBoolean(false);
/** /**
* worker group * worker group
*/ */
private final NioEventLoopGroup workerGroup; private final EventLoopGroup workerGroup;
/** /**
* client config * client config
*/ */
private final NettyClientConfig clientConfig; private final NettyClientConfig clientConfig;
/** /**
* saync semaphore * saync semaphore
*/ */
private final Semaphore asyncSemaphore = new Semaphore(200, true); private final Semaphore asyncSemaphore = new Semaphore(200, true);
/** /**
* callback thread executor * callback thread executor
*/ */
private final ExecutorService callbackExecutor; private final ExecutorService callbackExecutor;
/** /**
* client handler * client handler
*/ */
private final NettyClientHandler clientHandler; private final NettyClientHandler clientHandler;
/** /**
* response future executor * response future executor
*/ */
private final ScheduledExecutorService responseFutureExecutor; private final ScheduledExecutorService responseFutureExecutor;
/** /**
* client init * client init
*
* @param clientConfig client config * @param clientConfig client config
*/ */
public NettyRemotingClient(final NettyClientConfig clientConfig){ public NettyRemotingClient(final NettyClientConfig clientConfig) {
this.clientConfig = clientConfig; this.clientConfig = clientConfig;
this.workerGroup = new NioEventLoopGroup(clientConfig.getWorkerThreads(), new ThreadFactory() { if (NettyUtils.useEpoll()) {
private AtomicInteger threadIndex = new AtomicInteger(0); this.workerGroup = new EpollEventLoopGroup(clientConfig.getWorkerThreads(), new ThreadFactory() {
private AtomicInteger threadIndex = new AtomicInteger(0);
@Override @Override
public Thread newThread(Runnable r) { public Thread newThread(Runnable r) {
return new Thread(r, String.format("NettyClient_%d", this.threadIndex.incrementAndGet())); return new Thread(r, String.format("NettyClient_%d", this.threadIndex.incrementAndGet()));
} }
}); });
} else {
this.workerGroup = new NioEventLoopGroup(clientConfig.getWorkerThreads(), new ThreadFactory() {
private AtomicInteger threadIndex = new AtomicInteger(0);
@Override
public Thread newThread(Runnable r) {
return new Thread(r, String.format("NettyClient_%d", this.threadIndex.incrementAndGet()));
}
});
}
this.callbackExecutor = new ThreadPoolExecutor(5, 10, 1, TimeUnit.MINUTES, this.callbackExecutor = new ThreadPoolExecutor(5, 10, 1, TimeUnit.MINUTES,
new LinkedBlockingQueue<>(1000), new NamedThreadFactory("CallbackExecutor", 10), new LinkedBlockingQueue<>(1000), new NamedThreadFactory("CallbackExecutor", 10),
new CallerThreadExecutePolicy()); new CallerThreadExecutePolicy());
this.clientHandler = new NettyClientHandler(this, callbackExecutor); this.clientHandler = new NettyClientHandler(this, callbackExecutor);
this.responseFutureExecutor = Executors.newSingleThreadScheduledExecutor(new NamedThreadFactory("ResponseFutureExecutor")); this.responseFutureExecutor = Executors.newSingleThreadScheduledExecutor(new NamedThreadFactory("ResponseFutureExecutor"));
...@@ -128,26 +149,26 @@ public class NettyRemotingClient { ...@@ -128,26 +149,26 @@ public class NettyRemotingClient {
} }
/** /**
* start * start
*/ */
private void start(){ private void start() {
this.bootstrap this.bootstrap
.group(this.workerGroup) .group(this.workerGroup)
.channel(NioSocketChannel.class) .channel(NioSocketChannel.class)
.option(ChannelOption.SO_KEEPALIVE, clientConfig.isSoKeepalive()) .option(ChannelOption.SO_KEEPALIVE, clientConfig.isSoKeepalive())
.option(ChannelOption.TCP_NODELAY, clientConfig.isTcpNoDelay()) .option(ChannelOption.TCP_NODELAY, clientConfig.isTcpNoDelay())
.option(ChannelOption.SO_SNDBUF, clientConfig.getSendBufferSize()) .option(ChannelOption.SO_SNDBUF, clientConfig.getSendBufferSize())
.option(ChannelOption.SO_RCVBUF, clientConfig.getReceiveBufferSize()) .option(ChannelOption.SO_RCVBUF, clientConfig.getReceiveBufferSize())
.handler(new ChannelInitializer<SocketChannel>() { .handler(new ChannelInitializer<SocketChannel>() {
@Override @Override
public void initChannel(SocketChannel ch) throws Exception { public void initChannel(SocketChannel ch) throws Exception {
ch.pipeline().addLast( ch.pipeline().addLast(
new NettyDecoder(), new NettyDecoder(),
clientHandler, clientHandler,
encoder); encoder);
} }
}); });
this.responseFutureExecutor.scheduleAtFixedRate(new Runnable() { this.responseFutureExecutor.scheduleAtFixedRate(new Runnable() {
@Override @Override
public void run() { public void run() {
...@@ -159,10 +180,11 @@ public class NettyRemotingClient { ...@@ -159,10 +180,11 @@ public class NettyRemotingClient {
} }
/** /**
* async send * async send
* @param host host *
* @param command command * @param host host
* @param timeoutMillis timeoutMillis * @param command command
* @param timeoutMillis timeoutMillis
* @param invokeCallback callback function * @param invokeCallback callback function
* @throws InterruptedException * @throws InterruptedException
* @throws RemotingException * @throws RemotingException
...@@ -182,22 +204,22 @@ public class NettyRemotingClient { ...@@ -182,22 +204,22 @@ public class NettyRemotingClient {
* control concurrency number * control concurrency number
*/ */
boolean acquired = this.asyncSemaphore.tryAcquire(timeoutMillis, TimeUnit.MILLISECONDS); boolean acquired = this.asyncSemaphore.tryAcquire(timeoutMillis, TimeUnit.MILLISECONDS);
if(acquired){ if (acquired) {
final ReleaseSemaphore releaseSemaphore = new ReleaseSemaphore(this.asyncSemaphore); final ReleaseSemaphore releaseSemaphore = new ReleaseSemaphore(this.asyncSemaphore);
/** /**
* response future * response future
*/ */
final ResponseFuture responseFuture = new ResponseFuture(opaque, final ResponseFuture responseFuture = new ResponseFuture(opaque,
timeoutMillis, timeoutMillis,
invokeCallback, invokeCallback,
releaseSemaphore); releaseSemaphore);
try { try {
channel.writeAndFlush(command).addListener(new ChannelFutureListener(){ channel.writeAndFlush(command).addListener(new ChannelFutureListener() {
@Override @Override
public void operationComplete(ChannelFuture future) throws Exception { public void operationComplete(ChannelFuture future) throws Exception {
if(future.isSuccess()){ if (future.isSuccess()) {
responseFuture.setSendOk(true); responseFuture.setSendOk(true);
return; return;
} else { } else {
...@@ -207,28 +229,29 @@ public class NettyRemotingClient { ...@@ -207,28 +229,29 @@ public class NettyRemotingClient {
responseFuture.putResponse(null); responseFuture.putResponse(null);
try { try {
responseFuture.executeInvokeCallback(); responseFuture.executeInvokeCallback();
} catch (Throwable ex){ } catch (Throwable ex) {
logger.error("execute callback error", ex); logger.error("execute callback error", ex);
} finally{ } finally {
responseFuture.release(); responseFuture.release();
} }
} }
}); });
} catch (Throwable ex){ } catch (Throwable ex) {
responseFuture.release(); responseFuture.release();
throw new RemotingException(String.format("send command to host: %s failed", host), ex); throw new RemotingException(String.format("send command to host: %s failed", host), ex);
} }
} else{ } else {
String message = String.format("try to acquire async semaphore timeout: %d, waiting thread num: %d, total permits: %d", String message = String.format("try to acquire async semaphore timeout: %d, waiting thread num: %d, total permits: %d",
timeoutMillis, asyncSemaphore.getQueueLength(), asyncSemaphore.availablePermits()); timeoutMillis, asyncSemaphore.getQueueLength(), asyncSemaphore.availablePermits());
throw new RemotingTooMuchRequestException(message); throw new RemotingTooMuchRequestException(message);
} }
} }
/** /**
* sync send * sync send
* @param host host *
* @param command command * @param host host
* @param command command
* @param timeoutMillis timeoutMillis * @param timeoutMillis timeoutMillis
* @return command * @return command
* @throws InterruptedException * @throws InterruptedException
...@@ -244,7 +267,7 @@ public class NettyRemotingClient { ...@@ -244,7 +267,7 @@ public class NettyRemotingClient {
channel.writeAndFlush(command).addListener(new ChannelFutureListener() { channel.writeAndFlush(command).addListener(new ChannelFutureListener() {
@Override @Override
public void operationComplete(ChannelFuture future) throws Exception { public void operationComplete(ChannelFuture future) throws Exception {
if(future.isSuccess()){ if (future.isSuccess()) {
responseFuture.setSendOk(true); responseFuture.setSendOk(true);
return; return;
} else { } else {
...@@ -259,10 +282,10 @@ public class NettyRemotingClient { ...@@ -259,10 +282,10 @@ public class NettyRemotingClient {
* sync wait for result * sync wait for result
*/ */
Command result = responseFuture.waitResponse(); Command result = responseFuture.waitResponse();
if(result == null){ if (result == null) {
if(responseFuture.isSendOK()){ if (responseFuture.isSendOK()) {
throw new RemotingTimeoutException(host.toString(), timeoutMillis, responseFuture.getCause()); throw new RemotingTimeoutException(host.toString(), timeoutMillis, responseFuture.getCause());
} else{ } else {
throw new RemotingException(host.toString(), responseFuture.getCause()); throw new RemotingException(host.toString(), responseFuture.getCause());
} }
} }
...@@ -270,8 +293,9 @@ public class NettyRemotingClient { ...@@ -270,8 +293,9 @@ public class NettyRemotingClient {
} }
/** /**
* send task * send task
* @param host host *
* @param host host
* @param command command * @param command command
* @throws RemotingException * @throws RemotingException
*/ */
...@@ -296,33 +320,35 @@ public class NettyRemotingClient { ...@@ -296,33 +320,35 @@ public class NettyRemotingClient {
} }
/** /**
* register processor * register processor
*
* @param commandType command type * @param commandType command type
* @param processor processor * @param processor processor
*/ */
public void registerProcessor(final CommandType commandType, final NettyRequestProcessor processor) { public void registerProcessor(final CommandType commandType, final NettyRequestProcessor processor) {
this.registerProcessor(commandType, processor, null); this.registerProcessor(commandType, processor, null);
} }
/** /**
* register processor * register processor
* *
* @param commandType command type * @param commandType command type
* @param processor processor * @param processor processor
* @param executor thread executor * @param executor thread executor
*/ */
public void registerProcessor(final CommandType commandType, final NettyRequestProcessor processor, final ExecutorService executor) { public void registerProcessor(final CommandType commandType, final NettyRequestProcessor processor, final ExecutorService executor) {
this.clientHandler.registerProcessor(commandType, processor, executor); this.clientHandler.registerProcessor(commandType, processor, executor);
} }
/** /**
* get channel * get channel
*
* @param host * @param host
* @return * @return
*/ */
public Channel getChannel(Host host) { public Channel getChannel(Host host) {
Channel channel = channels.get(host); Channel channel = channels.get(host);
if(channel != null && channel.isActive()){ if (channel != null && channel.isActive()) {
return channel; return channel;
} }
return createChannel(host, true); return createChannel(host, true);
...@@ -330,17 +356,18 @@ public class NettyRemotingClient { ...@@ -330,17 +356,18 @@ public class NettyRemotingClient {
/** /**
* create channel * create channel
* @param host host *
* @param host host
* @param isSync sync flag * @param isSync sync flag
* @return channel * @return channel
*/ */
public Channel createChannel(Host host, boolean isSync) { public Channel createChannel(Host host, boolean isSync) {
ChannelFuture future; ChannelFuture future;
try { try {
synchronized (bootstrap){ synchronized (bootstrap) {
future = bootstrap.connect(new InetSocketAddress(host.getIp(), host.getPort())); future = bootstrap.connect(new InetSocketAddress(host.getIp(), host.getPort()));
} }
if(isSync){ if (isSync) {
future.sync(); future.sync();
} }
if (future.isSuccess()) { if (future.isSuccess()) {
...@@ -358,16 +385,16 @@ public class NettyRemotingClient { ...@@ -358,16 +385,16 @@ public class NettyRemotingClient {
* close * close
*/ */
public void close() { public void close() {
if(isStarted.compareAndSet(true, false)){ if (isStarted.compareAndSet(true, false)) {
try { try {
closeChannels(); closeChannels();
if(workerGroup != null){ if (workerGroup != null) {
this.workerGroup.shutdownGracefully(); this.workerGroup.shutdownGracefully();
} }
if(callbackExecutor != null){ if (callbackExecutor != null) {
this.callbackExecutor.shutdownNow(); this.callbackExecutor.shutdownNow();
} }
if(this.responseFutureExecutor != null){ if (this.responseFutureExecutor != null) {
this.responseFutureExecutor.shutdownNow(); this.responseFutureExecutor.shutdownNow();
} }
} catch (Exception ex) { } catch (Exception ex) {
...@@ -378,9 +405,9 @@ public class NettyRemotingClient { ...@@ -378,9 +405,9 @@ public class NettyRemotingClient {
} }
/** /**
* close channels * close channels
*/ */
private void closeChannels(){ private void closeChannels() {
for (Channel channel : this.channels.values()) { for (Channel channel : this.channels.values()) {
channel.close(); channel.close();
} }
...@@ -389,11 +416,12 @@ public class NettyRemotingClient { ...@@ -389,11 +416,12 @@ public class NettyRemotingClient {
/** /**
* close channel * close channel
*
* @param host host * @param host host
*/ */
public void closeChannel(Host host){ public void closeChannel(Host host) {
Channel channel = this.channels.remove(host); Channel channel = this.channels.remove(host);
if(channel != null){ if (channel != null) {
channel.close(); channel.close();
} }
} }
......
...@@ -22,9 +22,12 @@ import io.netty.channel.ChannelFuture; ...@@ -22,9 +22,12 @@ import io.netty.channel.ChannelFuture;
import io.netty.channel.ChannelInitializer; import io.netty.channel.ChannelInitializer;
import io.netty.channel.ChannelOption; import io.netty.channel.ChannelOption;
import io.netty.channel.ChannelPipeline; import io.netty.channel.ChannelPipeline;
import io.netty.channel.EventLoopGroup;
import io.netty.channel.epoll.EpollEventLoopGroup;
import io.netty.channel.nio.NioEventLoopGroup; import io.netty.channel.nio.NioEventLoopGroup;
import io.netty.channel.socket.nio.NioServerSocketChannel; import io.netty.channel.socket.nio.NioServerSocketChannel;
import io.netty.channel.socket.nio.NioSocketChannel; import io.netty.channel.socket.nio.NioSocketChannel;
import org.apache.dolphinscheduler.remote.codec.NettyDecoder; import org.apache.dolphinscheduler.remote.codec.NettyDecoder;
import org.apache.dolphinscheduler.remote.codec.NettyEncoder; import org.apache.dolphinscheduler.remote.codec.NettyEncoder;
import org.apache.dolphinscheduler.remote.command.CommandType; import org.apache.dolphinscheduler.remote.command.CommandType;
...@@ -32,6 +35,8 @@ import org.apache.dolphinscheduler.remote.config.NettyServerConfig; ...@@ -32,6 +35,8 @@ import org.apache.dolphinscheduler.remote.config.NettyServerConfig;
import org.apache.dolphinscheduler.remote.handler.NettyServerHandler; import org.apache.dolphinscheduler.remote.handler.NettyServerHandler;
import org.apache.dolphinscheduler.remote.processor.NettyRequestProcessor; import org.apache.dolphinscheduler.remote.processor.NettyRequestProcessor;
import org.apache.dolphinscheduler.remote.utils.Constants; import org.apache.dolphinscheduler.remote.utils.Constants;
import org.apache.dolphinscheduler.remote.utils.NettyUtils;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
...@@ -42,44 +47,44 @@ import java.util.concurrent.atomic.AtomicBoolean; ...@@ -42,44 +47,44 @@ import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicInteger;
/** /**
* remoting netty server * remoting netty server
*/ */
public class NettyRemotingServer { public class NettyRemotingServer {
private final Logger logger = LoggerFactory.getLogger(NettyRemotingServer.class); private final Logger logger = LoggerFactory.getLogger(NettyRemotingServer.class);
/** /**
* server bootstrap * server bootstrap
*/ */
private final ServerBootstrap serverBootstrap = new ServerBootstrap(); private final ServerBootstrap serverBootstrap = new ServerBootstrap();
/** /**
* encoder * encoder
*/ */
private final NettyEncoder encoder = new NettyEncoder(); private final NettyEncoder encoder = new NettyEncoder();
/** /**
* default executor * default executor
*/ */
private final ExecutorService defaultExecutor = Executors.newFixedThreadPool(Constants.CPUS); private final ExecutorService defaultExecutor = Executors.newFixedThreadPool(Constants.CPUS);
/** /**
* boss group * boss group
*/ */
private final NioEventLoopGroup bossGroup; private final EventLoopGroup bossGroup;
/** /**
* worker group * worker group
*/ */
private final NioEventLoopGroup workGroup; private final EventLoopGroup workGroup;
/** /**
* server config * server config
*/ */
private final NettyServerConfig serverConfig; private final NettyServerConfig serverConfig;
/** /**
* server handler * server handler
*/ */
private final NettyServerHandler serverHandler = new NettyServerHandler(this); private final NettyServerHandler serverHandler = new NettyServerHandler(this);
...@@ -89,59 +94,78 @@ public class NettyRemotingServer { ...@@ -89,59 +94,78 @@ public class NettyRemotingServer {
private final AtomicBoolean isStarted = new AtomicBoolean(false); private final AtomicBoolean isStarted = new AtomicBoolean(false);
/** /**
* server init * server init
* *
* @param serverConfig server config * @param serverConfig server config
*/ */
public NettyRemotingServer(final NettyServerConfig serverConfig){ public NettyRemotingServer(final NettyServerConfig serverConfig) {
this.serverConfig = serverConfig; this.serverConfig = serverConfig;
if (NettyUtils.useEpoll()) {
this.bossGroup = new EpollEventLoopGroup(1, new ThreadFactory() {
private AtomicInteger threadIndex = new AtomicInteger(0);
this.bossGroup = new NioEventLoopGroup(1, new ThreadFactory() { @Override
private AtomicInteger threadIndex = new AtomicInteger(0); public Thread newThread(Runnable r) {
return new Thread(r, String.format("NettyServerBossThread_%d", this.threadIndex.incrementAndGet()));
}
});
@Override this.workGroup = new EpollEventLoopGroup(serverConfig.getWorkerThread(), new ThreadFactory() {
public Thread newThread(Runnable r) { private AtomicInteger threadIndex = new AtomicInteger(0);
return new Thread(r, String.format("NettyServerBossThread_%d", this.threadIndex.incrementAndGet()));
}
});
this.workGroup = new NioEventLoopGroup(serverConfig.getWorkerThread(), new ThreadFactory() { @Override
private AtomicInteger threadIndex = new AtomicInteger(0); public Thread newThread(Runnable r) {
return new Thread(r, String.format("NettyServerWorkerThread_%d", this.threadIndex.incrementAndGet()));
}
});
} else {
this.bossGroup = new NioEventLoopGroup(1, new ThreadFactory() {
private AtomicInteger threadIndex = new AtomicInteger(0);
@Override
public Thread newThread(Runnable r) {
return new Thread(r, String.format("NettyServerBossThread_%d", this.threadIndex.incrementAndGet()));
}
});
@Override this.workGroup = new NioEventLoopGroup(serverConfig.getWorkerThread(), new ThreadFactory() {
public Thread newThread(Runnable r) { private AtomicInteger threadIndex = new AtomicInteger(0);
return new Thread(r, String.format("NettyServerWorkerThread_%d", this.threadIndex.incrementAndGet()));
} @Override
}); public Thread newThread(Runnable r) {
return new Thread(r, String.format("NettyServerWorkerThread_%d", this.threadIndex.incrementAndGet()));
}
});
}
} }
/** /**
* server start * server start
*/ */
public void start(){ public void start() {
if (isStarted.compareAndSet(false, true)) { if (isStarted.compareAndSet(false, true)) {
this.serverBootstrap this.serverBootstrap
.group(this.bossGroup, this.workGroup) .group(this.bossGroup, this.workGroup)
.channel(NioServerSocketChannel.class) .channel(NioServerSocketChannel.class)
.option(ChannelOption.SO_REUSEADDR, true) .option(ChannelOption.SO_REUSEADDR, true)
.option(ChannelOption.SO_BACKLOG, serverConfig.getSoBacklog()) .option(ChannelOption.SO_BACKLOG, serverConfig.getSoBacklog())
.childOption(ChannelOption.SO_KEEPALIVE, serverConfig.isSoKeepalive()) .childOption(ChannelOption.SO_KEEPALIVE, serverConfig.isSoKeepalive())
.childOption(ChannelOption.TCP_NODELAY, serverConfig.isTcpNoDelay()) .childOption(ChannelOption.TCP_NODELAY, serverConfig.isTcpNoDelay())
.childOption(ChannelOption.SO_SNDBUF, serverConfig.getSendBufferSize()) .childOption(ChannelOption.SO_SNDBUF, serverConfig.getSendBufferSize())
.childOption(ChannelOption.SO_RCVBUF, serverConfig.getReceiveBufferSize()) .childOption(ChannelOption.SO_RCVBUF, serverConfig.getReceiveBufferSize())
.childHandler(new ChannelInitializer<NioSocketChannel>() { .childHandler(new ChannelInitializer<NioSocketChannel>() {
@Override @Override
protected void initChannel(NioSocketChannel ch) throws Exception { protected void initChannel(NioSocketChannel ch) throws Exception {
initNettyChannel(ch); initNettyChannel(ch);
} }
}); });
ChannelFuture future; ChannelFuture future;
try { try {
future = serverBootstrap.bind(serverConfig.getListenPort()).sync(); future = serverBootstrap.bind(serverConfig.getListenPort()).sync();
} catch (Exception e) { } catch (Exception e) {
logger.error("NettyRemotingServer bind fail {}, exit",e.getMessage(), e); logger.error("NettyRemotingServer bind fail {}, exit", e.getMessage(), e);
throw new RuntimeException(String.format("NettyRemotingServer bind %s fail", serverConfig.getListenPort())); throw new RuntimeException(String.format("NettyRemotingServer bind %s fail", serverConfig.getListenPort()));
} }
if (future.isSuccess()) { if (future.isSuccess()) {
...@@ -155,11 +179,12 @@ public class NettyRemotingServer { ...@@ -155,11 +179,12 @@ public class NettyRemotingServer {
} }
/** /**
* init netty channel * init netty channel
*
* @param ch socket channel * @param ch socket channel
* @throws Exception * @throws Exception
*/ */
private void initNettyChannel(NioSocketChannel ch) throws Exception{ private void initNettyChannel(NioSocketChannel ch) throws Exception {
ChannelPipeline pipeline = ch.pipeline(); ChannelPipeline pipeline = ch.pipeline();
pipeline.addLast("encoder", encoder); pipeline.addLast("encoder", encoder);
pipeline.addLast("decoder", new NettyDecoder()); pipeline.addLast("decoder", new NettyDecoder());
...@@ -167,27 +192,29 @@ public class NettyRemotingServer { ...@@ -167,27 +192,29 @@ public class NettyRemotingServer {
} }
/** /**
* register processor * register processor
*
* @param commandType command type * @param commandType command type
* @param processor processor * @param processor processor
*/ */
public void registerProcessor(final CommandType commandType, final NettyRequestProcessor processor) { public void registerProcessor(final CommandType commandType, final NettyRequestProcessor processor) {
this.registerProcessor(commandType, processor, null); this.registerProcessor(commandType, processor, null);
} }
/** /**
* register processor * register processor
* *
* @param commandType command type * @param commandType command type
* @param processor processor * @param processor processor
* @param executor thread executor * @param executor thread executor
*/ */
public void registerProcessor(final CommandType commandType, final NettyRequestProcessor processor, final ExecutorService executor) { public void registerProcessor(final CommandType commandType, final NettyRequestProcessor processor, final ExecutorService executor) {
this.serverHandler.registerProcessor(commandType, processor, executor); this.serverHandler.registerProcessor(commandType, processor, executor);
} }
/** /**
* get default thread executor * get default thread executor
*
* @return thread executor * @return thread executor
*/ */
public ExecutorService getDefaultExecutor() { public ExecutorService getDefaultExecutor() {
...@@ -195,12 +222,12 @@ public class NettyRemotingServer { ...@@ -195,12 +222,12 @@ public class NettyRemotingServer {
} }
public void close() { public void close() {
if(isStarted.compareAndSet(true, false)){ if (isStarted.compareAndSet(true, false)) {
try { try {
if(bossGroup != null){ if (bossGroup != null) {
this.bossGroup.shutdownGracefully(); this.bossGroup.shutdownGracefully();
} }
if(workGroup != null){ if (workGroup != null) {
this.workGroup.shutdownGracefully(); this.workGroup.shutdownGracefully();
} }
defaultExecutor.shutdown(); defaultExecutor.shutdown();
......
...@@ -42,4 +42,14 @@ public class Constants { ...@@ -42,4 +42,14 @@ public class Constants {
public static final String LOCAL_ADDRESS = IPUtils.getFirstNoLoopbackIP4Address(); public static final String LOCAL_ADDRESS = IPUtils.getFirstNoLoopbackIP4Address();
/**
* netty epoll enable switch
*/
public static final String NETTY_EPOLL_ENABLE = System.getProperty("netty.epoll.enable");
/**
* OS Name
*/
public static final String OS_NAME = System.getProperty("os.name");
} }
/*
* 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.dolphinscheduler.remote.utils;
import io.netty.channel.epoll.Epoll;
/**
* NettyUtils
*/
public class NettyUtils {
private NettyUtils() {
}
public static boolean useEpoll() {
String osName = Constants.OS_NAME;
if (!osName.toLowerCase().contains("linux")) {
return false;
}
if (!Epoll.isAvailable()) {
return false;
}
String enableNettyEpoll = Constants.NETTY_EPOLL_ENABLE;
return Boolean.parseBoolean(enableNettyEpoll);
}
}
/*
* 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.dolphinscheduler.remote;
import org.apache.dolphinscheduler.remote.utils.NettyUtils;
import org.junit.Assert;
import org.junit.Test;
/**
* NettyUtilTest
*/
public class NettyUtilTest {
@Test
public void testUserEpoll() {
System.setProperty("netty.epoll.enable", "false");
Assert.assertFalse(NettyUtils.useEpoll());
}
}
...@@ -804,7 +804,8 @@ ...@@ -804,7 +804,8 @@
<include>**/remote/JsonSerializerTest.java</include> <include>**/remote/JsonSerializerTest.java</include>
<include>**/remote/RemoveTaskLogResponseCommandTest.java</include> <include>**/remote/RemoveTaskLogResponseCommandTest.java</include>
<include>**/remote/RemoveTaskLogRequestCommandTest.java</include> <include>**/remote/RemoveTaskLogRequestCommandTest.java</include>
<!--<include>**/remote/NettyRemotingClientTest.java</include>--> <include>**/remote/NettyRemotingClientTest.java</include>
<include>**/remote/NettyUtilTest.java</include>
<include>**/remote/ResponseFutureTest.java</include> <include>**/remote/ResponseFutureTest.java</include>
<include>**/server/log/LoggerServerTest.java</include> <include>**/server/log/LoggerServerTest.java</include>
<include>**/server/entity/SQLTaskExecutionContextTest.java</include> <include>**/server/entity/SQLTaskExecutionContextTest.java</include>
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册