Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
Show all changes
33 commits
Select commit Hold shift + click to select a range
9a301b1
[fix][broker]Broker OOM because ServerCnx handling too many pending w…
poorbarcode Jun 17, 2025
4b05c3f
1
poorbarcode Jun 17, 2025
bf2fe95
1
poorbarcode Jun 17, 2025
6a507ac
pendingCmdByes
poorbarcode Jun 18, 2025
a93a786
-
poorbarcode Jun 18, 2025
d4862ab
PIP-434
poorbarcode Aug 8, 2025
d4eeee4
address comments
poorbarcode Sep 16, 2025
a8bb760
address comment
poorbarcode Sep 16, 2025
aabc89e
checkstyle
poorbarcode Sep 16, 2025
961ca07
address comment
poorbarcode Sep 17, 2025
b5b68b3
checkstyle
poorbarcode Sep 17, 2025
99751e9
change log level
poorbarcode Sep 18, 2025
0367247
improve logs
poorbarcode Sep 18, 2025
da2c54d
address comment
poorbarcode Sep 22, 2025
7399164
address comment
poorbarcode Sep 22, 2025
680340d
address comment
poorbarcode Sep 23, 2025
84c42d3
address comment
poorbarcode Sep 23, 2025
6dd12dd
rename pulsarChannelFailureCooldownSeconds
poorbarcode Sep 23, 2025
9d59608
rename pulsarChannelUnReadableCooldownSeconds
poorbarcode Sep 23, 2025
9ceaaa3
checkstyle
poorbarcode Sep 23, 2025
d87c3a2
checkstyle
poorbarcode Sep 23, 2025
6ce132b
address comments
poorbarcode Sep 23, 2025
fcb62a3
address comments
poorbarcode Sep 23, 2025
4145f1e
checkstyle
poorbarcode Sep 24, 2025
1e53c20
address comments
poorbarcode Sep 24, 2025
94c91b5
checkstyle
poorbarcode Sep 24, 2025
020fa41
address comments
poorbarcode Sep 24, 2025
61e0c48
address comments
poorbarcode Sep 24, 2025
b711abd
checkstyle
poorbarcode Sep 24, 2025
32424ce
address comments
poorbarcode Sep 24, 2025
02cc0ea
address comments
poorbarcode Sep 24, 2025
454438a
address comments
poorbarcode Sep 24, 2025
3b36906
address comments
poorbarcode Sep 24, 2025
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 @@ -919,6 +919,58 @@ The max allowed delay for delayed delivery (in milliseconds). If the broker rece
)
private int brokerMaxConnections = 0;

@FieldContext(
category = CATEGORY_POLICIES,
doc = "It relates to configuration \"WriteBufferHighWaterMark\" of Netty Channel Config. If the number of bytes"
+ " queued in the write buffer exceeds this value, channel writable state will start to return \"false\"."
)
private int pulsarChannelWriteBufferHighWaterMark = 64 * 1024;

@FieldContext(
category = CATEGORY_POLICIES,
doc = "It relates to configuration \"WriteBufferLowWaterMark\" of Netty Channel Config. If the number of bytes"
+ " queued in the write buffer is smaller than this value, channel writable state will start to return"
+ " \"true\"."
)
private int pulsarChannelWriteBufferLowWaterMark = 32 * 1024;

@FieldContext(
category = CATEGORY_POLICIES,
doc = "If enabled, the broker will pause reading from the channel to deal with new request once the writer"
+ " buffer is full, until it is changed to writable."
)
private boolean pulsarChannelPauseReceivingRequestsIfUnwritable = false;

@FieldContext(
category = CATEGORY_POLICIES,
doc = "After the connection is recovered from an pause receiving state, the channel will be rate-limited"
+ " for a of time window to avoid overwhelming due to the backlog of requests. This parameter defines"
+ " how long the rate limiting should last, in millis. Once the bytes that are waiting to be sent out"
+ " reach the \"pulsarChannelWriteBufferHighWaterMark\", the timer will be reset. Setting a negative"
+ " value will disable the rate limiting."
)
private int pulsarChannelPauseReceivingCooldownMs = 5000;

@FieldContext(
category = CATEGORY_POLICIES,
doc = "After the connection is recovered from a pause receiving state, the channel will be rate-limited for a"
+ " period of time to avoid overwhelming due to the backlog of requests. This parameter defines how"
+ " many requests should be allowed in the rate limiting period."

)
private int pulsarChannelPauseReceivingCooldownRateLimitPermits = 5;

@FieldContext(
category = CATEGORY_POLICIES,
doc = "After the connection is recovered from a pause receiving state, the channel will be rate-limited for a"
+ " period of time defined by pulsarChannelPauseReceivingCooldownMs to avoid overwhelming due to the"
+ " backlog of requests. This parameter defines the period of the rate limiter in milliseconds. If the rate"
+ " limit period is set to 1000, then the unit is requests per 1000 milli seconds. When it's 10, the unit"
+ " is requests per every 10ms."

)
private int pulsarChannelPauseReceivingCooldownRateLimitPeriodMs = 10;

@FieldContext(
category = CATEGORY_POLICIES,
doc = "The maximum number of connections per IP. If it exceeds, new connections are rejected."
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -81,6 +81,8 @@ protected void initChannel(SocketChannel ch) throws Exception {
// disable auto read explicitly so that requests aren't served until auto read is enabled
// ServerCnx must enable auto read in channelActive after PulsarService is ready to accept incoming requests
ch.config().setAutoRead(false);
ch.config().setWriteBufferHighWaterMark(pulsar.getConfig().getPulsarChannelWriteBufferHighWaterMark());
ch.config().setWriteBufferLowWaterMark(pulsar.getConfig().getPulsarChannelWriteBufferLowWaterMark());
ch.pipeline().addLast("consolidation", new FlushConsolidationHandler(1024, true));
if (this.enableTls) {
ch.pipeline().addLast(TLS_HANDLER, new SslHandler(this.sslFactory.createServerSslEngine(ch.alloc())));
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -37,6 +37,7 @@
import io.netty.channel.ChannelHandler;
import io.netty.channel.ChannelHandlerContext;
import io.netty.channel.ChannelOption;
import io.netty.channel.ChannelOutboundBuffer;
import io.netty.handler.codec.haproxy.HAProxyMessage;
import io.netty.handler.ssl.SslHandler;
import io.netty.util.concurrent.FastThreadLocal;
Expand Down Expand Up @@ -181,6 +182,7 @@
import org.apache.pulsar.transaction.coordinator.TransactionCoordinatorID;
import org.apache.pulsar.transaction.coordinator.exceptions.CoordinatorException;
import org.apache.pulsar.transaction.coordinator.impl.MLTransactionMetadataStore;
import org.apache.pulsar.utils.TimedSingleThreadRateLimiter;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

Expand All @@ -191,6 +193,8 @@
* parameter instance lifecycle.
*/
public class ServerCnx extends PulsarHandler implements TransportCnx {
private static final Logger PAUSE_RECEIVING_LOG = LoggerFactory.getLogger(ServerCnx.class.getName()
+ ".pauseReceiving");
private final BrokerService service;
private final SchemaRegistryService schemaService;
private final String listenerName;
Expand Down Expand Up @@ -249,6 +253,10 @@ public class ServerCnx extends PulsarHandler implements TransportCnx {

private final long connectionLivenessCheckTimeoutMillis;
private final TopicsPattern.RegexImplementation topicsPatternImplementation;
private final boolean pauseReceivingRequestsIfUnwritable;
private final TimedSingleThreadRateLimiter requestRateLimiter;
private final int pauseReceivingCooldownMilliSeconds;
private boolean pausedDueToRateLimitation = false;

// Tracks and limits number of bytes pending to be published from a single specific IO thread.
static final class PendingBytesPerThreadTracker {
Expand Down Expand Up @@ -312,6 +320,14 @@ public ServerCnx(PulsarService pulsar, String listenerName) {
// the null check is a workaround for #13620
super(pulsar.getBrokerService() != null ? pulsar.getBrokerService().getKeepAliveIntervalSeconds() : 0,
TimeUnit.SECONDS);
this.pauseReceivingRequestsIfUnwritable =
pulsar.getConfig().isPulsarChannelPauseReceivingRequestsIfUnwritable();
this.requestRateLimiter = new TimedSingleThreadRateLimiter(
pulsar.getConfig().getPulsarChannelPauseReceivingCooldownRateLimitPermits(),
pulsar.getConfig().getPulsarChannelPauseReceivingCooldownRateLimitPeriodMs(),
TimeUnit.MILLISECONDS);
this.pauseReceivingCooldownMilliSeconds =
pulsar.getConfig().getPulsarChannelPauseReceivingCooldownMs();
this.service = pulsar.getBrokerService();
this.schemaService = pulsar.getSchemaRegistryService();
this.listenerName = listenerName;
Expand Down Expand Up @@ -438,11 +454,62 @@ public void channelInactive(ChannelHandlerContext ctx) throws Exception {
}
}

private void checkPauseReceivingRequestsAfterResumeRateLimit(BaseCommand cmd) {
if (!pauseReceivingRequestsIfUnwritable
|| pauseReceivingCooldownMilliSeconds <= 0 || cmd.getType() == BaseCommand.Type.PONG
|| cmd.getType() == BaseCommand.Type.PING) {
return;
}
if (PAUSE_RECEIVING_LOG.isDebugEnabled()) {
final ChannelOutboundBuffer outboundBuffer = ctx.channel().unsafe().outboundBuffer();
if (outboundBuffer != null) {
PAUSE_RECEIVING_LOG.debug("Start to handle request [{}], totalPendingWriteBytes: {}, channel"
+ " isWritable: {}", cmd.getType(), outboundBuffer.totalPendingWriteBytes(),
ctx.channel().isWritable());
} else {
PAUSE_RECEIVING_LOG.debug("Start to handle request [{}], channel isWritable: {}",
cmd.getType(), ctx.channel().isWritable());
}
}
// "requestRateLimiter" will return the permits that you acquired if it is not opening(has been called
// "timingOpen(duration)").
if (requestRateLimiter.acquire(1) == 0 && !pausedDueToRateLimitation) {
log.warn("[{}] Reached rate limitation", this);
// Stop receiving requests.
pausedDueToRateLimitation = true;
ctx.channel().config().setAutoRead(false);
// Resume after 1 second.
ctx.channel().eventLoop().schedule(() -> {
if (pausedDueToRateLimitation) {
log.info("[{}] Resuming connection after rate limitation", this);
ctx.channel().config().setAutoRead(true);
pausedDueToRateLimitation = false;
}
}, 1, TimeUnit.SECONDS);
}
}

@Override
public void channelWritabilityChanged(ChannelHandlerContext ctx) throws Exception {
if (log.isDebugEnabled()) {
log.debug("Channel writability has changed to: {}", ctx.channel().isWritable());
if (pauseReceivingRequestsIfUnwritable && ctx.channel().isWritable()) {
log.info("[{}] is writable, turn on channel auto-read", this);
ctx.channel().config().setAutoRead(true);
requestRateLimiter.timingOpen(pauseReceivingCooldownMilliSeconds, TimeUnit.MILLISECONDS);
} else if (pauseReceivingRequestsIfUnwritable && !ctx.channel().isWritable()) {
final ChannelOutboundBuffer outboundBuffer = ctx.channel().unsafe().outboundBuffer();
if (outboundBuffer != null) {
if (PAUSE_RECEIVING_LOG.isDebugEnabled()) {
PAUSE_RECEIVING_LOG.debug("[{}] is not writable, turn off channel auto-read,"
+ " totalPendingWriteBytes: {}", this, outboundBuffer.totalPendingWriteBytes());
}
} else {
if (PAUSE_RECEIVING_LOG.isDebugEnabled()) {
PAUSE_RECEIVING_LOG.debug("[{}] is not writable, turn off channel auto-read", this);
}
}
ctx.channel().config().setAutoRead(false);
}
ctx.fireChannelWritabilityChanged();
}

@Override
Expand Down Expand Up @@ -3645,8 +3712,9 @@ public CompletableFuture<Optional<Boolean>> checkConnectionLiveness() {
}

@Override
protected void messageReceived() {
super.messageReceived();
protected void messageReceived(BaseCommand cmd) {
checkPauseReceivingRequestsAfterResumeRateLimit(cmd);
super.messageReceived(cmd);
if (connectionCheckInProgress != null && !connectionCheckInProgress.isDone()) {
connectionCheckInProgress.complete(Optional.of(true));
connectionCheckInProgress = null;
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,83 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.pulsar.utils;

import java.util.concurrent.TimeUnit;
import lombok.Getter;
import lombok.extern.slf4j.Slf4j;

@Slf4j
public class TimedSingleThreadRateLimiter {

@Getter
private final int rate;
@Getter
private final long periodAtMs;
private long lastTimeReset;
@Getter
private int remaining;
private long closeAfterAtMs;

public TimedSingleThreadRateLimiter(final int rate, final long period, final TimeUnit unit) {
this.rate = rate;
this.periodAtMs = unit.toMillis(period);
this.lastTimeReset = System.currentTimeMillis();
this.remaining = rate;
}

public int acquire(int permits) {
final long now = System.currentTimeMillis();
if (permits < 0) {
return 0;
}
if (now > closeAfterAtMs) {
return permits;
}
mayRenew(now);
if (remaining > permits) {
remaining -= permits;
if (log.isDebugEnabled()) {
log.debug("acquired: {}, remaining:{}", permits, remaining);
}
return permits;
} else {
int acquired = remaining;
remaining = 0;
if (log.isDebugEnabled()) {
log.debug("acquired: {}, remaining:{}", acquired, remaining);
}
return acquired;
}
}

public void timingOpen(long closeAfter, final TimeUnit unit) {
if (closeAfter <= 0) {
this.closeAfterAtMs = 0;
} else {
this.closeAfterAtMs = System.currentTimeMillis() + unit.toMillis(closeAfter);
}
}

private void mayRenew(long now) {
if (now > lastTimeReset + periodAtMs) {
remaining = rate;
lastTimeReset = now;
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@
import io.netty.channel.embedded.EmbeddedChannel;
import io.netty.handler.codec.LengthFieldBasedFrameDecoder;
import java.util.Queue;
import org.apache.pulsar.common.api.proto.BaseCommand;
import org.apache.pulsar.common.api.proto.CommandAck;
import org.apache.pulsar.common.api.proto.CommandAddPartitionToTxnResponse;
import org.apache.pulsar.common.api.proto.CommandAddSubscriptionToTxnResponse;
Expand Down Expand Up @@ -71,7 +72,7 @@ public Object getCommand(Object obj) {
private final PulsarDecoder decoder = new PulsarDecoder() {

@Override
protected void messageReceived() {
protected void messageReceived(BaseCommand cmd) {
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -46,6 +46,7 @@
import org.apache.pulsar.client.api.MockBrokerServiceHooks.CommandSubscribeHook;
import org.apache.pulsar.client.api.MockBrokerServiceHooks.CommandTopicLookupHook;
import org.apache.pulsar.client.api.MockBrokerServiceHooks.CommandUnsubscribeHook;
import org.apache.pulsar.common.api.proto.BaseCommand;
import org.apache.pulsar.common.api.proto.CommandAck;
import org.apache.pulsar.common.api.proto.CommandCloseConsumer;
import org.apache.pulsar.common.api.proto.CommandCloseProducer;
Expand Down Expand Up @@ -132,7 +133,7 @@ public void channelActive(ChannelHandlerContext ctx) throws Exception {
}

@Override
protected void messageReceived() {
protected void messageReceived(BaseCommand cmd) {
}

@Override
Expand Down
Loading
Loading