Skip to content

Commit

Permalink
add redis client name
Browse files Browse the repository at this point in the history
  • Loading branch information
yifuzhou committed Nov 6, 2024
1 parent 97e9741 commit 1710ed2
Show file tree
Hide file tree
Showing 22 changed files with 364 additions and 34 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -56,16 +56,21 @@ protected void doSendRequest(final NettyClient nettyClient, ByteBuf byteBuf) {
}

if(getCommandTimeoutMilli() > 0 && scheduled != null){
int commandTimeoutMilli = getCommandTimeoutMilli();
if (nettyClient instanceof RedisAsyncNettyClient && !((RedisAsyncNettyClient) nettyClient).getDoAfterConnectedOver()) {
commandTimeoutMilli += ((RedisAsyncNettyClient) nettyClient).getAfterConnectCommandTimeoutMill();
}

getLogger().debug("[doSendRequest][schedule timeout]{}, {}", this, getCommandTimeoutMilli());
getLogger().debug("[doSendRequest][schedule timeout]{}, {}", this, commandTimeoutMilli);
int finalCommandTimeoutMilli = commandTimeoutMilli;
timeoutFuture = scheduled.schedule(new AbstractExceptionLogTask() {

@Override
public void doRun() {
getLogger().info("[{}][run][timeout]{}", AbstractNettyRequestResponseCommand.this, nettyClient);
future().setFailure(new CommandTimeoutException("timeout " + + getCommandTimeoutMilli()));
future().setFailure(new CommandTimeoutException("timeout " + finalCommandTimeoutMilli));
}
}, getCommandTimeoutMilli(), TimeUnit.MILLISECONDS);
}, commandTimeoutMilli, TimeUnit.MILLISECONDS);

future().addListener(new CommandFutureListener<V>() {

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -9,6 +9,8 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import java.util.concurrent.atomic.AtomicReference;


/**
* @author chen.zhu
Expand All @@ -21,6 +23,8 @@ public class AsyncNettyClient extends DefaultNettyClient {

private ChannelFuture future;

protected final AtomicReference<Boolean> doAfterConnectedOver = new AtomicReference<>(false);

public AsyncNettyClient(ChannelFuture future, Endpoint endpoint) {
super(future.channel());
this.future = future;
Expand All @@ -35,6 +39,8 @@ public void operationComplete(ChannelFuture future) {
} else {
desc.set(ChannelUtil.getDesc(future.channel()));
}
doAfterConnected();
doAfterConnectedOver.set(true);
} else {
logger.info("[async][connect-fail] endpint: {}", endpoint, future.cause());
}
Expand Down Expand Up @@ -93,4 +99,16 @@ public String toString() {
return super.toString();
}

protected void doAfterConnected() {

}

protected boolean getDoAfterConnectedOver() {
return doAfterConnectedOver.get();
}

protected int getAfterConnectCommandTimeoutMill() {
return 0;
}

}
Original file line number Diff line number Diff line change
@@ -0,0 +1,7 @@
package com.ctrip.xpipe.netty.commands;

public interface DoAfterConnectedHandler {

boolean shouldDo();

}
Original file line number Diff line number Diff line change
Expand Up @@ -46,7 +46,11 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception

@Override
public void read(Channel channel, ByteBuf byteBuf) throws ByteBufReadActionException {
nettyClient.handleResponse(channel, byteBuf);
try {
nettyClient.handleResponse(channel, byteBuf);
} catch (Exception e) {
logger.error("[NettyClientHandler][channelRead]", e);
}
}
});

Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,28 @@
package com.ctrip.xpipe.netty.commands;

import com.ctrip.xpipe.api.endpoint.Endpoint;
import io.netty.channel.ChannelFuture;
import org.apache.commons.pool2.PooledObject;
import org.apache.commons.pool2.impl.DefaultPooledObject;

public class NettyRedisPoolClientFactory extends NettyKeyedPoolClientFactory{

private String clientName;

private DoAfterConnectedHandler doAfterConnectedHandler;

public NettyRedisPoolClientFactory(int eventLoopThreads, String clientName, DoAfterConnectedHandler doAfterConnectedHandler) {
super(eventLoopThreads);
this.clientName = clientName;
this.doAfterConnectedHandler = doAfterConnectedHandler;
}

@Override
public PooledObject<NettyClient> makeObject(Endpoint key) throws Exception {
ChannelFuture f = b.connect(key.getHost(), key.getPort());
NettyClient nettyClient = new RedisAsyncNettyClient(f, key, clientName, doAfterConnectedHandler);
f.channel().attr(NettyClientHandler.KEY_CLIENT).set(nettyClient);
return new DefaultPooledObject<NettyClient>(nettyClient);
}

}
Original file line number Diff line number Diff line change
@@ -0,0 +1,139 @@
package com.ctrip.xpipe.netty.commands;

import com.ctrip.xpipe.api.codec.Codec;
import com.ctrip.xpipe.api.endpoint.Endpoint;
import com.ctrip.xpipe.api.monitor.Task;
import com.ctrip.xpipe.api.monitor.TransactionMonitor;
import com.ctrip.xpipe.utils.ChannelUtil;
import io.netty.buffer.ByteBuf;
import io.netty.buffer.Unpooled;
import io.netty.channel.Channel;
import io.netty.channel.ChannelFuture;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import java.io.ByteArrayOutputStream;
import java.util.HashMap;
import java.util.Map;

public class RedisAsyncNettyClient extends AsyncNettyClient{

protected Logger logger = LoggerFactory.getLogger(RedisAsyncNettyClient.class);

private String clientName;

private DoAfterConnectedHandler doAfterConnectedHandler;

private static final String CLIENT_SET_NAME = "CLIENT SETNAME ";

private static final String EXPECT_RESP = "OK";

private static final int DEFAULT_REDIS_COMMAND_TIME_OUT_MILLI = 660;

public RedisAsyncNettyClient(ChannelFuture future, Endpoint endpoint, String clientName, DoAfterConnectedHandler doAfterConnectedHandler) {
super(future, endpoint);
this.clientName = clientName;
this.doAfterConnectedHandler = doAfterConnectedHandler;
}

@Override
protected void doAfterConnected() {
if (doAfterConnectedHandler != null && !doAfterConnectedHandler.shouldDo()) {
return;
}
TransactionMonitor transaction = TransactionMonitor.DEFAULT;
transaction.logTransactionSwallowException("netty.client.setName", desc.get(), new Task() {
@Override
public void go() throws Exception {
String command = CLIENT_SET_NAME + clientName + "\r\n";
ByteBuf byteBuf = Unpooled.wrappedBuffer(command.getBytes(Codec.defaultCharset));
RedisAsyncNettyClient.super.sendRequest(byteBuf, new ByteBufReceiver() {
@Override
public RECEIVER_RESULT receive(Channel channel, ByteBuf byteBuf) {
String result = doReceiveResponse(byteBuf);
if (result == null) {
logger.warn("[redisAsync][clientSetName][wont-result][{}] {}", desc, result);
}
if (EXPECT_RESP.equalsIgnoreCase(result)) {
logger.info("[redisAsync][clientSetName][success][{}] {}", desc, result);
}
return RECEIVER_RESULT.SUCCESS;
}

@Override
public void clientClosed(NettyClient nettyClient) {
logger.warn("[redisAsync][clientSetName][wont-send][{}] {}", desc, nettyClient.channel());
}

@Override
public void clientClosed(NettyClient nettyClient, Throwable th) {
logger.warn("[redisAsync][clientSetName][wont-send][{}] {}", desc, nettyClient.channel(), th);
}
});
}

@Override
public Map getData() {
Map<String, Object> transactionData = new HashMap<>();
transactionData.put("remoteAddress ", ChannelUtil.getSimpleIpport(channel.remoteAddress()));
transactionData.put("clientName", clientName);
transactionData.put("commandTimeoutMills", getAfterConnectCommandTimeoutMill());
return transactionData;
}
});

}

@Override
protected int getAfterConnectCommandTimeoutMill() {
return DEFAULT_REDIS_COMMAND_TIME_OUT_MILLI;
}

public static String doReceiveResponse(ByteBuf byteBuf) {
ByteArrayOutputStream baous = new ByteArrayOutputStream();
CRLF_STATE crlfState = CRLF_STATE.CONTENT;
int readable = byteBuf.readableBytes();
for(int i=0; i < readable ;i++){

byte data = byteBuf.readByte();
baous.write(data);
switch(data){
case '\r':
crlfState = CRLF_STATE.CR;
break;
case '\n':
if(crlfState == CRLF_STATE.CR){
crlfState = CRLF_STATE.CRLF;
break;
}
default:
crlfState = CRLF_STATE.CONTENT;
break;
}

if(crlfState == CRLF_STATE.CRLF){
break;
}
}
if (baous.toByteArray().length != 0 && crlfState == CRLF_STATE.CRLF) {
String data = new String(baous.toByteArray(), Codec.defaultCharset);
int beginIndex = 0;
int endIndex = data.length();
if(data.charAt(0) == '+'){
beginIndex = 1;
}
if(data.charAt(endIndex - 2) == '\r' && data.charAt(endIndex - 1) == '\n'){
endIndex -= 2;
}
return data.substring(beginIndex, endIndex);
}
return null;
}

public enum CRLF_STATE{
CR,
CRLF,
CONTENT
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -61,7 +61,7 @@ public NettyClient borrowObject() throws BorrowObjectException {
try {
return objectPool.borrowObject();
} catch (Exception e) {
logger.error("[borrowObject]" + factory, e);
logger.error("[borrowObject] NumIdle:{}, NumActive:{}" + factory, objectPool.getNumIdle(), objectPool.getNumActive(), e);
throw new BorrowObjectException("borrow " + factory, e);
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -112,4 +112,6 @@ public interface CheckerConfig {

int getMarkInstanceMaxDelayMilli();

boolean getDoAfterNettyClientConnected();

}
Original file line number Diff line number Diff line change
Expand Up @@ -37,6 +37,8 @@ public class DataCenterConfigBean extends AbstractConfigBean {

public static final String KEY_HTTP_ACCEPT_ENCODING = "http.accept.encoding";

private static final String KEY_NETTY_CLIENT_DO_AFTER_CONNECTED = "netty.client.do.after.connected";

private AtomicReference<String> zkConnection = new AtomicReference<>();
private AtomicReference<String> zkNameSpace = new AtomicReference<>();

Expand Down Expand Up @@ -95,4 +97,8 @@ public String getHttpAcceptEncoding() {
return getProperty(KEY_HTTP_ACCEPT_ENCODING, null);
}

public boolean getDoAfterNettyClientConnected() {
return getBooleanProperty(KEY_NETTY_CLIENT_DO_AFTER_CONNECTED, true);
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -33,7 +33,7 @@
import java.util.concurrent.ScheduledExecutorService;
import java.util.stream.Collectors;

import static com.ctrip.xpipe.redis.checker.resource.Resource.KEYED_NETTY_CLIENT_POOL;
import static com.ctrip.xpipe.redis.checker.resource.Resource.REDIS_KEYED_NETTY_CLIENT_POOL;
import static com.ctrip.xpipe.spring.AbstractSpringConfigContext.SCHEDULED_EXECUTOR;

/**
Expand All @@ -49,7 +49,7 @@ public class MasterOverOneMonitor implements RedisMasterActionListener, OneWaySu
@Autowired
private AlertManager alertManager;

@Resource(name = KEYED_NETTY_CLIENT_POOL)
@Resource(name = REDIS_KEYED_NETTY_CLIENT_POOL)
private XpipeNettyClientKeyedObjectPool keyedObjectPool;

@Resource(name = SCHEDULED_EXECUTOR)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -46,7 +46,7 @@
import java.util.stream.Collectors;

import static com.ctrip.xpipe.redis.checker.healthcheck.actions.sentinel.SentinelHelloCheckAction.LOG_TITLE;
import static com.ctrip.xpipe.redis.checker.resource.Resource.KEYED_NETTY_CLIENT_POOL;
import static com.ctrip.xpipe.redis.checker.resource.Resource.SENTINEL_KEYED_NETTY_CLIENT_POOL;
import static com.ctrip.xpipe.spring.AbstractSpringConfigContext.THREAD_POOL_TIME_OUT;

/**
Expand Down Expand Up @@ -77,7 +77,7 @@ public class DefaultSentinelHelloCollector implements SentinelHelloCollector {
@Autowired
private PersistenceCache persistenceCache;

@Resource(name = KEYED_NETTY_CLIENT_POOL)
@Resource(name = SENTINEL_KEYED_NETTY_CLIENT_POOL)
private XpipeNettyClientKeyedObjectPool keyedObjectPool;

private SentinelLeakyBucket leakyBucket;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,7 @@
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ScheduledExecutorService;

import static com.ctrip.xpipe.redis.checker.resource.Resource.KEYED_NETTY_CLIENT_POOL;
import static com.ctrip.xpipe.redis.checker.resource.Resource.PROXY_KEYED_NETTY_CLIENT_POOL;
import static com.ctrip.xpipe.spring.AbstractSpringConfigContext.SCHEDULED_EXECUTOR;

@Component
Expand All @@ -28,7 +28,7 @@ public class ProxyConnectedChecker implements ProxyChecker {
@Autowired
private CheckerConfig checkerConfig;

@Resource(name = KEYED_NETTY_CLIENT_POOL)
@Resource(name = PROXY_KEYED_NETTY_CLIENT_POOL)
private XpipeNettyClientKeyedObjectPool keyedObjectPool;

@Resource(name = SCHEDULED_EXECUTOR)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -8,11 +8,15 @@ public class Resource {

public static final String REDIS_COMMAND_EXECUTOR = "redisCommandExecutor";

public static final String KEYED_NETTY_CLIENT_POOL = "keyedClientPool";
public static final String REDIS_KEYED_NETTY_CLIENT_POOL = "redisKeyedClientPool";

public static final String REDIS_SESSION_NETTY_CLIENT_POOL = "redisSessionClientPool";
public static final String SENTINEL_KEYED_NETTY_CLIENT_POOL = "sentinelKeyedClientPool";

public static final String KEEPER_KEYED_NETTY_CLIENT_POOL = "keeperKeyedClientPool";

public static final String MIGRATE_KEEPER_CLIENT_POOL = "migrateKeeperClientPool";
public static final String PROXY_KEYED_NETTY_CLIENT_POOL = "proxyKeyedClientPool";

public static final String REDIS_SESSION_NETTY_CLIENT_POOL = "redisSessionClientPool";

public static final String PING_DELAY_INFO_EXECUTORS = "pingDelayInfoExecutors";

Expand Down
Loading

0 comments on commit 1710ed2

Please sign in to comment.