Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[fix] [pulsar-client] Fix pendingLookupRequestSemaphore leak when channel inactive #17856

Merged
merged 3 commits into from
Oct 14, 2022
Merged
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 @@ -141,6 +141,9 @@ public class ClientCnx extends PulsarHandler {

private final CompletableFuture<Void> connectionFuture = new CompletableFuture<Void>();
private final ConcurrentLinkedQueue<RequestTime> requestTimeoutQueue = new ConcurrentLinkedQueue<>();

@VisibleForTesting
@Getter(AccessLevel.PACKAGE)
private final Semaphore pendingLookupRequestSemaphore;
private final Semaphore maxLookupRequestSemaphore;
private final EventLoopGroup eventLoopGroup;
Expand Down Expand Up @@ -776,6 +779,11 @@ public CompletableFuture<LookupDataResult> newLookup(ByteBuf request, long reque
TimedCompletableFuture<LookupDataResult> future = new TimedCompletableFuture<>();

if (pendingLookupRequestSemaphore.tryAcquire()) {
future.whenComplete((lookupDataResult, throwable) -> {
if (throwable instanceof ConnectException) {
pendingLookupRequestSemaphore.release();
}
});
addPendingLookupRequests(requestId, future);
ctx.writeAndFlush(request).addListener(writeFuture -> {
if (!writeFuture.isSuccess()) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -34,6 +34,7 @@
import io.netty.util.concurrent.DefaultThreadFactory;
import java.lang.reflect.Field;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ThreadFactory;
import java.util.function.Consumer;
Expand All @@ -50,6 +51,7 @@
import org.apache.pulsar.common.protocol.Commands;
import org.apache.pulsar.common.protocol.PulsarHandler;
import org.apache.pulsar.common.util.netty.EventLoopUtil;
import org.awaitility.Awaitility;
import org.testng.annotations.Test;

public class ClientCnxTest {
Expand Down Expand Up @@ -80,6 +82,72 @@ public void testClientCnxTimeout() throws Exception {
eventLoop.shutdownGracefully();
}

@Test
public void testPendingLookupRequestSemaphore() throws Exception {
EventLoopGroup eventLoop = EventLoopUtil.newEventLoopGroup(1, false, new DefaultThreadFactory("testClientCnxTimeout"));
ClientConfigurationData conf = new ClientConfigurationData();
conf.setOperationTimeoutMs(10_000);
conf.setKeepAliveIntervalSeconds(0);
ClientCnx cnx = new ClientCnx(conf, eventLoop);

ChannelHandlerContext ctx = mock(ChannelHandlerContext.class);
Channel channel = mock(Channel.class);
when(ctx.channel()).thenReturn(channel);
ChannelFuture listenerFuture = mock(ChannelFuture.class);
when(listenerFuture.addListener(any())).thenReturn(listenerFuture);
when(ctx.writeAndFlush(any())).thenReturn(listenerFuture);
cnx.channelActive(ctx);
CountDownLatch countDownLatch1 = new CountDownLatch(1);
CountDownLatch countDownLatch2 = new CountDownLatch(1);
new Thread(() -> {
try {
Thread.sleep(5_000);
CompletableFuture<BinaryProtoLookupService.LookupDataResult> future =
cnx.newLookup(null, 123);
countDownLatch1.countDown();
future.get();
} catch (Exception e) {
// ignore exception
} finally {
countDownLatch2.countDown();
}
}).start();
countDownLatch1.await();
cnx.channelInactive(ctx);
countDownLatch2.await();
// wait for subsequent calls over
Awaitility.await().untilAsserted(() -> {
assertEquals(cnx.getPendingLookupRequestSemaphore().availablePermits(), conf.getConcurrentLookupRequest());
});
eventLoop.shutdownGracefully();
}

@Test
public void testPendingWaitingLookupRequestSemaphore() throws Exception {
EventLoopGroup eventLoop = EventLoopUtil.newEventLoopGroup(1, false, new DefaultThreadFactory("testClientCnxTimeout"));
ClientConfigurationData conf = new ClientConfigurationData();
conf.setOperationTimeoutMs(10_000);
conf.setKeepAliveIntervalSeconds(0);
ClientCnx cnx = new ClientCnx(conf, eventLoop);

ChannelHandlerContext ctx = mock(ChannelHandlerContext.class);
Channel channel = mock(Channel.class);
when(ctx.channel()).thenReturn(channel);
ChannelFuture listenerFuture = mock(ChannelFuture.class);
when(listenerFuture.addListener(any())).thenReturn(listenerFuture);
when(ctx.writeAndFlush(any())).thenReturn(listenerFuture);
cnx.channelActive(ctx);
for (int i = 0; i < 5001; i++) {
cnx.newLookup(null, i);
}
cnx.channelInactive(ctx);
// wait for subsequent calls over
Awaitility.await().untilAsserted(() -> {
assertEquals(cnx.getPendingLookupRequestSemaphore().availablePermits(), conf.getConcurrentLookupRequest());
});
eventLoop.shutdownGracefully();
}

@Test
public void testReceiveErrorAtSendConnectFrameState() throws Exception {
ThreadFactory threadFactory = new DefaultThreadFactory("testReceiveErrorAtSendConnectFrameState");
Expand Down