Browse Source

Merge remote-tracking branch 'remotes/upstream/refactor-architecture' into dev

pull/2/head
qiaozhanwei 5 years ago
parent
commit
cfa882c437
  1. 87
      dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/NettyRemotingClient.java
  2. 21
      dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/exceptions/RemotingTimeoutException.java
  3. 24
      dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/exceptions/RemotingTooMuchRequestException.java
  4. 18
      dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/future/InvokeCallback.java
  5. 41
      dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/future/ReleaseSemaphore.java
  6. 39
      dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/future/ResponseFuture.java
  7. 19
      dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/handler/NettyClientHandler.java
  8. 38
      dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/utils/CallerThreadExecutePolicy.java
  9. 47
      dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/utils/NamedThreadFactory.java
  10. 39
      dolphinscheduler-remote/src/test/java/org/apache/dolphinscheduler/remote/NettyRemotingClientTest.java

87
dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/NettyRemotingClient.java

@ -28,20 +28,19 @@ import org.apache.dolphinscheduler.remote.command.Command;
import org.apache.dolphinscheduler.remote.config.NettyClientConfig; import org.apache.dolphinscheduler.remote.config.NettyClientConfig;
import org.apache.dolphinscheduler.remote.exceptions.RemotingException; import org.apache.dolphinscheduler.remote.exceptions.RemotingException;
import org.apache.dolphinscheduler.remote.exceptions.RemotingTimeoutException; import org.apache.dolphinscheduler.remote.exceptions.RemotingTimeoutException;
import org.apache.dolphinscheduler.remote.exceptions.RemotingTooMuchRequestException;
import org.apache.dolphinscheduler.remote.future.InvokeCallback; import org.apache.dolphinscheduler.remote.future.InvokeCallback;
import org.apache.dolphinscheduler.remote.future.ReleaseSemaphore;
import org.apache.dolphinscheduler.remote.future.ResponseFuture; import org.apache.dolphinscheduler.remote.future.ResponseFuture;
import org.apache.dolphinscheduler.remote.handler.NettyClientHandler; import org.apache.dolphinscheduler.remote.handler.NettyClientHandler;
import org.apache.dolphinscheduler.remote.utils.Address; import org.apache.dolphinscheduler.remote.utils.Address;
import org.apache.dolphinscheduler.remote.utils.Constants; import org.apache.dolphinscheduler.remote.utils.CallerThreadExecutePolicy;
import org.apache.dolphinscheduler.remote.utils.NamedThreadFactory;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
import java.net.InetSocketAddress; import java.net.InetSocketAddress;
import java.rmi.RemoteException; import java.util.concurrent.*;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.ThreadFactory;
import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicInteger;
@ -56,18 +55,20 @@ public class NettyRemotingClient {
private final NettyEncoder encoder = new NettyEncoder(); private final NettyEncoder encoder = new NettyEncoder();
private final ConcurrentHashMap<Address, Channel> channels = new ConcurrentHashMap(); private final ConcurrentHashMap<Address, Channel> channels = new ConcurrentHashMap(128);
private final ExecutorService defaultExecutor = Executors.newFixedThreadPool(Constants.CPUS);
private final AtomicBoolean isStarted = new AtomicBoolean(false); private final AtomicBoolean isStarted = new AtomicBoolean(false);
private final NioEventLoopGroup workerGroup; private final NioEventLoopGroup workerGroup;
private final NettyClientHandler clientHandler = new NettyClientHandler(this);
private final NettyClientConfig clientConfig; private final NettyClientConfig clientConfig;
private final Semaphore asyncSemaphore = new Semaphore(200, true);
private final ExecutorService callbackExecutor;
private final NettyClientHandler clientHandler;
public NettyRemotingClient(final NettyClientConfig clientConfig){ public NettyRemotingClient(final NettyClientConfig clientConfig){
this.clientConfig = clientConfig; this.clientConfig = clientConfig;
this.workerGroup = new NioEventLoopGroup(clientConfig.getWorkerThreads(), new ThreadFactory() { this.workerGroup = new NioEventLoopGroup(clientConfig.getWorkerThreads(), new ThreadFactory() {
@ -78,6 +79,10 @@ public class NettyRemotingClient {
return new Thread(r, String.format("NettyClient_%d", this.threadIndex.incrementAndGet())); return new Thread(r, String.format("NettyClient_%d", this.threadIndex.incrementAndGet()));
} }
}); });
this.callbackExecutor = new ThreadPoolExecutor(5, 10, 1, TimeUnit.MINUTES,
new LinkedBlockingQueue<>(1000), new NamedThreadFactory("CallbackExecutor", 10), new CallerThreadExecutePolicy());
this.clientHandler = new NettyClientHandler(this, callbackExecutor);
this.start(); this.start();
} }
@ -103,65 +108,79 @@ public class NettyRemotingClient {
isStarted.compareAndSet(false, true); isStarted.compareAndSet(false, true);
} }
//TODO public void sendAsync(final Address address, final Command command, final long timeoutMillis, final InvokeCallback invokeCallback) throws InterruptedException, RemotingException {
public void send(final Address address, final Command command, final InvokeCallback invokeCallback) throws RemotingException {
final Channel channel = getChannel(address); final Channel channel = getChannel(address);
if (channel == null) { if (channel == null) {
throw new RemotingException("network error"); throw new RemotingException("network error");
} }
final long opaque = command.getOpaque();
boolean acquired = this.asyncSemaphore.tryAcquire(timeoutMillis, TimeUnit.MILLISECONDS);
if(acquired){
final ReleaseSemaphore releaseSemaphore = new ReleaseSemaphore(this.asyncSemaphore);
final ResponseFuture responseFuture = new ResponseFuture(opaque, timeoutMillis, invokeCallback, 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()){
logger.info("sent command {} to {}", command, address); responseFuture.setSendOk(true);
return;
} else { } else {
logger.error("send command {} to {} failed, error {}", command, address, future.cause()); responseFuture.setSendOk(false);
}
responseFuture.setCause(future.cause());
responseFuture.putResponse(null);
try {
responseFuture.executeInvokeCallback();
} catch (Throwable ex){
logger.error("execute callback error", ex);
} finally{
responseFuture.release();
} }
} }
}); });
} catch (Exception ex) { } catch (Throwable ex){
String msg = String.format("send command %s to address %s encounter error", command, address); responseFuture.release();
throw new RemotingException(msg, ex); throw new RemotingException(String.format("send command to address: %s failed", address), ex);
}
} else{
String message = String.format("try to acquire async semaphore timeout: %d, waiting thread num: %d, total permits: %d",
timeoutMillis, asyncSemaphore.getQueueLength(), asyncSemaphore.availablePermits());
throw new RemotingTooMuchRequestException(message);
} }
} }
public Command sendSync(final Address address, final Command command, final long timeoutMillis) throws RemotingException { public Command sendSync(final Address address, final Command command, final long timeoutMillis) throws InterruptedException, RemotingException {
final Channel channel = getChannel(address); final Channel channel = getChannel(address);
if (channel == null) { if (channel == null) {
throw new RemotingException(String.format("connect to : %s fail", address)); throw new RemotingException(String.format("connect to : %s fail", address));
} }
final long opaque = command.getOpaque(); final long opaque = command.getOpaque();
try { final ResponseFuture responseFuture = new ResponseFuture(opaque, timeoutMillis, null, null);
final ResponseFuture responseFuture = new ResponseFuture(opaque, timeoutMillis, null);
channel.writeAndFlush(command).addListener(new ChannelFutureListener() { channel.writeAndFlush(command).addListener(new ChannelFutureListener() {
@Override @Override
public void operationComplete(ChannelFuture channelFuture) throws Exception { public void operationComplete(ChannelFuture future) throws Exception {
if(channelFuture.isSuccess()){ if(future.isSuccess()){
responseFuture.setSendOk(true); responseFuture.setSendOk(true);
return; return;
} else { } else {
responseFuture.setSendOk(false); responseFuture.setSendOk(false);
responseFuture.setCause(channelFuture.cause()); }
responseFuture.setCause(future.cause());
responseFuture.putResponse(null); responseFuture.putResponse(null);
logger.error("send command {} to address {} failed", command, address); logger.error("send command {} to address {} failed", command, address);
} }
}
}); });
Command result = responseFuture.waitResponse(); Command result = responseFuture.waitResponse();
if(result == null){ if(result == null){
if(responseFuture.isSendOK()){ if(responseFuture.isSendOK()){
throw new RemotingTimeoutException(address.toString(), timeoutMillis, responseFuture.getCause()); throw new RemotingTimeoutException(address.toString(), timeoutMillis, responseFuture.getCause());
} else{ } else{
throw new RemoteException(address.toString(), responseFuture.getCause()); throw new RemotingException(address.toString(), responseFuture.getCause());
} }
} }
return result; return result;
} catch (Exception ex) {
String msg = String.format("send command %s to address %s error", command, address);
throw new RemotingException(msg, ex);
}
} }
public Channel getChannel(Address address) { public Channel getChannel(Address address) {
@ -192,10 +211,6 @@ public class NettyRemotingClient {
return null; return null;
} }
public ExecutorService getDefaultExecutor() {
return defaultExecutor;
}
public void close() { public void close() {
if(isStarted.compareAndSet(true, false)){ if(isStarted.compareAndSet(true, false)){
try { try {
@ -203,8 +218,8 @@ public class NettyRemotingClient {
if(workerGroup != null){ if(workerGroup != null){
this.workerGroup.shutdownGracefully(); this.workerGroup.shutdownGracefully();
} }
if(defaultExecutor != null){ if(callbackExecutor != null){
defaultExecutor.shutdown(); this.callbackExecutor.shutdownNow();
} }
} catch (Exception ex) { } catch (Exception ex) {
logger.error("netty client close exception", ex); logger.error("netty client close exception", ex);

21
dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/exceptions/RemotingTimeoutException.java

@ -1,8 +1,23 @@
/*
* 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.exceptions; package org.apache.dolphinscheduler.remote.exceptions;
/**
* @Author: Tboy
*/
public class RemotingTimeoutException extends RemotingException{ public class RemotingTimeoutException extends RemotingException{
public RemotingTimeoutException(String message) { public RemotingTimeoutException(String message) {

24
dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/exceptions/RemotingTooMuchRequestException.java

@ -0,0 +1,24 @@
/*
* 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.exceptions;
public class RemotingTooMuchRequestException extends RemotingException{
public RemotingTooMuchRequestException(String message) {
super(message);
}
}

18
dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/future/InvokeCallback.java

@ -1,7 +1,23 @@
/*
* 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.future; package org.apache.dolphinscheduler.remote.future;
/** /**
* @Author: Tboy * invoke callback
*/ */
public interface InvokeCallback { public interface InvokeCallback {

41
dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/future/ReleaseSemaphore.java

@ -0,0 +1,41 @@
/*
* 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.future;
import java.util.concurrent.Semaphore;
import java.util.concurrent.atomic.AtomicBoolean;
/**
* release semaphore
*/
public class ReleaseSemaphore {
private final Semaphore semaphore;
private final AtomicBoolean released;
public ReleaseSemaphore(Semaphore semaphore){
this.semaphore = semaphore;
this.released = new AtomicBoolean(false);
}
public void release(){
if(this.released.compareAndSet(false, true)){
this.semaphore.release();
}
}
}

39
dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/future/ResponseFuture.java

@ -1,13 +1,28 @@
/*
* 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.future; package org.apache.dolphinscheduler.remote.future;
import org.apache.dolphinscheduler.remote.command.Command; import org.apache.dolphinscheduler.remote.command.Command;
import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.*;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
/** /**
* @Author: Tboy * response future
*/ */
public class ResponseFuture { public class ResponseFuture {
@ -19,6 +34,8 @@ public class ResponseFuture {
private final InvokeCallback invokeCallback; private final InvokeCallback invokeCallback;
private final ReleaseSemaphore releaseSemaphore;
private final CountDownLatch latch = new CountDownLatch(1); private final CountDownLatch latch = new CountDownLatch(1);
private final long beginTimestamp = System.currentTimeMillis(); private final long beginTimestamp = System.currentTimeMillis();
@ -29,11 +46,11 @@ public class ResponseFuture {
private volatile Throwable cause; private volatile Throwable cause;
public ResponseFuture(long opaque, long timeoutMillis, InvokeCallback invokeCallback, ReleaseSemaphore releaseSemaphore) {
public ResponseFuture(long opaque, long timeoutMillis, InvokeCallback invokeCallback) {
this.opaque = opaque; this.opaque = opaque;
this.timeoutMillis = timeoutMillis; this.timeoutMillis = timeoutMillis;
this.invokeCallback = invokeCallback; this.invokeCallback = invokeCallback;
this.releaseSemaphore = releaseSemaphore;
FUTURE_TABLE.put(opaque, this); FUTURE_TABLE.put(opaque, this);
} }
@ -95,7 +112,17 @@ public class ResponseFuture {
return responseCommand; return responseCommand;
} }
public void setResponseCommand(Command responseCommand) {
this.responseCommand = responseCommand;
}
public InvokeCallback getInvokeCallback() { public InvokeCallback getInvokeCallback() {
return invokeCallback; return invokeCallback;
} }
public void release() {
if(this.releaseSemaphore != null){
this.releaseSemaphore.release();
}
}
} }

19
dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/handler/NettyClientHandler.java

@ -24,6 +24,8 @@ import org.apache.dolphinscheduler.remote.utils.ChannelUtils;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
import java.util.concurrent.ExecutorService;
/** /**
* netty client request handler * netty client request handler
*/ */
@ -34,8 +36,11 @@ public class NettyClientHandler extends ChannelInboundHandlerAdapter {
private final NettyRemotingClient nettyRemotingClient; private final NettyRemotingClient nettyRemotingClient;
public NettyClientHandler(NettyRemotingClient nettyRemotingClient){ private final ExecutorService callbackExecutor;
public NettyClientHandler(NettyRemotingClient nettyRemotingClient, ExecutorService callbackExecutor){
this.nettyRemotingClient = nettyRemotingClient; this.nettyRemotingClient = nettyRemotingClient;
this.callbackExecutor = callbackExecutor;
} }
@Override @Override
@ -52,8 +57,18 @@ public class NettyClientHandler extends ChannelInboundHandlerAdapter {
private void processReceived(final Command responseCommand) { private void processReceived(final Command responseCommand) {
ResponseFuture future = ResponseFuture.getFuture(responseCommand.getOpaque()); ResponseFuture future = ResponseFuture.getFuture(responseCommand.getOpaque());
if(future != null){ if(future != null){
future.putResponse(responseCommand); future.setResponseCommand(responseCommand);
future.release();
if(future.getInvokeCallback() != null){
this.callbackExecutor.submit(new Runnable() {
@Override
public void run() {
future.executeInvokeCallback(); future.executeInvokeCallback();
}
});
} else{
future.putResponse(responseCommand);
}
} else{ } else{
logger.warn("receive response {}, but not matched any request ", responseCommand); logger.warn("receive response {}, but not matched any request ", responseCommand);
} }

38
dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/utils/CallerThreadExecutePolicy.java

@ -0,0 +1,38 @@
/*
* 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 org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.util.concurrent.RejectedExecutionHandler;
import java.util.concurrent.ThreadPoolExecutor;
/**
* caller thread execute
*/
public class CallerThreadExecutePolicy implements RejectedExecutionHandler {
private final Logger logger = LoggerFactory.getLogger(CallerThreadExecutePolicy.class);
@Override
public void rejectedExecution(Runnable r, ThreadPoolExecutor executor) {
logger.warn("queue is full, trigger caller thread execute");
r.run();
}
}

47
dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/utils/NamedThreadFactory.java

@ -0,0 +1,47 @@
/*
* 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 java.util.concurrent.ThreadFactory;
import java.util.concurrent.atomic.AtomicInteger;
public class NamedThreadFactory implements ThreadFactory {
private final AtomicInteger increment = new AtomicInteger(1);
private final String name;
private final int count;
public NamedThreadFactory(String name){
this(name, 0);
}
public NamedThreadFactory(String name, int count){
this.name = name;
this.count = count;
}
@Override
public Thread newThread(Runnable r) {
final String threadName = count > 0 ? String.format(name + "_%d_%d", count, increment.getAndIncrement())
: String.format(name + "_%d", increment.getAndIncrement());
Thread t = new Thread(r, threadName);
t.setDaemon(true);
return t;
}
}

39
dolphinscheduler-remote/src/test/java/org/apache/dolphinscheduler/remote/NettyRemotingClientTest.java

@ -24,6 +24,8 @@ import org.apache.dolphinscheduler.remote.command.Ping;
import org.apache.dolphinscheduler.remote.command.Pong; import org.apache.dolphinscheduler.remote.command.Pong;
import org.apache.dolphinscheduler.remote.config.NettyClientConfig; import org.apache.dolphinscheduler.remote.config.NettyClientConfig;
import org.apache.dolphinscheduler.remote.config.NettyServerConfig; import org.apache.dolphinscheduler.remote.config.NettyServerConfig;
import org.apache.dolphinscheduler.remote.future.InvokeCallback;
import org.apache.dolphinscheduler.remote.future.ResponseFuture;
import org.apache.dolphinscheduler.remote.processor.NettyRequestProcessor; import org.apache.dolphinscheduler.remote.processor.NettyRequestProcessor;
import org.apache.dolphinscheduler.remote.utils.Address; import org.apache.dolphinscheduler.remote.utils.Address;
import org.junit.Assert; import org.junit.Assert;
@ -42,7 +44,7 @@ public class NettyRemotingClientTest {
* test ping * test ping
*/ */
@Test @Test
public void testSend(){ public void testSendSync(){
NettyServerConfig serverConfig = new NettyServerConfig(); NettyServerConfig serverConfig = new NettyServerConfig();
NettyRemotingServer server = new NettyRemotingServer(serverConfig); NettyRemotingServer server = new NettyRemotingServer(serverConfig);
@ -52,6 +54,8 @@ public class NettyRemotingClientTest {
channel.writeAndFlush(Pong.create(command.getOpaque())); channel.writeAndFlush(Pong.create(command.getOpaque()));
} }
}); });
server.start(); server.start();
// //
final NettyClientConfig clientConfig = new NettyClientConfig(); final NettyClientConfig clientConfig = new NettyClientConfig();
@ -64,4 +68,37 @@ public class NettyRemotingClientTest {
e.printStackTrace(); e.printStackTrace();
} }
} }
@Test
public void testSendAsync(){
NettyServerConfig serverConfig = new NettyServerConfig();
NettyRemotingServer server = new NettyRemotingServer(serverConfig);
server.registerProcessor(CommandType.PING, new NettyRequestProcessor() {
@Override
public void process(Channel channel, Command command) {
channel.writeAndFlush(Pong.create(command.getOpaque()));
}
});
server.start();
//
final NettyClientConfig clientConfig = new NettyClientConfig();
NettyRemotingClient client = new NettyRemotingClient(clientConfig);
CountDownLatch latch = new CountDownLatch(1);
Command commandPing = Ping.create();
try {
final AtomicLong opaque = new AtomicLong(0);
client.sendAsync(new Address("127.0.0.1", serverConfig.getListenPort()), commandPing, 2000, new InvokeCallback() {
@Override
public void operationComplete(ResponseFuture responseFuture) {
opaque.set(responseFuture.getOpaque());
latch.countDown();
}
});
latch.await();
Assert.assertEquals(commandPing.getOpaque(), opaque.get());
} catch (Exception e) {
e.printStackTrace();
}
}
} }

Loading…
Cancel
Save