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

[feature][monitoring] Add a metric to track publish buffer usage. #23751

Open
wants to merge 1 commit into
base: master
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from all commits
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 @@ -60,11 +60,13 @@
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.Semaphore;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.LongAdder;
import java.util.stream.Collectors;
import javax.naming.AuthenticationException;
import javax.net.ssl.SSLSession;
import javax.ws.rs.WebApplicationException;
import javax.ws.rs.core.Response;
import io.prometheus.client.Gauge;
import org.apache.bookkeeper.mledger.AsyncCallbacks;
import org.apache.bookkeeper.mledger.Entry;
import org.apache.bookkeeper.mledger.ManagedLedger;
Expand Down Expand Up @@ -250,6 +252,26 @@ public class ServerCnx extends PulsarHandler implements TransportCnx {

private final long connectionLivenessCheckTimeoutMillis;

private static final LongAdder totalPendingBytes = new LongAdder();
private static volatile Gauge pendingPublishBufferUsage;
Comment on lines +255 to +256
Copy link
Member

@lhotari lhotari Dec 20, 2024

Choose a reason for hiding this comment

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

It's not a great idea to add stateful static fields. This causes problems with tests, for example. It also reduces code maintainability.
This should be handled in the BrokerService, I believe.


private static void initializePendingPublishBufferUsageGauge(PulsarService pulsar) {
if (pendingPublishBufferUsage == null) {
pendingPublishBufferUsage = Gauge.build()
.name("pulsar_pending_publish_buffer_usage")
.help("The usage of pending publish buffer")
.unit("bytes")
.labelNames("cluster")
.register()
.setChild(new Gauge.Child() {
@Override
public double get() {
return totalPendingBytes.longValue();
}
}, pulsar.getConfiguration().getClusterName());
}
}

// Tracks and limits number of bytes pending to be published from a single specific IO thread.
static final class PendingBytesPerThreadTracker {
private static final FastThreadLocal<PendingBytesPerThreadTracker> pendingBytesPerThread =
Expand All @@ -269,6 +291,7 @@ public static PendingBytesPerThreadTracker getInstance() {

public void incrementPublishBytes(long bytes, long maxPendingBytesPerThread) {
pendingBytes += bytes;
totalPendingBytes.add(bytes);
// when the limit is exceeded we throttle all connections that are sharing the same thread
if (maxPendingBytesPerThread > 0 && pendingBytes > maxPendingBytesPerThread
&& !limitExceeded) {
Expand All @@ -279,6 +302,7 @@ public void incrementPublishBytes(long bytes, long maxPendingBytesPerThread) {

public void decrementPublishBytes(long bytes, long resumeThresholdPendingBytesPerThread) {
pendingBytes -= bytes;
totalPendingBytes.add(-bytes);
// when the limit has been exceeded, and we are below the resume threshold
// we resume all connections sharing the same thread
if (limitExceeded && pendingBytes <= resumeThresholdPendingBytesPerThread) {
Expand Down Expand Up @@ -312,6 +336,7 @@ public ServerCnx(PulsarService pulsar, String listenerName) {
// the null check is a workaround for #13620
super(pulsar.getBrokerService() != null ? pulsar.getBrokerService().getKeepAliveIntervalSeconds() : 0,
TimeUnit.SECONDS);
initializePendingPublishBufferUsageGauge(pulsar);
this.service = pulsar.getBrokerService();
this.schemaService = pulsar.getSchemaRegistryService();
this.listenerName = listenerName;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1969,6 +1969,26 @@ public void testSplitTopicAndPartitionLabel() throws Exception {
consumer2.close();
}

@Test
public void testPendingPublishBufferUsageMetric() throws Exception {
@Cleanup
Producer<byte[]> producer = pulsarClient.newProducer()
.topic("persistent://my-property/use/my-ns/testPendingPublishBufferUsageMetric")
.create();
for (int i = 0; i < 10; i++) {
String message = "my-message-" + i;
producer.send(message.getBytes());
}
ByteArrayOutputStream statsOut = new ByteArrayOutputStream();
PrometheusMetricsTestUtil.generate(pulsar, false, false, false, statsOut);
String metricsStr = statsOut.toString();
Multimap<String, Metric> metrics = parseMetrics(metricsStr);
List<Metric> cm = (List<Metric>) metrics.get("pulsar_pending_publish_buffer_usage_bytes");
assertEquals(cm.size(), 1);
assertEquals(cm.get(0).tags.get("cluster"), pulsar.getConfiguration().getClusterName());
assertTrue(cm.get(0).value >= 0);
}

@Test
public void testMetricsGroupedByTypeDefinitions() throws Exception {
Producer<byte[]> p1 = pulsarClient.newProducer().topic("persistent://my-property/use/my-ns/my-topic1").create();
Expand Down
Loading