diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java index 5be21cc585f04b..ba874691f90104 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java @@ -1579,9 +1579,8 @@ protected void handleCloseProducer(CommandCloseProducer closeProducer) { CompletableFuture producerFuture = producers.get(producerId); if (producerFuture == null) { - log.warn("[{}] Producer was not registered on the connection. producerId={}", remoteAddress, producerId); - commandSender.sendErrorResponse(requestId, ServerError.UnknownError, - "Producer was not registered on the connection"); + log.info("[{}] Producer {} was not registered on the connection", remoteAddress, producerId); + ctx.writeAndFlush(Commands.newSuccess(requestId)); return; } @@ -1626,8 +1625,8 @@ protected void handleCloseConsumer(CommandCloseConsumer closeConsumer) { CompletableFuture consumerFuture = consumers.get(consumerId); if (consumerFuture == null) { - log.warn("[{}] Consumer was not registered on the connection: consumerId={}", remoteAddress, consumerId); - commandSender.sendErrorResponse(requestId, ServerError.MetadataError, "Consumer not found"); + log.info("[{}] Consumer was not registered on the connection: {}", consumerId, remoteAddress); + ctx.writeAndFlush(Commands.newSuccess(requestId)); return; }