Skip to content
Closed
Show file tree
Hide file tree
Changes from 1 commit
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -128,6 +128,8 @@ public abstract class AbstractRpcClient<T extends RpcConnection> implements RpcC
protected final long failureSleep; // Time to sleep before retry on failure.
protected final boolean tcpNoDelay; // if T then disable Nagle's Algorithm
protected final boolean tcpKeepAlive; // if T then use keepalives
protected final int bufferLowWatermark;
protected final int bufferHighWatermark;
protected final Codec codec;
protected final CompressionCodec compressor;
protected final boolean fallbackAllowed;
Expand Down Expand Up @@ -165,12 +167,15 @@ public AbstractRpcClient(Configuration conf, String clusterId, SocketAddress loc
MetricsConnection metrics) {
this.userProvider = UserProvider.instantiate(conf);
this.localAddr = localAddr;
this.tcpKeepAlive = conf.getBoolean("hbase.ipc.client.tcpkeepalive", true);
this.clusterId = clusterId != null ? clusterId : HConstants.CLUSTER_ID_DEFAULT;
this.failureSleep = conf.getLong(HConstants.HBASE_CLIENT_PAUSE,
HConstants.DEFAULT_HBASE_CLIENT_PAUSE);
this.maxRetries = conf.getInt("hbase.ipc.client.connect.max.retries", 0);
this.tcpNoDelay = conf.getBoolean("hbase.ipc.client.tcpnodelay", true);
this.maxRetries = conf.getInt(CLIENT_CONNECT_MAX_RETRIES, 0);
this.tcpNoDelay = conf.getBoolean(CLIENT_TCP_NODELAY, true);
this.tcpKeepAlive = conf.getBoolean(CLIENT_TCP_KEEPALIVE, true);
this.bufferLowWatermark = conf.getInt(CLIENT_BUFFER_LOW_WATERMARK, DEFAULT_CLIENT_BUFFER_LOW_WATERMARK);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This is only used for NettyRpcClient so better put this into the NettyRpcClient?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This is only used for NettyRpcClient so better put this into the NettyRpcClient?

Yeah,I referred to other constant of RpcClient like SOCKET_TIMEOUT_WRITE so that I put this into RpcClient.I will put these constants into NettyRpcClient.

this.bufferHighWatermark = conf.getInt(CLIENT_BUFFER_HIGH_WATERMARK, DEFAULT_CLIENT_BUFFER_HIGH_WATERMARK);

this.cellBlockBuilder = new CellBlockBuilder(conf);

this.minIdleTimeBeforeClose = conf.getInt(IDLE_TIME, 120000); // 2 minutes
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -36,6 +36,7 @@
import org.apache.hbase.thirdparty.io.netty.channel.ChannelHandler;
import org.apache.hbase.thirdparty.io.netty.channel.ChannelOption;
import org.apache.hbase.thirdparty.io.netty.channel.ChannelPipeline;
import org.apache.hbase.thirdparty.io.netty.channel.WriteBufferWaterMark;
import org.apache.hbase.thirdparty.io.netty.handler.codec.LengthFieldBasedFrameDecoder;
import org.apache.hbase.thirdparty.io.netty.handler.timeout.IdleStateHandler;
import org.apache.hbase.thirdparty.io.netty.util.ReferenceCountUtil;
Expand Down Expand Up @@ -257,6 +258,7 @@ private void connect() {
.option(ChannelOption.TCP_NODELAY, rpcClient.isTcpNoDelay())
.option(ChannelOption.SO_KEEPALIVE, rpcClient.tcpKeepAlive)
.option(ChannelOption.CONNECT_TIMEOUT_MILLIS, rpcClient.connectTO)
.option(ChannelOption.WRITE_BUFFER_WATER_MARK, new WriteBufferWaterMark(rpcClient.bufferLowWatermark, rpcClient.bufferHighWatermark))
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Create a WriteBufferWaterMark and store it in NettyRpcClient so we do not need to create it everytime?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Create a WriteBufferWaterMark and store it in NettyRpcClient so we do not need to create it everytime?

There is really no need to create a WriteBufferWaterMark every time.I will modify this definition in NettyRpcConnection.

.handler(new BufferCallBeforeInitHandler()).localAddress(rpcClient.localAddr)
.remoteAddress(remoteId.address).connect().addListener(new ChannelFutureListener() {

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -34,6 +34,15 @@
public interface RpcClient extends Closeable {
String FAILED_SERVER_EXPIRY_KEY = "hbase.ipc.client.failed.servers.expiry";
int FAILED_SERVER_EXPIRY_DEFAULT = 2000;

String CLIENT_CONNECT_MAX_RETRIES = "hbase.ipc.client.connect.max.retries";
String CLIENT_TCP_NODELAY = "hbase.ipc.client.tcpnodelay";
String CLIENT_TCP_KEEPALIVE = "hbase.ipc.client.tcpkeepalive";
String CLIENT_BUFFER_LOW_WATERMARK = "hbase.ipc.client.bufferlowwatermark";
String CLIENT_BUFFER_HIGH_WATERMARK = "hbase.ipc.client.bufferhighwatermark";
int DEFAULT_CLIENT_BUFFER_LOW_WATERMARK = 1024;
int DEFAULT_CLIENT_BUFFER_HIGH_WATERMARK = 64 * 1024;

String IDLE_TIME = "hbase.ipc.client.connection.minIdleTimeBeforeClose";
String IPC_CLIENT_FALLBACK_TO_SIMPLE_AUTH_ALLOWED_KEY =
"hbase.ipc.client.fallback-to-simple-auth-allowed";
Expand Down
Loading