HBASE-16335 RpcClient under heavy load leaks some netty bytebuf (Ram)
This commit is contained in:
parent
6eb6225456
commit
c5b8aababe
|
@ -215,6 +215,7 @@ public abstract class AbstractRpcClient<T extends RpcConnection> implements RpcC
|
||||||
if (conn.getLastTouched() < closeBeforeTime && !conn.isActive()) {
|
if (conn.getLastTouched() < closeBeforeTime && !conn.isActive()) {
|
||||||
LOG.info("Cleanup idle connection to " + conn.remoteId().address);
|
LOG.info("Cleanup idle connection to " + conn.remoteId().address);
|
||||||
connections.removeValue(conn.remoteId(), conn);
|
connections.removeValue(conn.remoteId(), conn);
|
||||||
|
conn.cleanupConnection();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -472,6 +473,9 @@ public abstract class AbstractRpcClient<T extends RpcConnection> implements RpcC
|
||||||
conn.shutdown();
|
conn.shutdown();
|
||||||
}
|
}
|
||||||
closeInternal();
|
closeInternal();
|
||||||
|
for (T conn : connToClose) {
|
||||||
|
conn.cleanupConnection();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -684,6 +684,11 @@ class BlockingRpcConnection extends RpcConnection implements Runnable {
|
||||||
closeConn(new IOException("connection to " + remoteId.address + " closed"));
|
closeConn(new IOException("connection to " + remoteId.address + " closed"));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void cleanupConnection() {
|
||||||
|
// do nothing
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public synchronized void sendRequest(final Call call, HBaseRpcController pcrc)
|
public synchronized void sendRequest(final Call call, HBaseRpcController pcrc)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
|
|
|
@ -36,6 +36,7 @@ import io.netty.channel.ChannelOption;
|
||||||
import io.netty.channel.ChannelPipeline;
|
import io.netty.channel.ChannelPipeline;
|
||||||
import io.netty.handler.codec.LengthFieldBasedFrameDecoder;
|
import io.netty.handler.codec.LengthFieldBasedFrameDecoder;
|
||||||
import io.netty.handler.timeout.IdleStateHandler;
|
import io.netty.handler.timeout.IdleStateHandler;
|
||||||
|
import io.netty.util.ReferenceCountUtil;
|
||||||
import io.netty.util.concurrent.Future;
|
import io.netty.util.concurrent.Future;
|
||||||
import io.netty.util.concurrent.FutureListener;
|
import io.netty.util.concurrent.FutureListener;
|
||||||
import io.netty.util.concurrent.Promise;
|
import io.netty.util.concurrent.Promise;
|
||||||
|
@ -119,6 +120,16 @@ class NettyRpcConnection extends RpcConnection {
|
||||||
shutdown0();
|
shutdown0();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public synchronized void cleanupConnection() {
|
||||||
|
if (connectionHeaderPreamble != null) {
|
||||||
|
ReferenceCountUtil.safeRelease(connectionHeaderPreamble);
|
||||||
|
}
|
||||||
|
if (connectionHeaderWithLength != null) {
|
||||||
|
ReferenceCountUtil.safeRelease(connectionHeaderWithLength);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
private void established(Channel ch) {
|
private void established(Channel ch) {
|
||||||
ch.write(connectionHeaderWithLength.retainedDuplicate());
|
ch.write(connectionHeaderWithLength.retainedDuplicate());
|
||||||
ChannelPipeline p = ch.pipeline();
|
ChannelPipeline p = ch.pipeline();
|
||||||
|
|
|
@ -252,4 +252,9 @@ abstract class RpcConnection {
|
||||||
public abstract void shutdown();
|
public abstract void shutdown();
|
||||||
|
|
||||||
public abstract void sendRequest(Call call, HBaseRpcController hrc) throws IOException;
|
public abstract void sendRequest(Call call, HBaseRpcController hrc) throws IOException;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Does the clean up work after the connection is removed from the connection pool
|
||||||
|
*/
|
||||||
|
public abstract void cleanupConnection();
|
||||||
}
|
}
|
||||||
|
|
|
@ -23,6 +23,7 @@ import io.netty.channel.ChannelHandlerContext;
|
||||||
import io.netty.channel.ChannelOutboundHandlerAdapter;
|
import io.netty.channel.ChannelOutboundHandlerAdapter;
|
||||||
import io.netty.channel.ChannelPromise;
|
import io.netty.channel.ChannelPromise;
|
||||||
import io.netty.channel.CoalescingBufferQueue;
|
import io.netty.channel.CoalescingBufferQueue;
|
||||||
|
import io.netty.util.ReferenceCountUtil;
|
||||||
import io.netty.util.concurrent.PromiseCombiner;
|
import io.netty.util.concurrent.PromiseCombiner;
|
||||||
|
|
||||||
import javax.security.sasl.SaslClient;
|
import javax.security.sasl.SaslClient;
|
||||||
|
@ -60,21 +61,33 @@ public class SaslWrapHandler extends ChannelOutboundHandlerAdapter {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void flush(ChannelHandlerContext ctx) throws Exception {
|
public void flush(ChannelHandlerContext ctx) throws Exception {
|
||||||
if (!queue.isEmpty()) {
|
ByteBuf buf = null;
|
||||||
ChannelPromise promise = ctx.newPromise();
|
try {
|
||||||
int readableBytes = queue.readableBytes();
|
if (!queue.isEmpty()) {
|
||||||
ByteBuf buf = queue.remove(readableBytes, promise);
|
ChannelPromise promise = ctx.newPromise();
|
||||||
byte[] bytes = new byte[readableBytes];
|
int readableBytes = queue.readableBytes();
|
||||||
buf.readBytes(bytes);
|
buf = queue.remove(readableBytes, promise);
|
||||||
byte[] wrapperBytes = saslClient.wrap(bytes, 0, bytes.length);
|
byte[] bytes = new byte[readableBytes];
|
||||||
ChannelPromise lenPromise = ctx.newPromise();
|
buf.readBytes(bytes);
|
||||||
ctx.write(ctx.alloc().buffer(4).writeInt(wrapperBytes.length), lenPromise);
|
byte[] wrapperBytes = saslClient.wrap(bytes, 0, bytes.length);
|
||||||
ChannelPromise contentPromise = ctx.newPromise();
|
ChannelPromise lenPromise = ctx.newPromise();
|
||||||
ctx.write(Unpooled.wrappedBuffer(wrapperBytes), contentPromise);
|
ctx.write(ctx.alloc().buffer(4).writeInt(wrapperBytes.length), lenPromise);
|
||||||
PromiseCombiner combiner = new PromiseCombiner();
|
ChannelPromise contentPromise = ctx.newPromise();
|
||||||
combiner.addAll(lenPromise, contentPromise);
|
ctx.write(Unpooled.wrappedBuffer(wrapperBytes), contentPromise);
|
||||||
combiner.finish(promise);
|
PromiseCombiner combiner = new PromiseCombiner();
|
||||||
|
combiner.addAll(lenPromise, contentPromise);
|
||||||
|
combiner.finish(promise);
|
||||||
|
}
|
||||||
|
ctx.flush();
|
||||||
|
} finally {
|
||||||
|
if (buf != null) {
|
||||||
|
ReferenceCountUtil.safeRelease(buf);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
ctx.flush();
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void close(ChannelHandlerContext ctx, ChannelPromise promise) throws Exception {
|
||||||
|
queue.releaseAndFailAll(new Throwable("Closed"));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
Loading…
Reference in New Issue