From 560fbdc942a03ccf246dfbb29011287ffb648a7a Mon Sep 17 00:00:00 2001 From: Asaf Mesika Date: Wed, 20 Dec 2023 11:46:04 +0200 Subject: [PATCH 1/9] wip --- .../mledger/impl/ManagedCursorContainer.java | 18 ++- .../service/persistent/PersistentTopic.java | 128 +++++++++++++----- 2 files changed, 112 insertions(+), 34 deletions(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorContainer.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorContainer.java index 58c83961d619f..e328de5084238 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorContainer.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorContainer.java @@ -25,6 +25,7 @@ import java.util.concurrent.ConcurrentMap; import java.util.concurrent.ConcurrentSkipListMap; import java.util.concurrent.locks.StampedLock; +import lombok.Value; import org.apache.bookkeeper.mledger.ManagedCursor; import org.apache.bookkeeper.mledger.Position; import org.apache.commons.lang3.tuple.Pair; @@ -44,6 +45,12 @@ */ public class ManagedCursorContainer implements Iterable { + @Value + public class CursorInfo { + ManagedCursor cursor; + PositionImpl markDeletePosition; + } + private static class Item { final ManagedCursor cursor; PositionImpl position; @@ -57,7 +64,6 @@ private static class Item { } public ManagedCursorContainer() { - } // Used to keep track of slowest cursor. @@ -204,6 +210,16 @@ public ManagedCursor getSlowestReader() { } } + public CursorInfo getCursorWithOldestMarkDeletePosition() { + long stamp = rwLock.readLock(); + try { + Item item = heap.get(0); + return heap.isEmpty() ? null : new CursorInfo(item.cursor, item.position); + } finally { + rwLock.unlockRead(stamp); + } + } + /** * Check whether there are any cursors. * @return true is there are no cursors and false if there are diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java index 5357bf728f34a..4f727bfaad16b 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java @@ -19,6 +19,7 @@ package org.apache.pulsar.broker.service.persistent; import static java.util.Objects.requireNonNull; +import static java.util.concurrent.TimeUnit.SECONDS; import static org.apache.commons.lang3.StringUtils.isBlank; import static org.apache.pulsar.broker.service.persistent.SubscribeRateLimiter.isSubscribeRateEnabled; import static org.apache.pulsar.common.naming.SystemTopicNames.isEventSystemTopic; @@ -52,6 +53,7 @@ import java.util.stream.Collectors; import javax.annotation.Nonnull; import lombok.Getter; +import lombok.Value; import org.apache.bookkeeper.client.api.LedgerMetadata; import org.apache.bookkeeper.mledger.AsyncCallbacks; import org.apache.bookkeeper.mledger.AsyncCallbacks.AddEntryCallback; @@ -73,6 +75,7 @@ import org.apache.bookkeeper.mledger.ManagedLedgerException.NonRecoverableLedgerException; import org.apache.bookkeeper.mledger.Position; import org.apache.bookkeeper.mledger.impl.ManagedCursorContainer; +import org.apache.bookkeeper.mledger.impl.ManagedCursorContainer.CursorInfo; import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl; import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; import org.apache.bookkeeper.mledger.impl.PositionImpl; @@ -1133,7 +1136,7 @@ private CompletableFuture getNonDurableSubscription(Stri private void resetSubscriptionCursor(Subscription subscription, CompletableFuture subscriptionFuture, long startMessageRollbackDurationSec) { long timestamp = System.currentTimeMillis() - - TimeUnit.SECONDS.toMillis(startMessageRollbackDurationSec); + - SECONDS.toMillis(startMessageRollbackDurationSec); final Subscription finalSubscription = subscription; subscription.resetCursor(timestamp).handle((s, ex) -> { if (ex != null) { @@ -1608,7 +1611,7 @@ CompletableFuture checkReplicationAndRetryOnFailure() { if (!(th.getCause() instanceof TopicFencedException)) { // retriable exception brokerService.executor().schedule(this::checkReplicationAndRetryOnFailure, - POLICY_UPDATE_FAILURE_RETRY_TIME_SECONDS, TimeUnit.SECONDS); + POLICY_UPDATE_FAILURE_RETRY_TIME_SECONDS, SECONDS); } result.completeExceptionally(th); return null; @@ -2842,7 +2845,7 @@ public void checkGC() { int maxInactiveDurationInSec = topicPolicies.getInactiveTopicPolicies().get().getMaxInactiveDurationSeconds(); if (isActive(deleteMode)) { lastActive = System.nanoTime(); - } else if (System.nanoTime() - lastActive < TimeUnit.SECONDS.toNanos(maxInactiveDurationInSec)) { + } else if (System.nanoTime() - lastActive < SECONDS.toNanos(maxInactiveDurationInSec)) { // Gc interval did not expire yet return; } else if (shouldTopicBeRetained()) { @@ -3151,6 +3154,15 @@ public boolean isSizeBacklogExceeded() { return (storageSize >= backlogQuotaLimitInBytes); } + @Value + private static class TimeBasedBacklogQuotaCheckResult { + PositionImpl oldestCursorMarkDeletePosition; + String cursorName; + Long positionPublishTimestamp; + } + + private volatile TimeBasedBacklogQuotaCheckResult timeBasedBacklogQuotaCheckResult; + /** * @return determine if backlog quota enforcement needs to be done for topic based on time limit */ @@ -3158,28 +3170,66 @@ public CompletableFuture checkTimeBacklogExceeded() { TopicName topicName = TopicName.get(getName()); int backlogQuotaLimitInSecond = getBacklogQuota(BacklogQuotaType.message_age).getLimitTime(); - // If backlog quota by time is not set and we have no durable cursor. + ManagedCursorContainer managedCursorContainer = (ManagedCursorContainer) ledger.getCursors(); + CursorInfo oldestMarkDeleteCursorInfo = managedCursorContainer.getCursorWithOldestMarkDeletePosition(); + + // If backlog quota by time is not set, and we have no durable cursor. if (backlogQuotaLimitInSecond <= 0 - || ((ManagedCursorContainer) ledger.getCursors()).getSlowestReaderPosition() == null) { + || oldestMarkDeleteCursorInfo == null + || oldestMarkDeleteCursorInfo.getMarkDeletePosition() == null) { return CompletableFuture.completedFuture(false); } + PositionImpl oldestMarkDeletePosition = oldestMarkDeleteCursorInfo.getMarkDeletePosition(); + + TimeBasedBacklogQuotaCheckResult lastCheckResult = timeBasedBacklogQuotaCheckResult; + if (lastCheckResult != null + && oldestMarkDeletePosition.compareTo(lastCheckResult.getOldestCursorMarkDeletePosition()) == 0) { + + // Same position, but the cursor causing it has changed? + if (!lastCheckResult.getCursorName().equals(oldestMarkDeleteCursorInfo.getCursor().getName())) { + lastCheckResult = new TimeBasedBacklogQuotaCheckResult( + lastCheckResult.getOldestCursorMarkDeletePosition(), + oldestMarkDeleteCursorInfo.getCursor().getName(), + lastCheckResult.getPositionPublishTimestamp()); + + timeBasedBacklogQuotaCheckResult = lastCheckResult; + } + + Long entryTimestamp = lastCheckResult.getPositionPublishTimestamp(); + if (entryTimestamp == null) { + return CompletableFuture.completedFuture(false); + } + boolean expired = MessageImpl.isEntryExpired(backlogQuotaLimitInSecond, entryTimestamp); + if (expired && log.isDebugEnabled()) { + log.debug("(Using cache) Time based backlog quota exceeded, oldest entry in cursor {}'s backlog" + + "exceeded quota {}", lastCheckResult.getCursorName(), backlogQuotaLimitInSecond); + } + + return CompletableFuture.completedFuture(expired); + } + if (brokerService.pulsar().getConfiguration().isPreciseTimeBasedBacklogQuotaCheck()) { CompletableFuture future = new CompletableFuture<>(); // Check if first unconsumed message(first message after mark delete position) // for slowest cursor's has expired. - PositionImpl position = ((ManagedLedgerImpl) ledger).getNextValidPosition(((ManagedCursorContainer) - ledger.getCursors()).getSlowestReaderPosition()); + PositionImpl position = ((ManagedLedgerImpl) ledger).getNextValidPosition(oldestMarkDeletePosition); ((ManagedLedgerImpl) ledger).asyncReadEntry(position, new AsyncCallbacks.ReadEntryCallback() { @Override public void readEntryComplete(Entry entry, Object ctx) { try { long entryTimestamp = Commands.getEntryTimestamp(entry.getDataBuffer()); + + timeBasedBacklogQuotaCheckResult = new TimeBasedBacklogQuotaCheckResult( + oldestMarkDeleteCursorInfo.getMarkDeletePosition(), + oldestMarkDeleteCursorInfo.getCursor().getName(), + entryTimestamp); + boolean expired = MessageImpl.isEntryExpired(backlogQuotaLimitInSecond, entryTimestamp); if (expired && log.isDebugEnabled()) { log.debug("Time based backlog quota exceeded, oldest entry in cursor {}'s backlog" - + "exceeded quota {}", ((ManagedLedgerImpl) ledger).getSlowestConsumer().getName(), + + "exceeded quota {}", ledger.getSlowestConsumer().getName(), backlogQuotaLimitInSecond); } future.complete(expired); @@ -3200,9 +3250,14 @@ public void readEntryFailed(ManagedLedgerException exception, Object ctx) { }, null); return future; } else { - PositionImpl slowestPosition = ((ManagedCursorContainer) ledger.getCursors()).getSlowestReaderPosition(); try { - return slowestReaderTimeBasedBacklogQuotaCheck(slowestPosition); + CheckResult checkResult = estimatedTimeBasedBacklogQuotaCheck(oldestMarkDeletePosition); + timeBasedBacklogQuotaCheckResult = new TimeBasedBacklogQuotaCheckResult( + oldestMarkDeleteCursorInfo.getMarkDeletePosition(), + oldestMarkDeleteCursorInfo.getCursor().getName(), + checkResult.getEstimatedOldestUnacknowledgedMessageTimestamp()); + + return CompletableFuture.completedFuture(checkResult.isShouldTruncateBacklogToMatchQuota()); } catch (Exception e) { log.error("[{}][{}] Error reading entry for precise time based backlog check", topicName, e); return CompletableFuture.completedFuture(false); @@ -3210,33 +3265,40 @@ public void readEntryFailed(ManagedLedgerException exception, Object ctx) { } } - private CompletableFuture slowestReaderTimeBasedBacklogQuotaCheck(PositionImpl slowestPosition) + @Value + private static class CheckResult { + boolean shouldTruncateBacklogToMatchQuota; + Long estimatedOldestUnacknowledgedMessageTimestamp; + } + private CheckResult estimatedTimeBasedBacklogQuotaCheck(PositionImpl markDeletePosition) throws ExecutionException, InterruptedException { int backlogQuotaLimitInSecond = getBacklogQuota(BacklogQuotaType.message_age).getLimitTime(); - Long ledgerId = slowestPosition.getLedgerId(); - if (((ManagedLedgerImpl) ledger).getLedgersInfo().lastKey().equals(ledgerId)) { - return CompletableFuture.completedFuture(false); + ManagedLedgerImpl managedLedger = (ManagedLedgerImpl) ledger; + if (managedLedger.getLedgersInfo().lastKey().equals(markDeletePosition.getLedgerId())) { + return new CheckResult(false, null); } - int result; + org.apache.bookkeeper.mledger.proto.MLDataFormats.ManagedLedgerInfo.LedgerInfo - ledgerInfo = ledger.getLedgerInfo(ledgerId).get(); - if (ledgerInfo != null && ledgerInfo.hasTimestamp() && ledgerInfo.getTimestamp() > 0 - && ((ManagedLedgerImpl) ledger).getClock().millis() - ledgerInfo.getTimestamp() - > backlogQuotaLimitInSecond * 1000 && (result = slowestPosition.compareTo( - new PositionImpl(ledgerInfo.getLedgerId(), ledgerInfo.getEntries() - 1))) <= 0) { - if (result < 0) { - if (log.isDebugEnabled()) { - log.debug("Time based backlog quota exceeded, quota {}, age of ledger " - + "slowest cursor currently on {}", backlogQuotaLimitInSecond * 1000, - ((ManagedLedgerImpl) ledger).getClock().millis() - ledgerInfo.getTimestamp()); - } - return CompletableFuture.completedFuture(true); - } else { - return slowestReaderTimeBasedBacklogQuotaCheck( - ((ManagedLedgerImpl) ledger).getNextValidPosition(slowestPosition)); - } + markDeletePositionLedgerInfo = ledger.getLedgerInfo(markDeletePosition.getLedgerId()).get(); + + org.apache.bookkeeper.mledger.proto.MLDataFormats.ManagedLedgerInfo.LedgerInfo positionToCheckLedgerInfo = + markDeletePositionLedgerInfo; + + // if the mark-delete position is the last entry it means all entries for + // that ledger are acknowledged + if (markDeletePosition.getEntryId() == markDeletePositionLedgerInfo.getEntries() - 1) { + PositionImpl positionToCheck = managedLedger.getNextValidPosition(markDeletePosition); + positionToCheckLedgerInfo = ledger.getLedgerInfo(positionToCheck.getLedgerId()).get(); + } + + if (positionToCheckLedgerInfo != null + && positionToCheckLedgerInfo.hasTimestamp() + && positionToCheckLedgerInfo.getTimestamp() > 0) { + long estimateMsgAgeMs = managedLedger.getClock().millis() - positionToCheckLedgerInfo.getTimestamp(); + boolean shouldTruncateBacklog = estimateMsgAgeMs > SECONDS.toMillis(backlogQuotaLimitInSecond); + return new CheckResult(shouldTruncateBacklog, positionToCheckLedgerInfo.getTimestamp()); } else { - return CompletableFuture.completedFuture(false); + return new CheckResult(false, null); } } @@ -3599,7 +3661,7 @@ private synchronized void fence() { final int timeout = brokerService.pulsar().getConfiguration().getTopicFencingTimeoutSeconds(); if (timeout > 0) { this.fencedTopicMonitoringTask = brokerService.executor().schedule(this::closeFencedTopicForcefully, - timeout, TimeUnit.SECONDS); + timeout, SECONDS); } } } From 1ae009ccbde41e4428f68b8e351d80d9f961326f Mon Sep 17 00:00:00 2001 From: Asaf Mesika Date: Sun, 24 Dec 2023 19:34:23 +0200 Subject: [PATCH 2/9] Code is added. Next are tests --- .../mledger/impl/ManagedCursorContainer.java | 86 ++++++++++++++--- .../broker/service/BacklogQuotaManager.java | 44 +++++---- .../pulsar/broker/service/BrokerService.java | 49 ++++++---- .../apache/pulsar/broker/service/Topic.java | 13 ++- .../nonpersistent/NonPersistentTopic.java | 7 +- .../service/persistent/PersistentTopic.java | 92 ++++++++++++++----- .../persistent/PersistentTopicMetrics.java | 33 +++++++ .../prometheus/AggregatedBrokerStats.java | 8 ++ .../prometheus/AggregatedNamespaceStats.java | 11 +++ .../prometheus/NamespaceStatsAggregator.java | 61 +++++++++++- .../broker/stats/prometheus/TopicStats.java | 31 +++++++ .../prometheus/metrics/PrometheusLabels.java | 14 +++ .../broker/admin/TopicPoliciesTest.java | 2 +- .../policies/data/stats/TopicStatsImpl.java | 25 +++++ 14 files changed, 394 insertions(+), 82 deletions(-) create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopicMetrics.java create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/metrics/PrometheusLabels.java diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorContainer.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorContainer.java index e328de5084238..a6e590c8042e6 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorContainer.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorContainer.java @@ -32,23 +32,32 @@ /** * Contains cursors for a ManagedLedger. - * - *

The goal is to always know the slowest consumer and hence decide which is the oldest ledger we need to keep. - * - *

This data structure maintains a heap and a map of cursors. The map is used to relate a cursor name with + *

+ * The goal is to always know the slowest consumer and hence decide which is the oldest ledger we need to keep. + *

+ * This data structure maintains a heap and a map of cursors. The map is used to relate a cursor name with * an entry index in the heap. The heap data structure sorts cursors in a binary tree which is represented * in a single array. More details about heap implementations: - * https://en.wikipedia.org/wiki/Heap_(data_structure)#Implementation - * - *

The heap is updated and kept sorted when a cursor is updated. + * here + *

+ * The heap is updated and kept sorted when a cursor is updated. * */ public class ManagedCursorContainer implements Iterable { + /** + * This field is incremented everytime the cursor information is updated. + */ + private volatile long version; + @Value - public class CursorInfo { + public static class CursorInfo { ManagedCursor cursor; PositionImpl markDeletePosition; + + // Use {@link DataVersion#compareVersions(long, long)} to compare between two versions, + // since it rolls over to 0 once reaching Long.MAX_VALUE + long version; } private static class Item { @@ -63,9 +72,50 @@ private static class Item { } } - public ManagedCursorContainer() { + /** + * Utility class to manage a data version, which rolls over to 0 when reaching Long.MAX_VALUE. + */ + public static final class DataVersion { + private DataVersion() {} + + public static int compareVersions(long v1, long v2) { + if (v1 == v2) { + return 0; + } + + // 0-------v1--------v2--------MAX_LONG + if (v2 > v1) { + long distance = v2 - v1; + long wrapAroundDistance = (Long.MAX_VALUE - v2) + v1; + if (distance < wrapAroundDistance) { + return 1; + } else { + return -1; + } + + // 0-------v2--------v1--------MAX_LONG + } else { + long distance = v1 - v2; + long wrapAroundDistance = (Long.MAX_VALUE - v1) + v2; + if (distance < wrapAroundDistance) { + return -1; // v1 is bigger + } else { + return 1; // v2 is bigger + } + } + } + + public static long incrementVersion(long existingVersion) { + if (existingVersion == Long.MAX_VALUE) { + return 0; + } else { + return existingVersion + 1; + } + } } + public ManagedCursorContainer() {} + // Used to keep track of slowest cursor. private final ArrayList heap = new ArrayList<>(); @@ -86,6 +136,7 @@ public ManagedCursorContainer() { * @param position position of the cursor to use for ordering, pass null if the cursor's position shouldn't be * tracked for the slowest reader. */ + @SuppressWarnings("NonAtomicOperationOnVolatileField") public void add(ManagedCursor cursor, Position position) { long stamp = rwLock.writeLock(); try { @@ -100,6 +151,7 @@ public void add(ManagedCursor cursor, Position position) { if (cursor.isDurable()) { durableCursorCount++; } + version = DataVersion.incrementVersion(version); } finally { rwLock.unlockWrite(stamp); } @@ -115,6 +167,7 @@ public ManagedCursor get(String name) { } } + @SuppressWarnings("NonAtomicOperationOnVolatileField") public boolean removeCursor(String name) { long stamp = rwLock.writeLock(); try { @@ -135,6 +188,7 @@ public boolean removeCursor(String name) { if (item.cursor.isDurable()) { durableCursorCount--; } + version = DataVersion.incrementVersion(version); return true; } else { return false; @@ -156,6 +210,7 @@ public boolean removeCursor(String name) { * @return a pair of positions, representing the previous slowest reader and the new slowest reader (after the * update). */ + @SuppressWarnings("NonAtomicOperationOnVolatileField") public Pair cursorUpdated(ManagedCursor cursor, Position newPosition) { requireNonNull(cursor); @@ -168,6 +223,7 @@ public Pair cursorUpdated(ManagedCursor cursor, Posi PositionImpl previousSlowestConsumer = heap.get(0).position; item.position = (PositionImpl) newPosition; + version = DataVersion.incrementVersion(version); if (heap.size() == 1) { return Pair.of(previousSlowestConsumer, item.position); @@ -210,11 +266,19 @@ public ManagedCursor getSlowestReader() { } } + /** + * @return Returns the CursorInfo for the cursor with oldest mark-delete position, or null if there aren't + * any tracked cursors + */ public CursorInfo getCursorWithOldestMarkDeletePosition() { long stamp = rwLock.readLock(); try { - Item item = heap.get(0); - return heap.isEmpty() ? null : new CursorInfo(item.cursor, item.position); + if (heap.isEmpty()) { + return null; + } else { + Item item = heap.get(0); + return new CursorInfo(item.cursor, item.position, version); + } } finally { rwLock.unlockRead(stamp); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BacklogQuotaManager.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BacklogQuotaManager.java index 6ad1697adfc39..16b2b3111c513 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BacklogQuotaManager.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BacklogQuotaManager.java @@ -18,10 +18,12 @@ */ package org.apache.pulsar.broker.service; +import static java.util.concurrent.TimeUnit.SECONDS; import java.util.ArrayList; import java.util.List; import java.util.Map; import java.util.concurrent.CompletableFuture; +import lombok.Getter; import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.mledger.ManagedCursor; import org.apache.bookkeeper.mledger.ManagedCursor.IndividualDeletedEntries; @@ -32,6 +34,7 @@ import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.resources.NamespaceResources; import org.apache.pulsar.broker.service.persistent.PersistentTopic; +import org.apache.pulsar.broker.service.persistent.PersistentTopicMetrics.BacklogQuotaMetrics; import org.apache.pulsar.common.naming.NamespaceName; import org.apache.pulsar.common.policies.data.BacklogQuota; import org.apache.pulsar.common.policies.data.BacklogQuota.BacklogQuotaType; @@ -41,9 +44,12 @@ @Slf4j public class BacklogQuotaManager { + @Getter private final BacklogQuotaImpl defaultQuota; private final NamespaceResources namespaceResources; + + public BacklogQuotaManager(PulsarService pulsar) { double backlogQuotaGB = pulsar.getConfiguration().getBacklogQuotaDefaultLimitGB(); this.defaultQuota = BacklogQuotaImpl.builder() @@ -55,10 +61,6 @@ public BacklogQuotaManager(PulsarService pulsar) { this.namespaceResources = pulsar.getPulsarResources().getNamespaceResources(); } - public BacklogQuotaImpl getDefaultQuota() { - return this.defaultQuota; - } - public BacklogQuotaImpl getBacklogQuota(NamespaceName namespace, BacklogQuotaType backlogQuotaType) { try { if (namespaceResources == null) { @@ -86,17 +88,21 @@ public BacklogQuotaImpl getBacklogQuota(NamespaceName namespace, BacklogQuotaTyp public void handleExceededBacklogQuota(PersistentTopic persistentTopic, BacklogQuotaType backlogQuotaType, boolean preciseTimeBasedBacklogQuotaCheck) { BacklogQuota quota = persistentTopic.getBacklogQuota(backlogQuotaType); + BacklogQuotaMetrics topicBacklogQuotaMetrics = + persistentTopic.getPersistentTopicMetrics().getBacklogQuotaMetrics(); log.info("Backlog quota type {} exceeded for topic [{}]. Applying [{}] policy", backlogQuotaType, persistentTopic.getName(), quota.getPolicy()); switch (quota.getPolicy()) { - case consumer_backlog_eviction: + case consumer_backlog_eviction: switch (backlogQuotaType) { case destination_storage: - dropBacklogForSizeLimit(persistentTopic, quota); - break; + dropBacklogForSizeLimit(persistentTopic, quota); + topicBacklogQuotaMetrics.recordSizeBasedBacklogEviction(); + break; case message_age: - dropBacklogForTimeLimit(persistentTopic, quota, preciseTimeBasedBacklogQuotaCheck); - break; + dropBacklogForTimeLimit(persistentTopic, quota, preciseTimeBasedBacklogQuotaCheck); + topicBacklogQuotaMetrics.recordTimeBasedBacklogEviction(); + break; default: break; } @@ -194,6 +200,7 @@ private void dropBacklogForSizeLimit(PersistentTopic persistentTopic, BacklogQuo * @param quota * Backlog quota set for the topic */ + @SuppressWarnings("checkstyle:LineLength") private void dropBacklogForTimeLimit(PersistentTopic persistentTopic, BacklogQuota quota, boolean preciseTimeBasedBacklogQuotaCheck) { // If enabled precise time based backlog quota check, will expire message based on the timeBaseQuota @@ -210,7 +217,7 @@ private void dropBacklogForTimeLimit(PersistentTopic persistentTopic, BacklogQuo ); } else { // If disabled precise time based backlog quota check, will try to remove whole ledger from cursor's backlog - Long currentMillis = ((ManagedLedgerImpl) persistentTopic.getManagedLedger()).getClock().millis(); + long currentMillis = ((ManagedLedgerImpl) persistentTopic.getManagedLedger()).getClock().millis(); ManagedLedgerImpl mLedger = (ManagedLedgerImpl) persistentTopic.getManagedLedger(); try { for (; ; ) { @@ -229,7 +236,7 @@ private void dropBacklogForTimeLimit(PersistentTopic persistentTopic, BacklogQuo } // Timestamp only > 0 if ledger has been closed if (ledgerInfo.getTimestamp() > 0 - && currentMillis - ledgerInfo.getTimestamp() > quota.getLimitTime() * 1000) { + && currentMillis - ledgerInfo.getTimestamp() > SECONDS.toMillis(quota.getLimitTime())) { // skip whole ledger for the slowest cursor PositionImpl nextPosition = PositionImpl.get(mLedger.getNextValidLedger(ledgerInfo.getLedgerId()), -1); @@ -263,19 +270,20 @@ private void disconnectProducers(PersistentTopic persistentTopic) { futures.add(producer.disconnect()); }); - FutureUtil.waitForAll(futures).thenRun(() -> { - log.info("All producers on topic [{}] are disconnected", persistentTopic.getName()); - }).exceptionally(exception -> { - log.error("Error in disconnecting producers on topic [{}] [{}]", persistentTopic.getName(), exception); - return null; - + FutureUtil.waitForAll(futures) + .thenRun(() -> + log.info("All producers on topic [{}] are disconnected", persistentTopic.getName())) + .exceptionally(exception -> { + log.error("Error in disconnecting producers on topic [{}] [{}]", persistentTopic.getName(), + exception); + return null; }); } /** * Advances the slowest cursor if that is a system cursor. * - * @param persistentTopic + * @param persistentTopic Persistent topic * @return true if the slowest cursor is a system cursor */ private boolean advanceSlowestSystemCursor(PersistentTopic persistentTopic) { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java index caefe36073ad8..15ab8f73d33f9 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java @@ -36,6 +36,7 @@ import io.netty.channel.socket.SocketChannel; import io.netty.handler.ssl.SslContext; import io.netty.util.concurrent.DefaultThreadFactory; +import io.prometheus.client.Histogram; import java.io.Closeable; import java.io.IOException; import java.lang.reflect.Field; @@ -199,6 +200,12 @@ public class BrokerService implements Closeable { private static final double GRACEFUL_SHUTDOWN_QUIET_PERIOD_RATIO_OF_TOTAL_TIMEOUT = 0.25d; private static final double GRACEFUL_SHUTDOWN_TIMEOUT_RATIO_OF_TOTAL_TIMEOUT = 0.5d; + private static final Histogram backlogQuotaCheckDuration = Histogram.build() + .name("pulsar_storage_backlog_quota_check_duration_seconds") + .help("The duration of the backlog quota check process.") + .buckets(5, 10, 30, 60, 300) + .register(); + private final PulsarService pulsar; private final ManagedLedgerFactory managedLedgerFactory; @@ -2179,27 +2186,29 @@ public BacklogQuotaManager getBacklogQuotaManager() { } public void monitorBacklogQuota() { - forEachPersistentTopic(topic -> { - if (topic.isSizeBacklogExceeded()) { - getBacklogQuotaManager().handleExceededBacklogQuota(topic, - BacklogQuota.BacklogQuotaType.destination_storage, false); - } else { - topic.checkTimeBacklogExceeded().thenAccept(isExceeded -> { - if (isExceeded) { - getBacklogQuotaManager().handleExceededBacklogQuota(topic, - BacklogQuota.BacklogQuotaType.message_age, - pulsar.getConfiguration().isPreciseTimeBasedBacklogQuotaCheck()); - } else { - if (log.isDebugEnabled()) { - log.debug("quota not exceeded for [{}]", topic.getName()); + backlogQuotaCheckDuration.time(() -> { + forEachPersistentTopic(topic -> { + if (topic.isSizeBacklogExceeded()) { + getBacklogQuotaManager().handleExceededBacklogQuota(topic, + BacklogQuota.BacklogQuotaType.destination_storage, false); + } else { + topic.checkTimeBacklogExceeded().thenAccept(isExceeded -> { + if (isExceeded) { + getBacklogQuotaManager().handleExceededBacklogQuota(topic, + BacklogQuota.BacklogQuotaType.message_age, + pulsar.getConfiguration().isPreciseTimeBasedBacklogQuotaCheck()); + } else { + if (log.isDebugEnabled()) { + log.debug("quota not exceeded for [{}]", topic.getName()); + } } - } - }).exceptionally(throwable -> { - log.error("Error when checkTimeBacklogExceeded({}) in monitorBacklogQuota", - topic.getName(), throwable); - return null; - }); - } + }).exceptionally(throwable -> { + log.error("Error when checkTimeBacklogExceeded({}) in monitorBacklogQuota", + topic.getName(), throwable); + return null; + }); + } + }); }); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Topic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Topic.java index 0cf3c55d9d392..82bbc4c093f8b 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Topic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Topic.java @@ -68,7 +68,7 @@ default void setOriginalSequenceId(long originalSequenceId) { /** * Return the producer name for the original producer. - * + *

* For messages published locally, this will return the same local producer name, though in case of replicated * messages, the original producer name will differ */ @@ -136,7 +136,7 @@ default void setEntryTimestamp(long entryTimestamp) { /** * Tries to add a producer to the topic. Several validations will be performed. * - * @param producer + * @param producer Producer to add * @param producerQueuedFuture * a future that will be triggered if the producer is being queued up prior of getting established * @return the "topic epoch" if there is one or empty @@ -148,7 +148,7 @@ default void setEntryTimestamp(long entryTimestamp) { /** * Wait TransactionBuffer Recovers completely. * Take snapshot after TB Recovers completely. - * @param isTxnEnabled + * @param isTxnEnabled isTxnEnabled * @return a future which has completely if isTxn = false. Or a future return by takeSnapshot. */ CompletableFuture checkIfTransactionBufferRecoverCompletely(boolean isTxnEnabled); @@ -263,6 +263,13 @@ CompletableFuture close( BacklogQuota getBacklogQuota(BacklogQuotaType backlogQuotaType); + /** + * Uses the best-effort (not necessarily up-to-date) information available to return the age. + * @return The oldest unacknowledged message age in seconds, or -1 if not available + */ + long getBestEffortOldestUnacknowledgedMessageAgeSeconds(); + + void updateRates(NamespaceStats nsStats, NamespaceBundleStats currentBundleStats, StatsOutputStream topicStatsStream, ClusterReplicationMetrics clusterReplicationMetrics, String namespaceName, boolean hydratePublishers); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopic.java index 67a338d08bbe3..c98ccb9b473cd 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopic.java @@ -168,7 +168,7 @@ public CompletableFuture initialize() { .getPoliciesAsync(TopicName.get(topic).getNamespaceObject()) .thenCompose(optPolicies -> { final Policies policies; - if (!optPolicies.isPresent()) { + if (optPolicies.isEmpty()) { log.warn("[{}] Policies not present and isEncryptionRequired will be set to false", topic); isEncryptionRequired = false; policies = new Policies(); @@ -1256,4 +1256,9 @@ protected boolean isMigrated() { public boolean isPersistent() { return false; } + + @Override + public long getBestEffortOldestUnacknowledgedMessageAgeSeconds() { + return -1; + } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java index 4f727bfaad16b..646b5563926cb 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java @@ -49,6 +49,7 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.atomic.AtomicReferenceFieldUpdater; import java.util.function.BiFunction; import java.util.stream.Collectors; import javax.annotation.Nonnull; @@ -266,10 +267,14 @@ protected TopicStatsHelper initialValue() { protected final TransactionBuffer transactionBuffer; // Record the last time a data message (ie: not an internal Pulsar marker) is published on the topic + @Getter private volatile long lastDataMessagePublishedTimestamp = 0; @Getter private final ExecutorService orderedExecutor; + @Getter + private final PersistentTopicMetrics persistentTopicMetrics = new PersistentTopicMetrics(); + private static class TopicStatsHelper { public double averageMsgSize; public double aggMsgRateIn; @@ -484,7 +489,7 @@ public CompletableFuture unloadSubscription(@Nonnull String subName) { if (!lock.writeLock().tryLock()) { return CompletableFuture.failedFuture(new SubscriptionConflictUnloadException(String.format("Conflict" + " topic-close, topic-delete, another-subscribe-unload, cannot unload subscription %s now", - topic, subName))); + subName))); } try { if (isFenced) { @@ -2387,6 +2392,14 @@ public CompletableFuture asyncGetStats(GetStatsOptions stats.lastOffloadFailureTimeStamp = ledger.getLastOffloadedFailureTimestamp(); Optional mxBean = getCompactorMXBean(); + stats.backlogQuotaLimitSize = getBacklogQuota(BacklogQuotaType.destination_storage).getLimitSize(); + stats.backlogQuotaLimitTime = getBacklogQuota(BacklogQuotaType.message_age).getLimitTime(); + + TimeBasedBacklogQuotaCheckResult backlogQuotaCheckResult = timeBasedBacklogQuotaCheckResult; + stats.oldestBacklogMessageAgeSeconds = TimeUnit.MILLISECONDS.toSeconds( + Clock.systemUTC().millis() - backlogQuotaCheckResult.getPositionPublishTimestampInMillis()); + stats.oldestBacklogMessageSubscriptionName = backlogQuotaCheckResult.getCursorName(); + stats.compaction.reset(); mxBean.flatMap(bean -> bean.getCompactionRecordForTopic(topic)).map(compactionRecord -> { stats.compaction.lastCompactionRemovedEventCount = compactionRecord.getLastCompactionRemovedEventCount(); @@ -3158,10 +3171,42 @@ public boolean isSizeBacklogExceeded() { private static class TimeBasedBacklogQuotaCheckResult { PositionImpl oldestCursorMarkDeletePosition; String cursorName; - Long positionPublishTimestamp; + long positionPublishTimestampInMillis; + long dataVersion; } private volatile TimeBasedBacklogQuotaCheckResult timeBasedBacklogQuotaCheckResult; + private static final AtomicReferenceFieldUpdater + TIME_BASED_BACKLOG_QUOTA_CHECK_RESULT_UPDATER = AtomicReferenceFieldUpdater.newUpdater( + PersistentTopic.class, + TimeBasedBacklogQuotaCheckResult.class, + "timeBasedBacklogQuotaCheckResult"); + + private static final long NOT_AVAILABLE_YET = -1; + + @Override + public long getBestEffortOldestUnacknowledgedMessageAgeSeconds() { + TimeBasedBacklogQuotaCheckResult result = timeBasedBacklogQuotaCheckResult; + if (result == null) { + return NOT_AVAILABLE_YET; + } else { + return TimeUnit.MILLISECONDS.toSeconds( + Clock.systemUTC().millis() - result.getPositionPublishTimestampInMillis()); + } + } + + private void updateResultIfNewer(TimeBasedBacklogQuotaCheckResult updatedResult) { + TIME_BASED_BACKLOG_QUOTA_CHECK_RESULT_UPDATER.updateAndGet(this, + existingResult -> { + if (ManagedCursorContainer.DataVersion.compareVersions( + updatedResult.getDataVersion(), existingResult.getDataVersion()) > 0) { + return updatedResult; + } else { + return existingResult; + } + }); + + } /** * @return determine if backlog quota enforcement needs to be done for topic based on time limit @@ -3188,18 +3233,16 @@ public CompletableFuture checkTimeBacklogExceeded() { // Same position, but the cursor causing it has changed? if (!lastCheckResult.getCursorName().equals(oldestMarkDeleteCursorInfo.getCursor().getName())) { - lastCheckResult = new TimeBasedBacklogQuotaCheckResult( + final TimeBasedBacklogQuotaCheckResult updatedResult = new TimeBasedBacklogQuotaCheckResult( lastCheckResult.getOldestCursorMarkDeletePosition(), oldestMarkDeleteCursorInfo.getCursor().getName(), - lastCheckResult.getPositionPublishTimestamp()); + lastCheckResult.getPositionPublishTimestampInMillis(), + oldestMarkDeleteCursorInfo.getVersion()); - timeBasedBacklogQuotaCheckResult = lastCheckResult; + updateResultIfNewer(updatedResult); } - Long entryTimestamp = lastCheckResult.getPositionPublishTimestamp(); - if (entryTimestamp == null) { - return CompletableFuture.completedFuture(false); - } + long entryTimestamp = lastCheckResult.getPositionPublishTimestampInMillis(); boolean expired = MessageImpl.isEntryExpired(backlogQuotaLimitInSecond, entryTimestamp); if (expired && log.isDebugEnabled()) { log.debug("(Using cache) Time based backlog quota exceeded, oldest entry in cursor {}'s backlog" @@ -3221,10 +3264,12 @@ public void readEntryComplete(Entry entry, Object ctx) { try { long entryTimestamp = Commands.getEntryTimestamp(entry.getDataBuffer()); - timeBasedBacklogQuotaCheckResult = new TimeBasedBacklogQuotaCheckResult( - oldestMarkDeleteCursorInfo.getMarkDeletePosition(), - oldestMarkDeleteCursorInfo.getCursor().getName(), - entryTimestamp); + updateResultIfNewer( + new TimeBasedBacklogQuotaCheckResult( + oldestMarkDeleteCursorInfo.getMarkDeletePosition(), + oldestMarkDeleteCursorInfo.getCursor().getName(), + entryTimestamp, + oldestMarkDeleteCursorInfo.getVersion())); boolean expired = MessageImpl.isEntryExpired(backlogQuotaLimitInSecond, entryTimestamp); if (expired && log.isDebugEnabled()) { @@ -3252,12 +3297,16 @@ public void readEntryFailed(ManagedLedgerException exception, Object ctx) { } else { try { CheckResult checkResult = estimatedTimeBasedBacklogQuotaCheck(oldestMarkDeletePosition); - timeBasedBacklogQuotaCheckResult = new TimeBasedBacklogQuotaCheckResult( - oldestMarkDeleteCursorInfo.getMarkDeletePosition(), - oldestMarkDeleteCursorInfo.getCursor().getName(), - checkResult.getEstimatedOldestUnacknowledgedMessageTimestamp()); + if (checkResult.getEstimatedOldestUnacknowledgedMessageTimestamp() != null) { + updateResultIfNewer( + new TimeBasedBacklogQuotaCheckResult( + oldestMarkDeleteCursorInfo.getMarkDeletePosition(), + oldestMarkDeleteCursorInfo.getCursor().getName(), + checkResult.getEstimatedOldestUnacknowledgedMessageTimestamp(), + oldestMarkDeleteCursorInfo.getVersion())); + } - return CompletableFuture.completedFuture(checkResult.isShouldTruncateBacklogToMatchQuota()); + return CompletableFuture.completedFuture(checkResult.isTruncateBacklogToMatchQuota()); } catch (Exception e) { log.error("[{}][{}] Error reading entry for precise time based backlog check", topicName, e); return CompletableFuture.completedFuture(false); @@ -3267,9 +3316,10 @@ public void readEntryFailed(ManagedLedgerException exception, Object ctx) { @Value private static class CheckResult { - boolean shouldTruncateBacklogToMatchQuota; + boolean truncateBacklogToMatchQuota; Long estimatedOldestUnacknowledgedMessageTimestamp; } + private CheckResult estimatedTimeBasedBacklogQuotaCheck(PositionImpl markDeletePosition) throws ExecutionException, InterruptedException { int backlogQuotaLimitInSecond = getBacklogQuota(BacklogQuotaType.message_age).getLimitTime(); @@ -3919,10 +3969,6 @@ private CompletableFuture transactionBufferCleanupAndClose() { return transactionBuffer.clearSnapshot().thenCompose(__ -> transactionBuffer.closeAsync()); } - public long getLastDataMessagePublishedTimestamp() { - return lastDataMessagePublishedTimestamp; - } - public Optional getShadowSourceTopic() { return Optional.ofNullable(shadowSourceTopic); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopicMetrics.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopicMetrics.java new file mode 100644 index 0000000000000..50ce1e0db7755 --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopicMetrics.java @@ -0,0 +1,33 @@ +package org.apache.pulsar.broker.service.persistent; + +import java.util.concurrent.atomic.LongAdder; +import lombok.Getter; + +@SuppressWarnings("LombokGetterMayBeUsed") +public class PersistentTopicMetrics { + + @Getter + private final BacklogQuotaMetrics backlogQuotaMetrics = new BacklogQuotaMetrics(); + + + public static class BacklogQuotaMetrics { + private final LongAdder timeBasedBacklogQuotaExceededEvictionCount = new LongAdder(); + private final LongAdder sizeBasedBacklogQuotaExceededEvictionCount = new LongAdder(); + + public void recordTimeBasedBacklogEviction() { + timeBasedBacklogQuotaExceededEvictionCount.increment(); + } + + public void recordSizeBasedBacklogEviction() { + sizeBasedBacklogQuotaExceededEvictionCount.increment(); + } + + public long getSizeBasedBacklogQuotaExceededEvictionCount() { + return sizeBasedBacklogQuotaExceededEvictionCount.longValue(); + } + + public long getTimeBasedBacklogQuotaExceededEvictionCount() { + return timeBasedBacklogQuotaExceededEvictionCount.longValue(); + } + } +} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/AggregatedBrokerStats.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/AggregatedBrokerStats.java index 715231d3c6ee1..037fb29a999e3 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/AggregatedBrokerStats.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/AggregatedBrokerStats.java @@ -33,7 +33,10 @@ public class AggregatedBrokerStats { public double storageReadRate; public double storageReadCacheMissesRate; public long msgBacklog; + public long sizeBasedBacklogQuotaExceededEvictionCount; + public long timeBasedBacklogQuotaExceededEvictionCount; + @SuppressWarnings("DuplicatedCode") void updateStats(TopicStats stats) { topicsCount++; subscriptionsCount += stats.subscriptionsCount; @@ -49,8 +52,11 @@ void updateStats(TopicStats stats) { storageReadRate += stats.managedLedgerStats.storageReadRate; storageReadCacheMissesRate += stats.managedLedgerStats.storageReadCacheMissesRate; msgBacklog += stats.msgBacklog; + timeBasedBacklogQuotaExceededEvictionCount += stats.timeBasedBacklogQuotaExceededEvictionCount; + sizeBasedBacklogQuotaExceededEvictionCount += stats.sizeBasedBacklogQuotaExceededEvictionCount; } + @SuppressWarnings("DuplicatedCode") public void reset() { topicsCount = 0; subscriptionsCount = 0; @@ -66,5 +72,7 @@ public void reset() { storageReadRate = 0; storageReadCacheMissesRate = 0; msgBacklog = 0; + sizeBasedBacklogQuotaExceededEvictionCount = 0; + timeBasedBacklogQuotaExceededEvictionCount = 0; } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/AggregatedNamespaceStats.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/AggregatedNamespaceStats.java index d0dc4fe2a7e7d..3975cd89cfa6b 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/AggregatedNamespaceStats.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/AggregatedNamespaceStats.java @@ -51,6 +51,9 @@ public class AggregatedNamespaceStats { long backlogQuotaLimit; long backlogQuotaLimitTime; + public long sizeBasedBacklogQuotaExceededEvictionCount; + public long timeBasedBacklogQuotaExceededEvictionCount; + public Map replicationStats = new HashMap<>(); public Map subscriptionStats = new HashMap<>(); @@ -68,6 +71,7 @@ public class AggregatedNamespaceStats { Map bucketDelayedIndexStats = new HashMap<>(); + @SuppressWarnings("DuplicatedCode") void updateStats(TopicStats stats) { topicsCount++; @@ -105,6 +109,9 @@ void updateStats(TopicStats stats) { backlogQuotaLimit = Math.max(backlogQuotaLimit, stats.backlogQuotaLimit); backlogQuotaLimitTime = Math.max(backlogQuotaLimitTime, stats.backlogQuotaLimitTime); + sizeBasedBacklogQuotaExceededEvictionCount += stats.sizeBasedBacklogQuotaExceededEvictionCount; + timeBasedBacklogQuotaExceededEvictionCount += stats.timeBasedBacklogQuotaExceededEvictionCount; + managedLedgerStats.storageWriteRate += stats.managedLedgerStats.storageWriteRate; managedLedgerStats.storageReadRate += stats.managedLedgerStats.storageReadRate; managedLedgerStats.storageReadCacheMissesRate += stats.managedLedgerStats.storageReadCacheMissesRate; @@ -172,6 +179,7 @@ void updateStats(TopicStats stats) { compactionLatencyBuckets.addAll(stats.compactionLatencyBuckets); } + @SuppressWarnings("DuplicatedCode") public void reset() { managedLedgerStats.reset(); topicsCount = 0; @@ -201,6 +209,9 @@ public void reset() { replicationStats.clear(); subscriptionStats.clear(); + sizeBasedBacklogQuotaExceededEvictionCount = 0; + timeBasedBacklogQuotaExceededEvictionCount = 0; + compactionRemovedEventCount = 0; compactionSucceedCount = 0; compactionFailedCount = 0; diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/NamespaceStatsAggregator.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/NamespaceStatsAggregator.java index 4e72fa0d72b16..3728c3edd1e8b 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/NamespaceStatsAggregator.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/NamespaceStatsAggregator.java @@ -32,7 +32,10 @@ import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.service.Topic; import org.apache.pulsar.broker.service.persistent.PersistentTopic; -import org.apache.pulsar.common.policies.data.BacklogQuota; +import org.apache.pulsar.broker.service.persistent.PersistentTopicMetrics; +import org.apache.pulsar.broker.service.persistent.PersistentTopicMetrics.BacklogQuotaMetrics; +import org.apache.pulsar.broker.stats.prometheus.metrics.PrometheusLabels; +import org.apache.pulsar.common.policies.data.BacklogQuota.BacklogQuotaType; import org.apache.pulsar.common.policies.data.stats.ConsumerStatsImpl; import org.apache.pulsar.common.policies.data.stats.NonPersistentSubscriptionStatsImpl; import org.apache.pulsar.common.policies.data.stats.NonPersistentTopicStatsImpl; @@ -159,14 +162,15 @@ private static void aggregateTopicStats(TopicStats stats, SubscriptionStatsImpl subsStats.bucketDelayedIndexStats = subscriptionStats.bucketDelayedIndexStats; } + @SuppressWarnings("OptionalUsedAsFieldOrParameterType") private static void getTopicStats(Topic topic, TopicStats stats, boolean includeConsumerMetrics, boolean includeProducerMetrics, boolean getPreciseBacklog, boolean subscriptionBacklogSize, Optional compactorMXBean) { stats.reset(); - if (topic instanceof PersistentTopic) { + if (topic instanceof PersistentTopic persistentTopic) { // Managed Ledger stats - ManagedLedger ml = ((PersistentTopic) topic).getManagedLedger(); + ManagedLedger ml = persistentTopic.getManagedLedger(); ManagedLedgerMBeanImpl mlStats = (ManagedLedgerMBeanImpl) ml.getStats(); stats.managedLedgerStats.storageSize = mlStats.getStoredMessagesSize(); @@ -174,9 +178,10 @@ private static void getTopicStats(Topic topic, TopicStats stats, boolean include stats.managedLedgerStats.backlogSize = ml.getEstimatedBacklogSize(); stats.managedLedgerStats.offloadedStorageUsed = ml.getOffloadedSize(); stats.backlogQuotaLimit = topic - .getBacklogQuota(BacklogQuota.BacklogQuotaType.destination_storage).getLimitSize(); + .getBacklogQuota(BacklogQuotaType.destination_storage).getLimitSize(); stats.backlogQuotaLimitTime = topic - .getBacklogQuota(BacklogQuota.BacklogQuotaType.message_age).getLimitTime(); + .getBacklogQuota(BacklogQuotaType.message_age).getLimitTime(); + stats.backlogAgeSeconds = topic.getBestEffortOldestUnacknowledgedMessageAgeSeconds(); stats.managedLedgerStats.storageWriteLatencyBuckets .addAll(mlStats.getInternalAddEntryLatencyBuckets()); @@ -191,7 +196,17 @@ private static void getTopicStats(Topic topic, TopicStats stats, boolean include stats.managedLedgerStats.storageWriteRate = mlStats.getAddEntryMessagesRate(); stats.managedLedgerStats.storageReadRate = mlStats.getReadEntriesRate(); stats.managedLedgerStats.storageReadCacheMissesRate = mlStats.getReadEntriesOpsCacheMissesRate(); + + // Topic Stats + PersistentTopicMetrics persistentTopicMetrics = persistentTopic.getPersistentTopicMetrics(); + + BacklogQuotaMetrics backlogQuotaMetrics = persistentTopicMetrics.getBacklogQuotaMetrics(); + stats.sizeBasedBacklogQuotaExceededEvictionCount = + backlogQuotaMetrics.getSizeBasedBacklogQuotaExceededEvictionCount(); + stats.timeBasedBacklogQuotaExceededEvictionCount = + backlogQuotaMetrics.getTimeBasedBacklogQuotaExceededEvictionCount(); } + TopicStatsImpl tStatus = topic.getStats(getPreciseBacklog, subscriptionBacklogSize, false); stats.msgInCounter = tStatus.msgInCounter; stats.bytesInCounter = tStatus.bytesInCounter; @@ -334,6 +349,14 @@ private static void printBrokerStats(PrometheusMetricStreams stream, String clus writeMetric(stream, "pulsar_broker_storage_read_rate", brokerStats.storageReadRate, cluster); writeMetric(stream, "pulsar_broker_storage_read_cache_misses_rate", brokerStats.storageReadCacheMissesRate, cluster); + + writePulsarBacklogQuotaMetricBrokerLevel(stream, + "pulsar_broker_storage_backlog_quota_exceeded_evictions_total", + brokerStats.sizeBasedBacklogQuotaExceededEvictionCount, cluster, BacklogQuotaType.destination_storage); + writePulsarBacklogQuotaMetricBrokerLevel(stream, + "pulsar_broker_storage_backlog_quota_exceeded_evictions_total", + brokerStats.timeBasedBacklogQuotaExceededEvictionCount, cluster, BacklogQuotaType.message_age); + writeMetric(stream, "pulsar_broker_msg_backlog", brokerStats.msgBacklog, cluster); } @@ -372,6 +395,7 @@ private static void printNamespaceStats(PrometheusMetricStreams stream, Aggregat stats.managedLedgerStats.storageLogicalSize, cluster, namespace); writeMetric(stream, "pulsar_storage_backlog_size", stats.managedLedgerStats.backlogSize, cluster, namespace); + writeMetric(stream, "pulsar_storage_offloaded_size", stats.managedLedgerStats.offloadedStorageUsed, cluster, namespace); @@ -392,6 +416,14 @@ private static void printNamespaceStats(PrometheusMetricStreams stream, Aggregat }); writePulsarMsgBacklog(stream, stats.msgBacklog, cluster, namespace); + writePulsarBacklogQuotaMetricNamespaceLevel(stream, + "pulsar_storage_backlog_quota_exceeded_evictions_total", + stats.sizeBasedBacklogQuotaExceededEvictionCount, cluster, namespace, + BacklogQuotaType.destination_storage); + writePulsarBacklogQuotaMetricNamespaceLevel(stream, + "pulsar_storage_backlog_quota_exceeded_evictions_total", + stats.timeBasedBacklogQuotaExceededEvictionCount, cluster, namespace, + BacklogQuotaType.message_age); stats.managedLedgerStats.storageWriteLatencyBuckets.refresh(); long[] latencyBuckets = stats.managedLedgerStats.storageWriteLatencyBuckets.getBuckets(); @@ -471,6 +503,25 @@ private static void printNamespaceStats(PrometheusMetricStreams stream, Aggregat replStats -> replStats.replicationDelayInSeconds, cluster, namespace); } + @SuppressWarnings("SameParameterValue") + private static void writePulsarBacklogQuotaMetricBrokerLevel(PrometheusMetricStreams stream, String metricName, + Number value, String cluster, + BacklogQuotaType backlogQuotaType) { + String quotaTypeLabelValue = PrometheusLabels.backlogQuotaTypeLabel(backlogQuotaType); + stream.writeSample(metricName, value, "cluster", cluster, + "quota_type", quotaTypeLabelValue); + } + + @SuppressWarnings("SameParameterValue") + private static void writePulsarBacklogQuotaMetricNamespaceLevel(PrometheusMetricStreams stream, String metricName, + Number value, String cluster, String namespace, + BacklogQuotaType backlogQuotaType) { + String quotaTypeLabelValue = PrometheusLabels.backlogQuotaTypeLabel(backlogQuotaType); + stream.writeSample(metricName, value, "cluster", cluster, + "namespace", namespace, + "quota_type", quotaTypeLabelValue); + } + private static void writePulsarMsgBacklog(PrometheusMetricStreams stream, Number value, String cluster, String namespace) { stream.writeSample("pulsar_msg_backlog", value, "cluster", cluster, "namespace", namespace, diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/TopicStats.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/TopicStats.java index dda03e3e59dd4..eeda0308b2144 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/TopicStats.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/TopicStats.java @@ -25,6 +25,8 @@ import org.apache.bookkeeper.mledger.util.StatsBuckets; import org.apache.commons.lang3.ArrayUtils; import org.apache.pulsar.broker.service.Consumer; +import org.apache.pulsar.broker.stats.prometheus.metrics.PrometheusLabels; +import org.apache.pulsar.common.policies.data.BacklogQuota.BacklogQuotaType; import org.apache.pulsar.common.policies.data.stats.TopicMetricBean; import org.apache.pulsar.compaction.CompactionRecord; import org.apache.pulsar.compaction.CompactorMXBean; @@ -52,6 +54,7 @@ class TopicStats { long backlogQuotaLimit; long backlogQuotaLimitTime; + long backlogAgeSeconds; ManagedLedgerStats managedLedgerStats = new ManagedLedgerStats(); @@ -73,6 +76,11 @@ class TopicStats { Map bucketDelayedIndexStats = new HashMap<>(); + public long sizeBasedBacklogQuotaExceededEvictionCount; + public long timeBasedBacklogQuotaExceededEvictionCount; + + + @SuppressWarnings("DuplicatedCode") public void reset() { subscriptionsCount = 0; producersCount = 0; @@ -111,8 +119,12 @@ public void reset() { compactionLatencyBuckets.reset(); delayedMessageIndexSizeInBytes = 0; bucketDelayedIndexStats.clear(); + + timeBasedBacklogQuotaExceededEvictionCount = 0; + sizeBasedBacklogQuotaExceededEvictionCount = 0; } + @SuppressWarnings("OptionalUsedAsFieldOrParameterType") public static void printTopicStats(PrometheusMetricStreams stream, TopicStats stats, Optional compactorMXBean, String cluster, String namespace, String topic, boolean splitTopicAndPartitionIndexLabel) { @@ -165,6 +177,14 @@ public static void printTopicStats(PrometheusMetricStreams stream, TopicStats st cluster, namespace, topic, splitTopicAndPartitionIndexLabel); writeMetric(stream, "pulsar_storage_backlog_quota_limit_time", stats.backlogQuotaLimitTime, cluster, namespace, topic, splitTopicAndPartitionIndexLabel); + writeMetric(stream, "pulsar_storage_backlog_age_seconds", stats.backlogAgeSeconds, + cluster, namespace, topic, splitTopicAndPartitionIndexLabel); + writeBacklogQuotaMetric(stream, "pulsar_storage_backlog_quota_exceeded_evictions_total", + stats.sizeBasedBacklogQuotaExceededEvictionCount, cluster, namespace, topic, + splitTopicAndPartitionIndexLabel, BacklogQuotaType.destination_storage); + writeBacklogQuotaMetric(stream, "pulsar_storage_backlog_quota_exceeded_evictions_total", + stats.timeBasedBacklogQuotaExceededEvictionCount, cluster, namespace, topic, + splitTopicAndPartitionIndexLabel, BacklogQuotaType.message_age); writeMetric(stream, "pulsar_delayed_message_index_size_bytes", stats.delayedMessageIndexSizeInBytes, cluster, namespace, topic, splitTopicAndPartitionIndexLabel); @@ -442,6 +462,17 @@ private static void writeMetric(PrometheusMetricStreams stream, String metricNam writeTopicMetric(stream, metricName, value, cluster, namespace, topic, splitTopicAndPartitionIndexLabel); } + @SuppressWarnings({"CheckStyle", "SameParameterValue"}) + private static void writeBacklogQuotaMetric(PrometheusMetricStreams stream, String metricName, Number value, + String cluster, String namespace, String topic, + boolean splitTopicAndPartitionIndexLabel, + BacklogQuotaType backlogQuotaType) { + + String quotaTypeLabelValue = PrometheusLabels.backlogQuotaTypeLabel(backlogQuotaType); + writeTopicMetric(stream, metricName, value, cluster, namespace, topic, splitTopicAndPartitionIndexLabel, + "quota_type", quotaTypeLabelValue); + } + private static void writeMetric(PrometheusMetricStreams stream, String metricName, Number value, String cluster, String namespace, String topic, String remoteCluster, boolean splitTopicAndPartitionIndexLabel) { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/metrics/PrometheusLabels.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/metrics/PrometheusLabels.java new file mode 100644 index 0000000000000..558f8946a497e --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/metrics/PrometheusLabels.java @@ -0,0 +1,14 @@ +package org.apache.pulsar.broker.stats.prometheus.metrics; + +import org.apache.pulsar.common.policies.data.BacklogQuota.BacklogQuotaType; + +public class PrometheusLabels { + + public static String backlogQuotaTypeLabel(BacklogQuotaType backlogQuotaType) { + if (backlogQuotaType == BacklogQuotaType.message_age) { + return "time"; + } else /* destination_storage */ { + return "size"; + } + } +} diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicPoliciesTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicPoliciesTest.java index 25ca3bf1444d2..2dbf12a15b10f 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicPoliciesTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicPoliciesTest.java @@ -145,7 +145,7 @@ public void updatePropertiesForAutoCreatedTopicTest() throws Exception { @Cleanup Producer producer = pulsarClient.newProducer().topic(testTopic).create(); HashMap properties = new HashMap<>(); - properties.put("backlogQuotaType", "message_age"); + properties.put("backlogQuotaTypeLabel", "message_age"); admin.topics().updateProperties(testTopic, properties); admin.topics().delete(topicName.toString(), true); } diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/TopicStatsImpl.java b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/TopicStatsImpl.java index e022c885d663b..2381aef52c2a9 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/TopicStatsImpl.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/TopicStatsImpl.java @@ -84,6 +84,31 @@ public class TopicStatsImpl implements TopicStats { /** Get estimated total unconsumed or backlog size in bytes. */ public long backlogSize; + /** the size in bytes of the topic backlog quota. */ + public long backlogQuotaLimitSize; + + /** the topic backlog age quota, in seconds. */ + public long backlogQuotaLimitTime; + + /** + * Age of oldest unacknowledged message, as recorded in last backlog quota check interval. + *

+ * The age of the oldest unacknowledged (i.e. backlog) message, measured by the time elapsed from its published + * time, in seconds. This value is recorded every backlog quota check interval, hence it represents the value + * seen in the last check. + *

+ */ + public long oldestBacklogMessageAgeSeconds; + + /** + * The subscription name containing oldest unacknowledged message as recorded in last backlog quota check. + *

+ * The name of the subscription containing the oldest unacknowledged message. This value is recorded every backlog + * quota check interval, hence it represents the value seen in the last check. + *

+ */ + public String oldestBacklogMessageSubscriptionName; + /** The number of times the publishing rate limit was triggered. */ public long publishRateLimitedTimes; From da42aa8c80dfcf7096a3ce9a90fa681b10a32fae Mon Sep 17 00:00:00 2001 From: Asaf Mesika Date: Tue, 26 Dec 2023 13:00:03 +0200 Subject: [PATCH 3/9] Part of test --- .../mledger/impl/ManagedCursorContainer.java | 23 +- .../impl/ManagedCursorContainerTest.java | 123 ++++++-- pom.xml | 8 + pulsar-broker/pom.xml | 8 +- .../service/persistent/PersistentTopic.java | 75 +++-- .../service/BacklogQuotaManagerTest.java | 155 ++++++++- .../persistent/BucketDelayedDeliveryTest.java | 28 +- .../persistent/PersistentTopicTest.java | 8 +- .../service/schema/SchemaServiceTest.java | 22 +- .../broker/stats/ConsumerStatsTest.java | 7 +- .../broker/stats/MetadataStoreStatsTest.java | 29 +- .../broker/stats/PrometheusMetricsTest.java | 296 +++++++----------- .../broker/stats/SubscriptionStatsTest.java | 11 +- .../broker/stats/TransactionMetricsTest.java | 25 +- .../prometheus/PrometheusMetricsClient.java | 105 +++++++ .../buffer/TransactionBufferClientTest.java | 18 +- .../pendingack/PendingAckPersistentTest.java | 16 +- .../pulsar/broker/web/WebServiceTest.java | 24 +- .../common/policies/data/TopicStats.java | 25 ++ .../policies/data/stats/TopicStatsImpl.java | 10 + 20 files changed, 683 insertions(+), 333 deletions(-) create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsClient.java diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorContainer.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorContainer.java index a6e590c8042e6..e5850d398b316 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorContainer.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorContainer.java @@ -29,6 +29,7 @@ import org.apache.bookkeeper.mledger.ManagedCursor; import org.apache.bookkeeper.mledger.Position; import org.apache.commons.lang3.tuple.Pair; +import org.slf4j.LoggerFactory; /** * Contains cursors for a ManagedLedger. @@ -45,6 +46,8 @@ */ public class ManagedCursorContainer implements Iterable { + private static final org.slf4j.Logger log = LoggerFactory.getLogger(ManagedCursorContainer.class); + /** * This field is incremented everytime the cursor information is updated. */ @@ -78,6 +81,17 @@ private static class Item { public static final class DataVersion { private DataVersion() {} + /** + * Compares two data versions, which either rolls overs to 0 when reaching Long.MAX_VALUE. + *

+ * Use {@link DataVersion#incrementVersion(long)} to increment the versions + *

+ * @param v1 First version to compare + * @param v2 Second version to compare + * @return the value {@code 0} if {@code v1 == v2}; + * a value less than {@code 0} if {@code v1 < v2}; and + * a value greater than {@code 0} if {@code v1 > v2} + */ public static int compareVersions(long v1, long v2) { if (v1 == v2) { return 0; @@ -88,9 +102,9 @@ public static int compareVersions(long v1, long v2) { long distance = v2 - v1; long wrapAroundDistance = (Long.MAX_VALUE - v2) + v1; if (distance < wrapAroundDistance) { - return 1; - } else { return -1; + } else { + return 1; } // 0-------v2--------v1--------MAX_LONG @@ -98,9 +112,9 @@ public static int compareVersions(long v1, long v2) { long distance = v1 - v2; long wrapAroundDistance = (Long.MAX_VALUE - v1) + v2; if (distance < wrapAroundDistance) { - return -1; // v1 is bigger + return 1; // v1 is bigger } else { - return 1; // v2 is bigger + return -1; // v2 is bigger } } } @@ -222,6 +236,7 @@ public Pair cursorUpdated(ManagedCursor cursor, Posi } PositionImpl previousSlowestConsumer = heap.get(0).position; + log.info("Cursor {} change position from {} to {}", cursor.getName(), item.position, newPosition); item.position = (PositionImpl) newPosition; version = DataVersion.incrementVersion(version); diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorContainerTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorContainerTest.java index 04d99d3bdf480..5c4dbe8913632 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorContainerTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorContainerTest.java @@ -18,6 +18,7 @@ */ package org.apache.bookkeeper.mledger.impl; +import static org.assertj.core.api.Assertions.assertThat; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.spy; import static org.testng.Assert.assertEquals; @@ -46,7 +47,6 @@ import org.apache.bookkeeper.mledger.ManagedCursor; import org.apache.bookkeeper.mledger.ManagedCursorMXBean; import org.apache.bookkeeper.mledger.ManagedLedger; -import org.apache.bookkeeper.mledger.ManagedLedgerException; import org.apache.bookkeeper.mledger.Position; import org.testng.annotations.Test; @@ -105,7 +105,7 @@ public boolean isDurable() { } @Override - public List readEntries(int numberOfEntriesToRead) throws ManagedLedgerException { + public List readEntries(int numberOfEntriesToRead) { return new ArrayList(); } @@ -137,14 +137,14 @@ public long getNumberOfEntriesInBacklog(boolean isPrecise) { } @Override - public void markDelete(Position position) throws ManagedLedgerException { + public void markDelete(Position position) { markDelete(position, Collections.emptyMap()); } @Override - public void markDelete(Position position, Map properties) throws ManagedLedgerException { + public void markDelete(Position position, Map properties) { this.position = position; - container.cursorUpdated(this, (PositionImpl) position); + container.cursorUpdated(this, position); } @Override @@ -209,7 +209,7 @@ public void asyncClose(AsyncCallbacks.CloseCallback callback, Object ctx) { } @Override - public void delete(Position position) throws InterruptedException, ManagedLedgerException { + public void delete(Position position) { } @Override @@ -217,7 +217,7 @@ public void asyncDelete(Position position, DeleteCallback callback, Object ctx) } @Override - public void delete(Iterable positions) throws InterruptedException, ManagedLedgerException { + public void delete(Iterable positions) { } @Override @@ -225,7 +225,7 @@ public void asyncDelete(Iterable position, DeleteCallback callback, Ob } @Override - public void clearBacklog() throws InterruptedException, ManagedLedgerException { + public void clearBacklog() { } @Override @@ -233,8 +233,7 @@ public void asyncClearBacklog(ClearBacklogCallback callback, Object ctx) { } @Override - public void skipEntries(int numEntriesToSkip, IndividualDeletedEntries deletedEntries) - throws InterruptedException, ManagedLedgerException { + public void skipEntries(int numEntriesToSkip, IndividualDeletedEntries deletedEntries) { } @Override @@ -243,13 +242,12 @@ public void asyncSkipEntries(int numEntriesToSkip, IndividualDeletedEntries dele } @Override - public Position findNewestMatching(Predicate condition) - throws InterruptedException, ManagedLedgerException { + public Position findNewestMatching(Predicate condition) { return null; } @Override - public Position findNewestMatching(FindPositionConstraint constraint, Predicate condition) throws InterruptedException, ManagedLedgerException { + public Position findNewestMatching(FindPositionConstraint constraint, Predicate condition) { return null; } @@ -270,7 +268,7 @@ public void asyncResetCursor(final Position position, boolean forceReset, } @Override - public void resetCursor(final Position position) throws ManagedLedgerException, InterruptedException { + public void resetCursor(final Position position) { } @@ -284,8 +282,7 @@ public void setAlwaysInactive() { } @Override - public List replayEntries(Set positions) - throws InterruptedException, ManagedLedgerException { + public List replayEntries(Set positions) { return null; } @@ -300,8 +297,7 @@ public Set asyncReplayEntries(Set positi } @Override - public List readEntriesOrWait(int numberOfEntriesToRead) - throws InterruptedException, ManagedLedgerException { + public List readEntriesOrWait(int numberOfEntriesToRead) { return null; } @@ -322,8 +318,7 @@ public boolean cancelPendingReadRequest() { } @Override - public Entry getNthEntry(int N, IndividualDeletedEntries deletedEntries) - throws InterruptedException, ManagedLedgerException { + public Entry getNthEntry(int N, IndividualDeletedEntries deletedEntries) { return null; } @@ -399,13 +394,8 @@ public ManagedCursorMXBean getStats() { return null; } - public void asyncReadEntriesOrWait(int maxEntries, long maxSizeBytes, ReadEntriesCallback callback, - Object ctx) { - } - @Override - public List readEntriesOrWait(int maxEntries, long maxSizeBytes) - throws InterruptedException, ManagedLedgerException { + public List readEntriesOrWait(int maxEntries, long maxSizeBytes) { return null; } @@ -421,7 +411,7 @@ public boolean isClosed() { } @Test - public void testSlowestReadPositionForActiveCursors() throws Exception { + public void testSlowestReadPositionForActiveCursors() { ManagedCursorContainer container = new ManagedCursorContainer(); assertNull(container.getSlowestReaderPosition()); @@ -466,14 +456,20 @@ public void simple() throws Exception { ManagedCursor cursor1 = new MockManagedCursor(container, "test1", new PositionImpl(5, 5)); container.add(cursor1, cursor1.getMarkDeletedPosition()); assertEquals(container.getSlowestReaderPosition(), new PositionImpl(5, 5)); + assertEqualsCursorAndMarkDelete(container.getCursorWithOldestMarkDeletePosition(), + cursor1, new PositionImpl(5, 5)); ManagedCursor cursor2 = new MockManagedCursor(container, "test2", new PositionImpl(2, 2)); container.add(cursor2, cursor2.getMarkDeletedPosition()); assertEquals(container.getSlowestReaderPosition(), new PositionImpl(2, 2)); + assertEqualsCursorAndMarkDelete(container.getCursorWithOldestMarkDeletePosition(), + cursor2, new PositionImpl(2, 2)); ManagedCursor cursor3 = new MockManagedCursor(container, "test3", new PositionImpl(2, 0)); container.add(cursor3, cursor3.getMarkDeletedPosition()); assertEquals(container.getSlowestReaderPosition(), new PositionImpl(2, 0)); + assertEqualsCursorAndMarkDelete(container.getCursorWithOldestMarkDeletePosition(), + cursor3, new PositionImpl(2, 0)); assertEquals(container.toString(), "[test1=5:5, test2=2:2, test3=2:0]"); @@ -487,6 +483,8 @@ public void simple() throws Exception { cursor3.markDelete(new PositionImpl(3, 0)); assertEquals(container.getSlowestReaderPosition(), new PositionImpl(2, 2)); + assertEqualsCursorAndMarkDelete(container.getCursorWithOldestMarkDeletePosition(), + cursor2, new PositionImpl(2, 2)); cursor2.markDelete(new PositionImpl(10, 5)); assertEquals(container.getSlowestReaderPosition(), new PositionImpl(3, 0)); @@ -498,6 +496,8 @@ public void simple() throws Exception { container.removeCursor(cursor5.getName()); container.removeCursor(cursor1.getName()); assertEquals(container.getSlowestReaderPosition(), new PositionImpl(4, 0)); + assertEqualsCursorAndMarkDelete(container.getCursorWithOldestMarkDeletePosition(), + cursor4, new PositionImpl(4, 0)); assertTrue(container.hasDurableCursors()); @@ -514,7 +514,7 @@ public void simple() throws Exception { } @Test - public void updatingCursorOutsideContainer() throws Exception { + public void updatingCursorOutsideContainer() { ManagedCursorContainer container = new ManagedCursorContainer(); ManagedCursor cursor1 = new MockManagedCursor(container, "test1", new PositionImpl(5, 5)); @@ -533,10 +533,19 @@ public void updatingCursorOutsideContainer() throws Exception { container.cursorUpdated(cursor2, cursor2.position); assertEquals(container.getSlowestReaderPosition(), new PositionImpl(5, 5)); + assertEqualsCursorAndMarkDelete(container.getCursorWithOldestMarkDeletePosition(), + cursor1, new PositionImpl(5, 5)); + } + + private void assertEqualsCursorAndMarkDelete(ManagedCursorContainer.CursorInfo cursorInfo, + ManagedCursor expectedCursor, + PositionImpl expectedMarkDeletePosition) { + assertThat(cursorInfo.getCursor().getName()).isEqualTo(expectedCursor.getName()); + assertThat(cursorInfo.getMarkDeletePosition()).isEqualTo(expectedMarkDeletePosition); } @Test - public void removingCursor() throws Exception { + public void removingCursor() { ManagedCursorContainer container = new ManagedCursorContainer(); ManagedCursor cursor1 = new MockManagedCursor(container, "test1", new PositionImpl(5, 5)); @@ -607,7 +616,7 @@ public void ordering() throws Exception { } @Test - public void orderingWithUpdates() throws Exception { + public void orderingWithUpdates() { ManagedCursorContainer container = new ManagedCursorContainer(); MockManagedCursor c1 = new MockManagedCursor(container, "test1", new PositionImpl(5, 5)); @@ -672,7 +681,7 @@ public void orderingWithUpdates() throws Exception { } @Test - public void orderingWithUpdatesAndReset() throws Exception { + public void orderingWithUpdatesAndReset() { ManagedCursorContainer container = new ManagedCursorContainer(); MockManagedCursor c1 = new MockManagedCursor(container, "test1", new PositionImpl(5, 5)); @@ -735,4 +744,56 @@ public void orderingWithUpdatesAndReset() throws Exception { assertFalse(container.hasDurableCursors()); } + + @Test + public void testDataVersion() { + assertThat(ManagedCursorContainer.DataVersion.compareVersions(1L, 3L)).isNegative(); + assertThat(ManagedCursorContainer.DataVersion.compareVersions(3L, 1L)).isPositive(); + assertThat(ManagedCursorContainer.DataVersion.compareVersions(3L, 3L)).isZero(); + + long v1 = Long.MAX_VALUE - 1; + long v2 = ManagedCursorContainer.DataVersion.incrementVersion(v1); + + assertThat(ManagedCursorContainer.DataVersion.compareVersions(v1, v2)).isNegative(); + + v2 = ManagedCursorContainer.DataVersion.incrementVersion(v2); + assertThat(ManagedCursorContainer.DataVersion.compareVersions(v1, v2)).isNegative(); + + v1 = ManagedCursorContainer.DataVersion.incrementVersion(v1); + assertThat(ManagedCursorContainer.DataVersion.compareVersions(v1, v2)).isNegative(); + + v1 = ManagedCursorContainer.DataVersion.incrementVersion(v1); + assertThat(ManagedCursorContainer.DataVersion.compareVersions(v1, v2)).isZero(); + + v1 = ManagedCursorContainer.DataVersion.incrementVersion(v1); + assertThat(ManagedCursorContainer.DataVersion.compareVersions(v1, v2)).isPositive(); + } + + @Test + public void testVersions() { + ManagedCursorContainer container = new ManagedCursorContainer(); + + MockManagedCursor c1 = new MockManagedCursor(container, "test1", new PositionImpl(5, 5)); + MockManagedCursor c2 = new MockManagedCursor(container, "test2", new PositionImpl(5, 1)); + + container.add(c1, c1.getMarkDeletedPosition()); + long version = container.getCursorWithOldestMarkDeletePosition().getVersion(); + + container.add(c2, c2.getMarkDeletedPosition()); + long newVersion = container.getCursorWithOldestMarkDeletePosition().getVersion(); + // newVersion > version + assertThat(ManagedCursorContainer.DataVersion.compareVersions(newVersion, version)).isPositive(); + version = newVersion; + + container.cursorUpdated(c2, new PositionImpl(5, 8)); + newVersion = container.getCursorWithOldestMarkDeletePosition().getVersion(); + // newVersion > version + assertThat(ManagedCursorContainer.DataVersion.compareVersions(newVersion, version)).isPositive(); + version = newVersion; + + container.removeCursor("test2"); + newVersion = container.getCursorWithOldestMarkDeletePosition().getVersion(); + // newVersion > version + assertThat(ManagedCursorContainer.DataVersion.compareVersions(newVersion, version)).isPositive(); + } } diff --git a/pom.xml b/pom.xml index 2ce1407ecfd81..a63067a56638e 100644 --- a/pom.xml +++ b/pom.xml @@ -253,6 +253,7 @@ flexible messaging model and an intuitive client API. 1.18.3 2.2 + 5.4.0 3.3.0 @@ -1423,6 +1424,13 @@ flexible messaging model and an intuitive client API. checker-qual ${checkerframework.version} + + + io.rest-assured + rest-assured + ${restassured.version} + test + diff --git a/pulsar-broker/pom.xml b/pulsar-broker/pom.xml index 6aac8959a1065..172b1cad7789b 100644 --- a/pulsar-broker/pom.xml +++ b/pulsar-broker/pom.xml @@ -26,7 +26,7 @@ org.apache.pulsar pulsar 3.2.0-SNAPSHOT - .. + ../pom.xml pulsar-broker @@ -430,6 +430,12 @@ javax.activation + + io.rest-assured + rest-assured + test + + diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java index 646b5563926cb..47a41295a5b0f 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java @@ -196,6 +196,14 @@ public class PersistentTopic extends AbstractTopic implements Topic, AddEntryCallback { + private static final AtomicReferenceFieldUpdater + TIME_BASED_BACKLOG_QUOTA_CHECK_RESULT_UPDATER = AtomicReferenceFieldUpdater.newUpdater( + PersistentTopic.class, + TimeBasedBacklogQuotaCheckResult.class, + "timeBasedBacklogQuotaCheckResult"); + + private static final long NOT_AVAILABLE_YET = -1; + // Managed ledger associated with the topic protected final ManagedLedger ledger; @@ -275,6 +283,22 @@ protected TopicStatsHelper initialValue() { @Getter private final PersistentTopicMetrics persistentTopicMetrics = new PersistentTopicMetrics(); + private volatile TimeBasedBacklogQuotaCheckResult timeBasedBacklogQuotaCheckResult; + + @Value + private static class TimeBasedBacklogQuotaCheckResult { + PositionImpl oldestCursorMarkDeletePosition; + String cursorName; + long positionPublishTimestampInMillis; + long dataVersion; + } + + @Value + private static class CheckResult { + boolean truncateBacklogToMatchQuota; + Long estimatedOldestUnacknowledgedMessageTimestamp; + } + private static class TopicStatsHelper { public double averageMsgSize; public double aggMsgRateIn; @@ -2396,9 +2420,14 @@ public CompletableFuture asyncGetStats(GetStatsOptions stats.backlogQuotaLimitTime = getBacklogQuota(BacklogQuotaType.message_age).getLimitTime(); TimeBasedBacklogQuotaCheckResult backlogQuotaCheckResult = timeBasedBacklogQuotaCheckResult; - stats.oldestBacklogMessageAgeSeconds = TimeUnit.MILLISECONDS.toSeconds( + stats.oldestBacklogMessageAgeSeconds = (backlogQuotaCheckResult == null) + ? NOT_AVAILABLE_YET + : TimeUnit.MILLISECONDS.toSeconds( Clock.systemUTC().millis() - backlogQuotaCheckResult.getPositionPublishTimestampInMillis()); - stats.oldestBacklogMessageSubscriptionName = backlogQuotaCheckResult.getCursorName(); + + stats.oldestBacklogMessageSubscriptionName = (backlogQuotaCheckResult == null) + ? null + : backlogQuotaCheckResult.getCursorName(); stats.compaction.reset(); mxBean.flatMap(bean -> bean.getCompactionRecordForTopic(topic)).map(compactionRecord -> { @@ -3167,23 +3196,6 @@ public boolean isSizeBacklogExceeded() { return (storageSize >= backlogQuotaLimitInBytes); } - @Value - private static class TimeBasedBacklogQuotaCheckResult { - PositionImpl oldestCursorMarkDeletePosition; - String cursorName; - long positionPublishTimestampInMillis; - long dataVersion; - } - - private volatile TimeBasedBacklogQuotaCheckResult timeBasedBacklogQuotaCheckResult; - private static final AtomicReferenceFieldUpdater - TIME_BASED_BACKLOG_QUOTA_CHECK_RESULT_UPDATER = AtomicReferenceFieldUpdater.newUpdater( - PersistentTopic.class, - TimeBasedBacklogQuotaCheckResult.class, - "timeBasedBacklogQuotaCheckResult"); - - private static final long NOT_AVAILABLE_YET = -1; - @Override public long getBestEffortOldestUnacknowledgedMessageAgeSeconds() { TimeBasedBacklogQuotaCheckResult result = timeBasedBacklogQuotaCheckResult; @@ -3198,8 +3210,9 @@ public long getBestEffortOldestUnacknowledgedMessageAgeSeconds() { private void updateResultIfNewer(TimeBasedBacklogQuotaCheckResult updatedResult) { TIME_BASED_BACKLOG_QUOTA_CHECK_RESULT_UPDATER.updateAndGet(this, existingResult -> { - if (ManagedCursorContainer.DataVersion.compareVersions( - updatedResult.getDataVersion(), existingResult.getDataVersion()) > 0) { + if (existingResult == null + || ManagedCursorContainer.DataVersion.compareVersions( + updatedResult.getDataVersion(), existingResult.getDataVersion()) > 0) { return updatedResult; } else { return existingResult; @@ -3240,6 +3253,10 @@ public CompletableFuture checkTimeBacklogExceeded() { oldestMarkDeleteCursorInfo.getVersion()); updateResultIfNewer(updatedResult); + log.info("checkTimeBacklogExceeded for topic {}: Same location, different cursor. " + + "Cursor " + oldestMarkDeleteCursorInfo.getCursor().getName() + + ", markDeletePosition = " + oldestMarkDeleteCursorInfo.getMarkDeletePosition(), + topic); } long entryTimestamp = lastCheckResult.getPositionPublishTimestampInMillis(); @@ -3249,10 +3266,16 @@ public CompletableFuture checkTimeBacklogExceeded() { + "exceeded quota {}", lastCheckResult.getCursorName(), backlogQuotaLimitInSecond); } + log.info("checkTimeBacklogExceeded for topic {}: Same position, getting it from cache. " + + "Cursor " + oldestMarkDeleteCursorInfo.getCursor().getName() + + ", markDeletePosition = " + oldestMarkDeleteCursorInfo.getMarkDeletePosition(), topic); return CompletableFuture.completedFuture(expired); } if (brokerService.pulsar().getConfiguration().isPreciseTimeBasedBacklogQuotaCheck()) { + log.info("checkTimeBacklogExceeded for topic {}: Location changed. " + + "Cursor " + oldestMarkDeleteCursorInfo.getCursor().getName() + + ", markDeletePosition = " + oldestMarkDeleteCursorInfo.getMarkDeletePosition(), topic); CompletableFuture future = new CompletableFuture<>(); // Check if first unconsumed message(first message after mark delete position) // for slowest cursor's has expired. @@ -3295,6 +3318,10 @@ public void readEntryFailed(ManagedLedgerException exception, Object ctx) { }, null); return future; } else { + log.info("checkTimeBacklogExceeded for topic {}: Location changed (not precise). " + + "Cursor " + oldestMarkDeleteCursorInfo.getCursor().getName() + + ", markDeletePosition = " + oldestMarkDeleteCursorInfo.getMarkDeletePosition(), topic); + try { CheckResult checkResult = estimatedTimeBasedBacklogQuotaCheck(oldestMarkDeletePosition); if (checkResult.getEstimatedOldestUnacknowledgedMessageTimestamp() != null) { @@ -3314,12 +3341,6 @@ public void readEntryFailed(ManagedLedgerException exception, Object ctx) { } } - @Value - private static class CheckResult { - boolean truncateBacklogToMatchQuota; - Long estimatedOldestUnacknowledgedMessageTimestamp; - } - private CheckResult estimatedTimeBasedBacklogQuotaCheck(PositionImpl markDeletePosition) throws ExecutionException, InterruptedException { int backlogQuotaLimitInSecond = getBacklogQuota(BacklogQuotaType.message_age).getLimitTime(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BacklogQuotaManagerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BacklogQuotaManagerTest.java index 3c829b02cb858..f6ef30c7f3b5c 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BacklogQuotaManagerTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BacklogQuotaManagerTest.java @@ -18,6 +18,13 @@ */ package org.apache.pulsar.broker.service; +import static java.util.Map.entry; +import static java.util.concurrent.TimeUnit.MILLISECONDS; +import static java.util.concurrent.TimeUnit.SECONDS; +import static org.apache.pulsar.common.policies.data.BacklogQuota.BacklogQuotaType.destination_storage; +import static org.apache.pulsar.common.policies.data.BacklogQuota.BacklogQuotaType.message_age; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.AssertionsForClassTypes.within; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertFalse; import static org.testng.Assert.assertTrue; @@ -35,6 +42,7 @@ import java.util.concurrent.CyclicBarrier; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicLong; import lombok.Cleanup; import org.apache.bookkeeper.mledger.Position; import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; @@ -42,6 +50,9 @@ import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; import org.apache.pulsar.broker.service.persistent.PersistentTopic; +import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient; +import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient.Metric; +import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient.Metrics; import org.apache.pulsar.client.admin.GetStatsOptions; import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.admin.PulsarAdminException; @@ -73,6 +84,9 @@ @Test(groups = "broker") public class BacklogQuotaManagerTest { + private static final Logger log = LoggerFactory.getLogger(BacklogQuotaManagerTest.class); + + public static final String CLUSTER_NAME = "usc"; PulsarService pulsar; ServiceConfiguration config; @@ -80,6 +94,7 @@ public class BacklogQuotaManagerTest { PulsarAdmin admin; LocalBookkeeperEnsemble bkEnsemble; + PrometheusMetricsClient prometheusMetricsClient; private static final int TIME_TO_CHECK_BACKLOG_QUOTA = 2; private static final int MAX_ENTRIES_PER_LEDGER = 5; @@ -117,7 +132,7 @@ void setup() throws Exception { config.setMetadataStoreUrl("zk:127.0.0.1:" + bkEnsemble.getZookeeperPort()); config.setAdvertisedAddress("localhost"); config.setWebServicePort(Optional.of(0)); - config.setClusterName("usc"); + config.setClusterName(CLUSTER_NAME); config.setBrokerShutdownTimeoutMs(0L); config.setLoadBalancerOverrideBrokerNicSpeedGbps(Optional.of(1.0d)); config.setBrokerServicePort(Optional.of(0)); @@ -136,6 +151,7 @@ void setup() throws Exception { adminUrl = new URL("http://127.0.0.1" + ":" + pulsar.getListenPortHTTP().get()); admin = PulsarAdmin.builder().serviceHttpUrl(adminUrl.toString()).build(); + prometheusMetricsClient = new PrometheusMetricsClient("127.0.0.1", pulsar.getListenPortHTTP().get()); admin.clusters().createCluster("usc", ClusterData.builder().serviceUrl(adminUrl.toString()).build()); admin.tenants().createTenant("prop", @@ -344,6 +360,143 @@ public void testTriggerBacklogQuotaSizeWithReader() throws Exception { } } + @Test + public void backlogsStats() throws PulsarAdminException, PulsarClientException, InterruptedException { + final String namespace = "prop/ns-quota"; + assertEquals(admin.namespaces().getBacklogQuotaMap(namespace), new HashMap<>()); + final int sizeLimitBytes = 15 * 1024 * 1024; + final int timeLimitSeconds = 123; + admin.namespaces().setBacklogQuota( + namespace, + BacklogQuota.builder() + .limitSize(sizeLimitBytes) + .retentionPolicy(BacklogQuota.RetentionPolicy.producer_exception) + .build(), + destination_storage); + admin.namespaces().setBacklogQuota( + namespace, + BacklogQuota.builder() + .limitTime(timeLimitSeconds) + .retentionPolicy(BacklogQuota.RetentionPolicy.producer_exception) + .build(), + message_age); + + try (PulsarClient client = PulsarClient.builder().serviceUrl(adminUrl.toString()) + .statsInterval(0, TimeUnit.SECONDS).build()) { + final String topic1 = "persistent://prop/ns-quota/topic2" + UUID.randomUUID(); + + final String subName1 = "c1"; + final String subName2 = "c2"; + final int numMsgs = 20; + + Consumer consumer1 = client.newConsumer().topic(topic1).subscriptionName(subName1).subscribe(); + Consumer consumer2 = client.newConsumer().topic(topic1).subscriptionName(subName2).subscribe(); + org.apache.pulsar.client.api.Producer producer = createProducer(client, topic1); + + byte[] content = new byte[1024]; + List messageIds = new ArrayList<>(numMsgs); + for (int i = 0; i < numMsgs; i++) { + Thread.sleep(100); + MessageId messageId = producer.send(content); + messageIds.add(messageId); + } + + final long initialQuotaCheckCount = getQuotaCheckCount(); + // Move subscription 1, one message, such that subscription 2 is the oldest + Message oldestMessage = consumer1.receive(); + consumer1.acknowledge(oldestMessage); + + log.info("Subscription 1 moved 1 message. Now subscription 2 is the oldest"); + + AtomicLong quotaCheckCount = new AtomicLong(); + Awaitility.await().pollInterval(1, SECONDS).atMost(TIME_TO_CHECK_BACKLOG_QUOTA*3, SECONDS).until(() -> { + quotaCheckCount.set(getQuotaCheckCount()); + return quotaCheckCount.get() > initialQuotaCheckCount + 1; + }); + + Metrics metrics = prometheusMetricsClient.getMetrics(); + + TopicStats topicStats = getTopicStats(topic1); + assertThat(topicStats.getBacklogQuotaLimitSize()).isEqualTo(sizeLimitBytes); + assertThat(topicStats.getBacklogQuotaLimitTime()).isEqualTo(timeLimitSeconds); + + long expectedMessageAgeSeconds = MILLISECONDS.toSeconds(System.currentTimeMillis() - oldestMessage.getPublishTime()); + assertThat(topicStats.getOldestBacklogMessageAgeSeconds()) + .isCloseTo(expectedMessageAgeSeconds, within(1L)); + assertThat(topicStats.getOldestBacklogMessageSubscriptionName()).isEqualTo(subName2); + + List backlogAgeMetrics = + metrics.findByNameAndLabels("pulsar_storage_backlog_age_seconds", "topic", topic1); + assertThat(backlogAgeMetrics).hasSize(1); + Metric backlogAgeMetric = backlogAgeMetrics.get(0); + assertThat(backlogAgeMetric.tags).containsExactly( + entry("cluster", CLUSTER_NAME), + entry("namespace", namespace), + entry("topic", topic1)); + assertThat((long) backlogAgeMetric.value).isCloseTo(expectedMessageAgeSeconds, within(1L)); + + // Move subscription 2 away from being the oldest mark delete + Message secondOldestMessage = consumer2.receive(); + consumer2.acknowledge(secondOldestMessage); + + // Switch subscription 1 to be where subscription 2 was in terms of oldest mark delete + consumer1.seek(oldestMessage.getMessageId()); + + log.info("Subscription 1 moved to be the oldest"); + + final long firstQuotaCheckCount = quotaCheckCount.get(); + Awaitility.await().pollInterval(1, SECONDS).atMost(TIME_TO_CHECK_BACKLOG_QUOTA*3, SECONDS).until(() -> { + quotaCheckCount.set(getQuotaCheckCount()); + return quotaCheckCount.get() > firstQuotaCheckCount+1; + }); + + metrics = prometheusMetricsClient.getMetrics(); + long actualAge = (long) metrics.findByNameAndLabels( + "pulsar_storage_backlog_age_seconds", "topic", topic1) + .get(0).value; + + expectedMessageAgeSeconds = MILLISECONDS.toSeconds(System.currentTimeMillis() - oldestMessage.getPublishTime()); + assertThat(actualAge).isCloseTo(expectedMessageAgeSeconds, within(1L)); + + topicStats = getTopicStats(topic1); + assertThat(topicStats.getOldestBacklogMessageSubscriptionName()).isEqualTo(subName1); + + // Move subscription 1 passed subscription 2 + List ids = new ArrayList<>(); + for (int i = 0; i < 3; i++) { + Message message = consumer1.receive(); + ids.add(message.getMessageId()); + } + consumer1.acknowledge(ids); + + log.info("Subscription 1 moved 3 messages. Now subscription 2 is the oldest"); + + final long secondQuotaCheckCount = quotaCheckCount.get(); + Awaitility.await().pollInterval(1, SECONDS).atMost(TIME_TO_CHECK_BACKLOG_QUOTA*3, SECONDS).until(() -> { + quotaCheckCount.set(getQuotaCheckCount()); + boolean result = quotaCheckCount.get() > (secondQuotaCheckCount + 1); + System.out.println("Quota Check Count = " +quotaCheckCount.get() + + ", second quota check count = "+secondQuotaCheckCount + + ", result = " + result); + return result; + }); + + topicStats = getTopicStats(topic1); + expectedMessageAgeSeconds = MILLISECONDS.toSeconds(System.currentTimeMillis() - secondOldestMessage.getPublishTime()); + assertThat(topicStats.getOldestBacklogMessageAgeSeconds()).isCloseTo(expectedMessageAgeSeconds, within(1L)); + assertThat(topicStats.getOldestBacklogMessageSubscriptionName()).isEqualTo(subName2); + + } + } + + private long getQuotaCheckCount() { + Metrics metrics = prometheusMetricsClient.getMetrics(); + return (long) metrics.findByNameAndLabels( + "pulsar_storage_backlog_quota_check_duration_seconds_count", + "cluster", CLUSTER_NAME) + .get(0).value; + } + /** * Time based backlog quota won't affect reader since broker doesn't keep track of consuming position for reader * and can't do message age check against the quota. diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/BucketDelayedDeliveryTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/BucketDelayedDeliveryTest.java index 54fec3934ddbc..40f271e0940e8 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/BucketDelayedDeliveryTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/BucketDelayedDeliveryTest.java @@ -41,7 +41,7 @@ import org.apache.pulsar.broker.BrokerTestUtil; import org.apache.pulsar.broker.delayed.BucketDelayedDeliveryTrackerFactory; import org.apache.pulsar.broker.service.Dispatcher; -import org.apache.pulsar.broker.stats.PrometheusMetricsTest; +import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient; import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsGenerator; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.MessageId; @@ -219,9 +219,9 @@ public void testBucketDelayedIndexMetrics() throws Exception { ByteArrayOutputStream output = new ByteArrayOutputStream(); PrometheusMetricsGenerator.generate(pulsar, true, true, true, output); String metricsStr = output.toString(StandardCharsets.UTF_8); - Multimap metricsMap = PrometheusMetricsTest.parseMetrics(metricsStr); + Multimap metricsMap = PrometheusMetricsClient.parseMetrics(metricsStr); - List bucketsMetrics = + List bucketsMetrics = metricsMap.get("pulsar_delayed_message_index_bucket_total").stream() .filter(metric -> metric.tags.get("topic").equals(topic)).toList(); MutableInt bucketsSum = new MutableInt(); @@ -230,12 +230,12 @@ public void testBucketDelayedIndexMetrics() throws Exception { bucketsSum.add(metric.value); }); assertEquals(6, bucketsSum.intValue()); - Optional bucketsTopicMetric = + Optional bucketsTopicMetric = bucketsMetrics.stream().filter(metric -> !metric.tags.containsKey("subscription")).findFirst(); assertTrue(bucketsTopicMetric.isPresent()); assertEquals(bucketsSum.intValue(), bucketsTopicMetric.get().value); - List loadedIndexMetrics = + List loadedIndexMetrics = metricsMap.get("pulsar_delayed_message_index_loaded").stream() .filter(metric -> metric.tags.get("topic").equals(topic)).toList(); MutableInt loadedIndexSum = new MutableInt(); @@ -244,12 +244,12 @@ public void testBucketDelayedIndexMetrics() throws Exception { loadedIndexSum.add(metric.value); }).count(); assertEquals(2, count); - Optional loadedIndexTopicMetrics = + Optional loadedIndexTopicMetrics = bucketsMetrics.stream().filter(metric -> !metric.tags.containsKey("subscription")).findFirst(); assertTrue(loadedIndexTopicMetrics.isPresent()); assertEquals(loadedIndexSum.intValue(), loadedIndexTopicMetrics.get().value); - List snapshotSizeBytesMetrics = + List snapshotSizeBytesMetrics = metricsMap.get("pulsar_delayed_message_index_bucket_snapshot_size_bytes").stream() .filter(metric -> metric.tags.get("topic").equals(topic)).toList(); MutableInt snapshotSizeBytesSum = new MutableInt(); @@ -259,12 +259,12 @@ public void testBucketDelayedIndexMetrics() throws Exception { snapshotSizeBytesSum.add(metric.value); }).count(); assertEquals(2, count); - Optional snapshotSizeBytesTopicMetrics = + Optional snapshotSizeBytesTopicMetrics = snapshotSizeBytesMetrics.stream().filter(metric -> !metric.tags.containsKey("subscription")).findFirst(); assertTrue(snapshotSizeBytesTopicMetrics.isPresent()); assertEquals(snapshotSizeBytesSum.intValue(), snapshotSizeBytesTopicMetrics.get().value); - List opCountMetrics = + List opCountMetrics = metricsMap.get("pulsar_delayed_message_index_bucket_op_count").stream() .filter(metric -> metric.tags.get("topic").equals(topic)).toList(); MutableInt opCountMetricsSum = new MutableInt(); @@ -276,14 +276,14 @@ public void testBucketDelayedIndexMetrics() throws Exception { opCountMetricsSum.add(metric.value); }).count(); assertEquals(2, count); - Optional opCountTopicMetrics = + Optional opCountTopicMetrics = opCountMetrics.stream() .filter(metric -> metric.tags.get("state").equals("succeed") && metric.tags.get("type") .equals("create") && !metric.tags.containsKey("subscription")).findFirst(); assertTrue(opCountTopicMetrics.isPresent()); assertEquals(opCountMetricsSum.intValue(), opCountTopicMetrics.get().value); - List opLatencyMetrics = + List opLatencyMetrics = metricsMap.get("pulsar_delayed_message_index_bucket_op_latency_ms").stream() .filter(metric -> metric.tags.get("topic").equals(topic)).toList(); MutableInt opLatencyMetricsSum = new MutableInt(); @@ -295,7 +295,7 @@ public void testBucketDelayedIndexMetrics() throws Exception { opLatencyMetricsSum.add(metric.value); }).count(); assertTrue(count >= 2); - Optional opLatencyTopicMetrics = + Optional opLatencyTopicMetrics = opCountMetrics.stream() .filter(metric -> metric.tags.get("type").equals("create") && !metric.tags.containsKey("subscription")).findFirst(); @@ -304,9 +304,9 @@ public void testBucketDelayedIndexMetrics() throws Exception { ByteArrayOutputStream namespaceOutput = new ByteArrayOutputStream(); PrometheusMetricsGenerator.generate(pulsar, false, true, true, namespaceOutput); - Multimap namespaceMetricsMap = PrometheusMetricsTest.parseMetrics(namespaceOutput.toString(StandardCharsets.UTF_8)); + Multimap namespaceMetricsMap = PrometheusMetricsClient.parseMetrics(namespaceOutput.toString(StandardCharsets.UTF_8)); - Optional namespaceMetric = + Optional namespaceMetric = namespaceMetricsMap.get("pulsar_delayed_message_index_bucket_total").stream().findFirst(); assertTrue(namespaceMetric.isPresent()); assertEquals(6, namespaceMetric.get().value); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentTopicTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentTopicTest.java index 6f60a13fd4894..0e2ab7e58002f 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentTopicTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentTopicTest.java @@ -62,7 +62,7 @@ import org.apache.pulsar.broker.service.BrokerService; import org.apache.pulsar.broker.service.BrokerTestBase; import org.apache.pulsar.broker.service.TopicPoliciesService; -import org.apache.pulsar.broker.stats.PrometheusMetricsTest; +import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient; import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsGenerator; import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.api.Consumer; @@ -356,14 +356,14 @@ public void testDelayedDeliveryTrackerMemoryUsageMetric(String topic, boolean ex PrometheusMetricsGenerator.generate(pulsar, exposeTopicLevelMetrics, true, true, output); String metricsStr = output.toString(StandardCharsets.UTF_8); - Multimap metricsMap = PrometheusMetricsTest.parseMetrics(metricsStr); - Collection metrics = metricsMap.get("pulsar_delayed_message_index_size_bytes"); + Multimap metricsMap = PrometheusMetricsClient.parseMetrics(metricsStr); + Collection metrics = metricsMap.get("pulsar_delayed_message_index_size_bytes"); Assert.assertTrue(metrics.size() > 0); int topicLevelNum = 0; int namespaceLevelNum = 0; int subscriptionLevelNum = 0; - for (PrometheusMetricsTest.Metric metric : metrics) { + for (PrometheusMetricsClient.Metric metric : metrics) { if (exposeTopicLevelMetrics && metric.tags.get("topic").equals(topic)) { Assert.assertTrue(metric.value > 0); topicLevelNum++; diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/schema/SchemaServiceTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/schema/SchemaServiceTest.java index fbf734f331f2b..97bf6e172ddf3 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/schema/SchemaServiceTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/schema/SchemaServiceTest.java @@ -45,7 +45,7 @@ import org.apache.pulsar.broker.PulsarServerException; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; import org.apache.pulsar.broker.service.schema.SchemaRegistry.SchemaAndMetadata; -import org.apache.pulsar.broker.stats.PrometheusMetricsTest; +import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient; import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsGenerator; import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.impl.schema.KeyValueSchemaInfo; @@ -123,29 +123,29 @@ public void testSchemaRegistryMetrics() throws Exception { PrometheusMetricsGenerator.generate(pulsar, false, false, false, output); output.flush(); String metricsStr = output.toString(StandardCharsets.UTF_8); - Multimap metrics = PrometheusMetricsTest.parseMetrics(metricsStr); + Multimap metrics = PrometheusMetricsClient.parseMetrics(metricsStr); - Collection delMetrics = metrics.get("pulsar_schema_del_ops_failed_total"); + Collection delMetrics = metrics.get("pulsar_schema_del_ops_failed_total"); Assert.assertEquals(delMetrics.size(), 0); - Collection getMetrics = metrics.get("pulsar_schema_get_ops_failed_total"); + Collection getMetrics = metrics.get("pulsar_schema_get_ops_failed_total"); Assert.assertEquals(getMetrics.size(), 0); - Collection putMetrics = metrics.get("pulsar_schema_put_ops_failed_total"); + Collection putMetrics = metrics.get("pulsar_schema_put_ops_failed_total"); Assert.assertEquals(putMetrics.size(), 0); - Collection deleteLatency = metrics.get("pulsar_schema_del_ops_latency_count"); - for (PrometheusMetricsTest.Metric metric : deleteLatency) { + Collection deleteLatency = metrics.get("pulsar_schema_del_ops_latency_count"); + for (PrometheusMetricsClient.Metric metric : deleteLatency) { Assert.assertEquals(metric.tags.get("namespace"), namespace); Assert.assertTrue(metric.value > 0); } - Collection getLatency = metrics.get("pulsar_schema_get_ops_latency_count"); - for (PrometheusMetricsTest.Metric metric : getLatency) { + Collection getLatency = metrics.get("pulsar_schema_get_ops_latency_count"); + for (PrometheusMetricsClient.Metric metric : getLatency) { Assert.assertEquals(metric.tags.get("namespace"), namespace); Assert.assertTrue(metric.value > 0); } - Collection putLatency = metrics.get("pulsar_schema_put_ops_latency_count"); - for (PrometheusMetricsTest.Metric metric : putLatency) { + Collection putLatency = metrics.get("pulsar_schema_put_ops_latency_count"); + for (PrometheusMetricsClient.Metric metric : putLatency) { Assert.assertEquals(metric.tags.get("namespace"), namespace); Assert.assertTrue(metric.value > 0); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/ConsumerStatsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/ConsumerStatsTest.java index f29c643a8f50b..d04fcffd15bc8 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/ConsumerStatsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/ConsumerStatsTest.java @@ -50,6 +50,7 @@ import org.apache.pulsar.broker.service.plugin.EntryFilter; import org.apache.pulsar.broker.service.plugin.EntryFilterProducerTest; import org.apache.pulsar.broker.service.plugin.EntryFilterWithClassLoader; +import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient; import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsGenerator; import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.api.Consumer; @@ -336,11 +337,11 @@ private void testMessageAckRateMetric(String topicName, boolean exposeTopicLevel PrometheusMetricsGenerator.generate(pulsar, exposeTopicLevelMetrics, true, true, output); String metricStr = output.toString(StandardCharsets.UTF_8); - Multimap metricsMap = PrometheusMetricsTest.parseMetrics(metricStr); - Collection ackRateMetric = metricsMap.get("pulsar_consumer_msg_ack_rate"); + Multimap metricsMap = PrometheusMetricsClient.parseMetrics(metricStr); + Collection ackRateMetric = metricsMap.get("pulsar_consumer_msg_ack_rate"); String rateOutMetricName = exposeTopicLevelMetrics ? "pulsar_consumer_msg_rate_out" : "pulsar_rate_out"; - Collection rateOutMetric = metricsMap.get(rateOutMetricName); + Collection rateOutMetric = metricsMap.get(rateOutMetricName); Assert.assertTrue(ackRateMetric.size() > 0); Assert.assertTrue(rateOutMetric.size() > 0); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/MetadataStoreStatsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/MetadataStoreStatsTest.java index 8ae0242c6232a..71bb2c4fe07a2 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/MetadataStoreStatsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/MetadataStoreStatsTest.java @@ -31,6 +31,7 @@ import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.authentication.AuthenticationProviderToken; import org.apache.pulsar.broker.service.BrokerTestBase; +import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient; import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsGenerator; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.Message; @@ -101,12 +102,12 @@ public void testMetadataStoreStats() throws Exception { ByteArrayOutputStream output = new ByteArrayOutputStream(); PrometheusMetricsGenerator.generate(pulsar, false, false, false, false, output); String metricsStr = output.toString(); - Multimap metricsMap = PrometheusMetricsTest.parseMetrics(metricsStr); + Multimap metricsMap = PrometheusMetricsClient.parseMetrics(metricsStr); String metricsDebugMessage = "Assertion failed with metrics:\n" + metricsStr + "\n"; - Collection opsLatency = metricsMap.get("pulsar_metadata_store_ops_latency_ms" + "_sum"); - Collection putBytes = metricsMap.get("pulsar_metadata_store_put_bytes" + "_total"); + Collection opsLatency = metricsMap.get("pulsar_metadata_store_ops_latency_ms" + "_sum"); + Collection putBytes = metricsMap.get("pulsar_metadata_store_put_bytes" + "_total"); Assert.assertTrue(opsLatency.size() > 1, metricsDebugMessage); Assert.assertTrue(putBytes.size() > 1, metricsDebugMessage); @@ -116,7 +117,7 @@ public void testMetadataStoreStats() throws Exception { expectedMetadataStoreName.add(MetadataStoreConfig.CONFIGURATION_METADATA_STORE); AtomicInteger matchCount = new AtomicInteger(0); - for (PrometheusMetricsTest.Metric m : opsLatency) { + for (PrometheusMetricsClient.Metric m : opsLatency) { Assert.assertEquals(m.tags.get("cluster"), "test", metricsDebugMessage); String metadataStoreName = m.tags.get("name"); if (!isExpectedLabel(metadataStoreName, expectedMetadataStoreName, matchCount)) { @@ -150,7 +151,7 @@ public void testMetadataStoreStats() throws Exception { Assert.assertEquals(matchCount.get(), expectedMetadataStoreName.size() * 6); matchCount = new AtomicInteger(0); - for (PrometheusMetricsTest.Metric m : putBytes) { + for (PrometheusMetricsClient.Metric m : putBytes) { Assert.assertEquals(m.tags.get("cluster"), "test", metricsDebugMessage); String metadataStoreName = m.tags.get("name"); if (!isExpectedLabel(metadataStoreName, expectedMetadataStoreName, matchCount)) { @@ -191,12 +192,12 @@ public void testBatchMetadataStoreMetrics() throws Exception { ByteArrayOutputStream output = new ByteArrayOutputStream(); PrometheusMetricsGenerator.generate(pulsar, false, false, false, false, output); String metricsStr = output.toString(); - Multimap metricsMap = PrometheusMetricsTest.parseMetrics(metricsStr); + Multimap metricsMap = PrometheusMetricsClient.parseMetrics(metricsStr); - Collection executorQueueSize = metricsMap.get("pulsar_batch_metadata_store_executor_queue_size"); - Collection opsWaiting = metricsMap.get("pulsar_batch_metadata_store_queue_wait_time_ms" + "_sum"); - Collection batchExecuteTime = metricsMap.get("pulsar_batch_metadata_store_batch_execute_time_ms" + "_sum"); - Collection opsPerBatch = metricsMap.get("pulsar_batch_metadata_store_batch_size" + "_sum"); + Collection executorQueueSize = metricsMap.get("pulsar_batch_metadata_store_executor_queue_size"); + Collection opsWaiting = metricsMap.get("pulsar_batch_metadata_store_queue_wait_time_ms" + "_sum"); + Collection batchExecuteTime = metricsMap.get("pulsar_batch_metadata_store_batch_execute_time_ms" + "_sum"); + Collection opsPerBatch = metricsMap.get("pulsar_batch_metadata_store_batch_size" + "_sum"); String metricsDebugMessage = "Assertion failed with metrics:\n" + metricsStr + "\n"; @@ -210,7 +211,7 @@ public void testBatchMetadataStoreMetrics() throws Exception { expectedMetadataStoreName.add(MetadataStoreConfig.CONFIGURATION_METADATA_STORE); AtomicInteger matchCount = new AtomicInteger(0); - for (PrometheusMetricsTest.Metric m : executorQueueSize) { + for (PrometheusMetricsClient.Metric m : executorQueueSize) { Assert.assertEquals(m.tags.get("cluster"), "test", metricsDebugMessage); String metadataStoreName = m.tags.get("name"); if (isExpectedLabel(metadataStoreName, expectedMetadataStoreName, matchCount)) { @@ -221,7 +222,7 @@ public void testBatchMetadataStoreMetrics() throws Exception { Assert.assertEquals(matchCount.get(), expectedMetadataStoreName.size()); matchCount = new AtomicInteger(0); - for (PrometheusMetricsTest.Metric m : opsWaiting) { + for (PrometheusMetricsClient.Metric m : opsWaiting) { Assert.assertEquals(m.tags.get("cluster"), "test", metricsDebugMessage); String metadataStoreName = m.tags.get("name"); if (isExpectedLabel(metadataStoreName, expectedMetadataStoreName, matchCount)) { @@ -232,7 +233,7 @@ public void testBatchMetadataStoreMetrics() throws Exception { Assert.assertEquals(matchCount.get(), expectedMetadataStoreName.size()); matchCount = new AtomicInteger(0); - for (PrometheusMetricsTest.Metric m : batchExecuteTime) { + for (PrometheusMetricsClient.Metric m : batchExecuteTime) { Assert.assertEquals(m.tags.get("cluster"), "test", metricsDebugMessage); String metadataStoreName = m.tags.get("name"); if (isExpectedLabel(metadataStoreName, expectedMetadataStoreName, matchCount)) { @@ -243,7 +244,7 @@ public void testBatchMetadataStoreMetrics() throws Exception { Assert.assertEquals(matchCount.get(), expectedMetadataStoreName.size()); matchCount = new AtomicInteger(0); - for (PrometheusMetricsTest.Metric m : opsPerBatch) { + for (PrometheusMetricsClient.Metric m : opsPerBatch) { Assert.assertEquals(m.tags.get("cluster"), "test", metricsDebugMessage); String metadataStoreName = m.tags.get("name"); if (isExpectedLabel(metadataStoreName, expectedMetadataStoreName, matchCount)) { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/PrometheusMetricsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/PrometheusMetricsTest.java index 80f9cd8913f64..e7312431e2927 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/PrometheusMetricsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/PrometheusMetricsTest.java @@ -23,9 +23,7 @@ import static org.testng.Assert.assertNull; import static org.testng.Assert.assertTrue; import static org.testng.Assert.fail; -import com.google.common.base.MoreObjects; import com.google.common.base.Splitter; -import com.google.common.collect.ArrayListMultimap; import com.google.common.collect.Multimap; import io.jsonwebtoken.SignatureAlgorithm; import io.prometheus.client.Collector; @@ -49,7 +47,6 @@ import java.util.Properties; import java.util.Random; import java.util.Set; -import java.util.TreeMap; import java.util.UUID; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; @@ -73,6 +70,7 @@ import org.apache.pulsar.broker.service.persistent.PersistentMessageExpiryMonitor; import org.apache.pulsar.broker.service.persistent.PersistentSubscription; import org.apache.pulsar.broker.service.persistent.PersistentTopic; +import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient; import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsGenerator; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.MessageRoutingMode; @@ -169,7 +167,7 @@ private void checkPublishRateLimitedTimes(boolean preciseRateLimit) throws Excep ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut); String metricsStr = statsOut.toString(); - Multimap metrics = parseMetrics(metricsStr); + Multimap metrics = PrometheusMetricsClient.parseMetrics(metricsStr); assertTrue(metrics.containsKey("pulsar_publish_rate_limit_times")); metrics.get("pulsar_publish_rate_limit_times").forEach(item -> { if (ns1.equals(item.tags.get("namespace"))) { @@ -205,7 +203,7 @@ private void checkPublishRateLimitedTimes(boolean preciseRateLimit) throws Excep ByteArrayOutputStream statsOut2 = new ByteArrayOutputStream(); PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut2); String metricsStr2 = statsOut2.toString(); - Multimap metrics2 = parseMetrics(metricsStr2); + Multimap metrics2 = PrometheusMetricsClient.parseMetrics(metricsStr2); assertTrue(metrics2.containsKey("pulsar_publish_rate_limit_times")); metrics2.get("pulsar_publish_rate_limit_times").forEach(item -> { if (ns1.equals(item.tags.get("namespace"))) { @@ -237,8 +235,8 @@ public void testMetricsTopicCount() throws Exception { ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut); String metricsStr = statsOut.toString(); - Multimap metrics = parseMetrics(metricsStr); - Collection metric = metrics.get("pulsar_topics_count"); + Multimap metrics = PrometheusMetricsClient.parseMetrics(metricsStr); + Collection metric = metrics.get("pulsar_topics_count"); metric.forEach(item -> { if (ns1.equals(item.tags.get("namespace"))) { assertEquals(item.value, 6.0); @@ -247,12 +245,12 @@ public void testMetricsTopicCount() throws Exception { assertEquals(item.value, 3.0); } }); - Collection pulsarTopicLoadTimesMetrics = metrics.get("pulsar_topic_load_times"); - Collection pulsarTopicLoadTimesCountMetrics = metrics.get("pulsar_topic_load_times_count"); + Collection pulsarTopicLoadTimesMetrics = metrics.get("pulsar_topic_load_times"); + Collection pulsarTopicLoadTimesCountMetrics = metrics.get("pulsar_topic_load_times_count"); assertEquals(pulsarTopicLoadTimesMetrics.size(), 6); assertEquals(pulsarTopicLoadTimesCountMetrics.size(), 1); - Collection topicLoadTimeP999Metrics = metrics.get("pulsar_topic_load_time_99_9_percentile_ms"); - Collection topicLoadTimeFailedCountMetrics = metrics.get("pulsar_topic_load_failed_count"); + Collection topicLoadTimeP999Metrics = metrics.get("pulsar_topic_load_time_99_9_percentile_ms"); + Collection topicLoadTimeFailedCountMetrics = metrics.get("pulsar_topic_load_failed_count"); assertEquals(topicLoadTimeP999Metrics.size(), 1); assertEquals(topicLoadTimeFailedCountMetrics.size(), 1); } @@ -274,10 +272,10 @@ public void testMetricsAvgMsgSize2() throws Exception { ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); PrometheusMetricsGenerator.generate(pulsar, true, false, true, statsOut); String metricsStr = statsOut.toString(); - Multimap metrics = parseMetrics(metricsStr); + Multimap metrics = PrometheusMetricsClient.parseMetrics(metricsStr); assertTrue(metrics.containsKey("pulsar_average_msg_size")); assertEquals(metrics.get("pulsar_average_msg_size").size(), 1); - Collection avgMsgSizes = metrics.get("pulsar_average_msg_size"); + Collection avgMsgSizes = metrics.get("pulsar_average_msg_size"); avgMsgSizes.forEach(item -> { if (ns1.equals(item.tags.get("namespace"))) { assertEquals(item.value, 10); @@ -317,50 +315,50 @@ public void testPerTopicStats() throws Exception { ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut); String metricsStr = statsOut.toString(); - Multimap metrics = parseMetrics(metricsStr); + Multimap metrics = PrometheusMetricsClient.parseMetrics(metricsStr); metrics.entries().forEach(e -> { System.out.println(e.getKey() + ": " + e.getValue()); }); // There should be 2 metrics with different tags for each topic - List cm = (List) metrics.get("pulsar_storage_write_latency_le_1"); + List cm = (List) metrics.get("pulsar_storage_write_latency_le_1"); assertEquals(cm.size(), 2); assertEquals(cm.get(0).tags.get("topic"), "persistent://my-property/use/my-ns/my-topic2"); assertEquals(cm.get(0).tags.get("namespace"), "my-property/use/my-ns"); assertEquals(cm.get(1).tags.get("topic"), "persistent://my-property/use/my-ns/my-topic1"); assertEquals(cm.get(1).tags.get("namespace"), "my-property/use/my-ns"); - cm = (List) metrics.get("pulsar_producers_count"); + cm = (List) metrics.get("pulsar_producers_count"); assertEquals(cm.size(), 2); assertEquals(cm.get(1).tags.get("topic"), "persistent://my-property/use/my-ns/my-topic1"); assertEquals(cm.get(1).tags.get("namespace"), "my-property/use/my-ns"); assertEquals(cm.get(1).tags.get("topic"), "persistent://my-property/use/my-ns/my-topic1"); assertEquals(cm.get(1).tags.get("namespace"), "my-property/use/my-ns"); - cm = (List) metrics.get("pulsar_topic_load_times_count"); + cm = (List) metrics.get("pulsar_topic_load_times_count"); assertEquals(cm.size(), 1); assertEquals(cm.get(0).tags.get("cluster"), "test"); - cm = (List) metrics.get("topic_load_failed_total"); + cm = (List) metrics.get("topic_load_failed_total"); assertEquals(cm.size(), 1); assertEquals(cm.get(0).tags.get("cluster"), "test"); - cm = (List) metrics.get("pulsar_in_bytes_total"); + cm = (List) metrics.get("pulsar_in_bytes_total"); assertEquals(cm.size(), 2); assertEquals(cm.get(0).tags.get("topic"), "persistent://my-property/use/my-ns/my-topic2"); assertEquals(cm.get(0).tags.get("namespace"), "my-property/use/my-ns"); assertEquals(cm.get(1).tags.get("topic"), "persistent://my-property/use/my-ns/my-topic1"); assertEquals(cm.get(1).tags.get("namespace"), "my-property/use/my-ns"); - cm = (List) metrics.get("pulsar_in_messages_total"); + cm = (List) metrics.get("pulsar_in_messages_total"); assertEquals(cm.size(), 2); assertEquals(cm.get(0).tags.get("topic"), "persistent://my-property/use/my-ns/my-topic2"); assertEquals(cm.get(0).tags.get("namespace"), "my-property/use/my-ns"); assertEquals(cm.get(1).tags.get("topic"), "persistent://my-property/use/my-ns/my-topic1"); assertEquals(cm.get(1).tags.get("namespace"), "my-property/use/my-ns"); - cm = (List) metrics.get("pulsar_out_bytes_total"); + cm = (List) metrics.get("pulsar_out_bytes_total"); assertEquals(cm.size(), 2); assertEquals(cm.get(0).tags.get("topic"), "persistent://my-property/use/my-ns/my-topic2"); assertEquals(cm.get(0).tags.get("namespace"), "my-property/use/my-ns"); @@ -369,7 +367,7 @@ public void testPerTopicStats() throws Exception { assertEquals(cm.get(1).tags.get("namespace"), "my-property/use/my-ns"); assertEquals(cm.get(1).tags.get("subscription"), "test"); - cm = (List) metrics.get("pulsar_out_messages_total"); + cm = (List) metrics.get("pulsar_out_messages_total"); assertEquals(cm.size(), 2); assertEquals(cm.get(0).tags.get("topic"), "persistent://my-property/use/my-ns/my-topic2"); assertEquals(cm.get(0).tags.get("namespace"), "my-property/use/my-ns"); @@ -415,9 +413,9 @@ public void testPerBrokerStats() throws Exception { ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut); String metricsStr = statsOut.toString(); - Multimap metrics = parseMetrics(metricsStr); + Multimap metrics = PrometheusMetricsClient.parseMetrics(metricsStr); - Collection brokerMetrics = metrics.get("pulsar_broker_topics_count"); + Collection brokerMetrics = metrics.get("pulsar_broker_topics_count"); assertEquals(brokerMetrics.size(), 1); assertEquals(brokerMetrics.stream().toList().get(0).tags.get("cluster"), "test"); @@ -524,32 +522,32 @@ public void testPerTopicStatsReconnect() throws Exception { ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut); String metricsStr = statsOut.toString(); - Multimap metrics = parseMetrics(metricsStr); + Multimap metrics = PrometheusMetricsClient.parseMetrics(metricsStr); metrics.entries().forEach(e -> { System.out.println(e.getKey() + ": " + e.getValue()); }); - List cm = (List) metrics.get("pulsar_in_bytes_total"); + List cm = (List) metrics.get("pulsar_in_bytes_total"); assertEquals(cm.size(), 1); assertEquals(cm.get(0).value, (messageSizeBytes * messages * 2)); assertEquals(cm.get(0).tags.get("topic"), "persistent://my-property/use/my-ns/my-topic1"); assertEquals(cm.get(0).tags.get("namespace"), "my-property/use/my-ns"); - cm = (List) metrics.get("pulsar_in_messages_total"); + cm = (List) metrics.get("pulsar_in_messages_total"); assertEquals(cm.size(), 1); assertEquals(cm.get(0).value, (messages * 2)); assertEquals(cm.get(0).tags.get("topic"), "persistent://my-property/use/my-ns/my-topic1"); assertEquals(cm.get(0).tags.get("namespace"), "my-property/use/my-ns"); - cm = (List) metrics.get("pulsar_out_bytes_total"); + cm = (List) metrics.get("pulsar_out_bytes_total"); assertEquals(cm.size(), 1); assertEquals(cm.get(0).value, (messageSizeBytes * messages * 2)); assertEquals(cm.get(0).tags.get("topic"), "persistent://my-property/use/my-ns/my-topic1"); assertEquals(cm.get(0).tags.get("namespace"), "my-property/use/my-ns"); assertEquals(cm.get(0).tags.get("subscription"), "test"); - cm = (List) metrics.get("pulsar_out_messages_total"); + cm = (List) metrics.get("pulsar_out_messages_total"); assertEquals(cm.size(), 1); assertEquals(cm.get(0).value, (messages * 2)); assertEquals(cm.get(0).tags.get("topic"), "persistent://my-property/use/my-ns/my-topic1"); @@ -602,11 +600,11 @@ public void testStorageReadCacheMissesRate(boolean cacheEnable) throws Exception ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut); String metricsStr = statsOut.toString(); - Multimap metrics = parseMetrics(metricsStr); + Multimap metrics = PrometheusMetricsClient.parseMetrics(metricsStr); metrics.entries().forEach(e -> System.out.println(e.getKey() + ": " + e.getValue())); - List cm = (List) metrics.get("pulsar_storage_read_cache_misses_rate"); + List cm = (List) metrics.get("pulsar_storage_read_cache_misses_rate"); assertEquals(cm.size(), 1); if (cacheEnable) { assertEquals(cm.get(0).value, 1.0); @@ -618,7 +616,7 @@ public void testStorageReadCacheMissesRate(boolean cacheEnable) throws Exception assertEquals(cm.get(0).tags.get("namespace"), ns); assertEquals(cm.get(0).tags.get("cluster"), "test"); - List brokerMetric = (List) metrics.get("pulsar_broker_storage_read_cache_misses_rate"); + List brokerMetric = (List) metrics.get("pulsar_broker_storage_read_cache_misses_rate"); assertEquals(brokerMetric.size(), 1); if (cacheEnable) { assertEquals(brokerMetric.get(0).value, 1.0); @@ -634,11 +632,11 @@ public void testStorageReadCacheMissesRate(boolean cacheEnable) throws Exception ByteArrayOutputStream statsOut2 = new ByteArrayOutputStream(); PrometheusMetricsGenerator.generate(pulsar, false, false, false, statsOut2); String metricsStr2 = statsOut2.toString(); - Multimap metrics2 = parseMetrics(metricsStr2); + Multimap metrics2 = PrometheusMetricsClient.parseMetrics(metricsStr2); metrics2.entries().forEach(e -> System.out.println(e.getKey() + ": " + e.getValue())); - List cm2 = (List) metrics2.get("pulsar_storage_read_cache_misses_rate"); + List cm2 = (List) metrics2.get("pulsar_storage_read_cache_misses_rate"); assertEquals(cm2.size(), 1); if (cacheEnable) { assertEquals(cm2.get(0).value, 1.0); @@ -650,7 +648,7 @@ public void testStorageReadCacheMissesRate(boolean cacheEnable) throws Exception assertEquals(cm2.get(0).tags.get("namespace"), ns); assertEquals(cm2.get(0).tags.get("cluster"), "test"); - List brokerMetric2 = (List) metrics.get("pulsar_broker_storage_read_cache_misses_rate"); + List brokerMetric2 = (List) metrics.get("pulsar_broker_storage_read_cache_misses_rate"); assertEquals(brokerMetric2.size(), 1); if (cacheEnable) { assertEquals(brokerMetric2.get(0).value, 1.0); @@ -662,7 +660,7 @@ public void testStorageReadCacheMissesRate(boolean cacheEnable) throws Exception assertNull(brokerMetric2.get(0).tags.get("topic")); // test ManagedLedgerMetrics - List mlMetric = ((List) metrics.get("pulsar_ml_ReadEntriesOpsCacheMissesRate")); + List mlMetric = ((List) metrics.get("pulsar_ml_ReadEntriesOpsCacheMissesRate")); assertEquals(mlMetric.size(), 1); if (cacheEnable) { assertEquals(mlMetric.get(0).value, 1.0); @@ -718,9 +716,9 @@ public void testPerTopicExpiredStat() throws Exception { ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut); String metricsStr = statsOut.toString(); - Multimap metrics = parseMetrics(metricsStr); + Multimap metrics = PrometheusMetricsClient.parseMetrics(metricsStr); // There should be 2 metrics with different tags for each topic - List cm = (List) metrics.get("pulsar_subscription_last_expire_timestamp"); + List cm = (List) metrics.get("pulsar_subscription_last_expire_timestamp"); assertEquals(cm.size(), 2); assertEquals(cm.get(0).tags.get("topic"), topic2); assertEquals(cm.get(0).tags.get("namespace"), ns); @@ -736,7 +734,7 @@ public void testPerTopicExpiredStat() throws Exception { assertEquals((long) field.get(subscription), (long) cm.get(i).value); } - cm = (List) metrics.get("pulsar_subscription_msg_rate_expired"); + cm = (List) metrics.get("pulsar_subscription_msg_rate_expired"); assertEquals(cm.size(), 2); assertEquals(cm.get(0).tags.get("topic"), topic2); assertEquals(cm.get(0).tags.get("namespace"), ns); @@ -755,7 +753,7 @@ public void testPerTopicExpiredStat() throws Exception { assertEquals(Double.valueOf(nf.format(monitor.getMessageExpiryRate())).doubleValue(), cm.get(i).value); } - cm = (List) metrics.get("pulsar_subscription_total_msg_expired"); + cm = (List) metrics.get("pulsar_subscription_total_msg_expired"); assertEquals(cm.size(), 2); assertEquals(cm.get(0).tags.get("topic"), topic2); assertEquals(cm.get(0).tags.get("namespace"), ns); @@ -799,7 +797,7 @@ public void testBundlesMetrics() throws Exception { ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); PrometheusMetricsGenerator.generate(pulsar, false, false, false, statsOut); String metricsStr = statsOut.toString(); - Multimap metrics = parseMetrics(metricsStr); + Multimap metrics = PrometheusMetricsClient.parseMetrics(metricsStr); assertTrue(metrics.containsKey("pulsar_bundle_msg_rate_in")); assertTrue(metrics.containsKey("pulsar_bundle_msg_rate_out")); assertTrue(metrics.containsKey("pulsar_bundle_topics_count")); @@ -844,7 +842,7 @@ public void testNonPersistentSubMetrics() throws Exception { ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut); String metricsStr = statsOut.toString(); - Multimap metrics = parseMetrics(metricsStr); + Multimap metrics = PrometheusMetricsClient.parseMetrics(metricsStr); assertTrue(metrics.containsKey("pulsar_subscription_back_log")); assertTrue(metrics.containsKey("pulsar_subscription_back_log_no_delayed")); assertTrue(metrics.containsKey("pulsar_subscription_msg_throughput_out")); @@ -892,36 +890,36 @@ public void testPerNamespaceStats() throws Exception { PrometheusMetricsGenerator.generate(pulsar, false, false, false, statsOut); String metricsStr = statsOut.toString(); - Multimap metrics = parseMetrics(metricsStr); + Multimap metrics = PrometheusMetricsClient.parseMetrics(metricsStr); metrics.entries().forEach(e -> { System.out.println(e.getKey() + ": " + e.getValue()); }); // There should be 1 metric aggregated per namespace - List cm = (List) metrics.get("pulsar_storage_write_latency_le_1"); + List cm = (List) metrics.get("pulsar_storage_write_latency_le_1"); assertEquals(cm.size(), 1); assertNull(cm.get(0).tags.get("topic")); assertEquals(cm.get(0).tags.get("namespace"), "my-property/use/my-ns"); - cm = (List) metrics.get("pulsar_producers_count"); + cm = (List) metrics.get("pulsar_producers_count"); assertEquals(cm.size(), 1); assertNull(cm.get(0).tags.get("topic")); assertEquals(cm.get(0).tags.get("namespace"), "my-property/use/my-ns"); - cm = (List) metrics.get("pulsar_in_bytes_total"); + cm = (List) metrics.get("pulsar_in_bytes_total"); assertEquals(cm.size(), 1); assertEquals(cm.get(0).tags.get("namespace"), "my-property/use/my-ns"); - cm = (List) metrics.get("pulsar_in_messages_total"); + cm = (List) metrics.get("pulsar_in_messages_total"); assertEquals(cm.size(), 1); assertEquals(cm.get(0).tags.get("namespace"), "my-property/use/my-ns"); - cm = (List) metrics.get("pulsar_out_bytes_total"); + cm = (List) metrics.get("pulsar_out_bytes_total"); assertEquals(cm.size(), 1); assertEquals(cm.get(0).tags.get("namespace"), "my-property/use/my-ns"); - cm = (List) metrics.get("pulsar_out_messages_total"); + cm = (List) metrics.get("pulsar_out_messages_total"); assertEquals(cm.size(), 1); assertEquals(cm.get(0).tags.get("namespace"), "my-property/use/my-ns"); @@ -965,13 +963,13 @@ public void testPerProducerStats() throws Exception { PrometheusMetricsGenerator.generate(pulsar, true, false, true, statsOut); String metricsStr = statsOut.toString(); - Multimap metrics = parseMetrics(metricsStr); + Multimap metrics = PrometheusMetricsClient.parseMetrics(metricsStr); metrics.entries().forEach(e -> { System.out.println(e.getKey() + ": " + e.getValue()); }); - List cm = (List) metrics.get("pulsar_producer_msg_rate_in"); + List cm = (List) metrics.get("pulsar_producer_msg_rate_in"); assertEquals(cm.size(), 2); assertEquals(cm.get(0).tags.get("namespace"), "my-property/use/my-ns"); assertEquals(cm.get(0).tags.get("topic"), "persistent://my-property/use/my-ns/my-topic2"); @@ -983,7 +981,7 @@ public void testPerProducerStats() throws Exception { assertEquals(cm.get(1).tags.get("producer_name"), "producer1"); assertEquals(cm.get(1).tags.get("producer_id"), "0"); - cm = (List) metrics.get("pulsar_producer_msg_throughput_in"); + cm = (List) metrics.get("pulsar_producer_msg_throughput_in"); assertEquals(cm.size(), 2); assertEquals(cm.get(0).tags.get("namespace"), "my-property/use/my-ns"); assertEquals(cm.get(0).tags.get("topic"), "persistent://my-property/use/my-ns/my-topic2"); @@ -1033,14 +1031,14 @@ public void testPerConsumerStats() throws Exception { PrometheusMetricsGenerator.generate(pulsar, true, true, false, statsOut); String metricsStr = statsOut.toString(); - Multimap metrics = parseMetrics(metricsStr); + Multimap metrics = PrometheusMetricsClient.parseMetrics(metricsStr); metrics.entries().forEach(e -> { System.out.println(e.getKey() + ": " + e.getValue()); }); // There should be 1 metric aggregated per namespace - List cm = (List) metrics.get("pulsar_out_bytes_total"); + List cm = (List) metrics.get("pulsar_out_bytes_total"); assertEquals(cm.size(), 4); assertEquals(cm.get(0).tags.get("namespace"), "my-property/use/my-ns"); assertEquals(cm.get(0).tags.get("topic"), "persistent://my-property/use/my-ns/my-topic2"); @@ -1060,7 +1058,7 @@ public void testPerConsumerStats() throws Exception { assertEquals(cm.get(3).tags.get("subscription"), "test"); assertEquals(cm.get(3).tags.get("consumer_id"), "0"); - cm = (List) metrics.get("pulsar_out_messages_total"); + cm = (List) metrics.get("pulsar_out_messages_total"); assertEquals(cm.size(), 4); assertEquals(cm.get(0).tags.get("namespace"), "my-property/use/my-ns"); assertEquals(cm.get(0).tags.get("topic"), "persistent://my-property/use/my-ns/my-topic2"); @@ -1224,17 +1222,17 @@ public void testManagedLedgerCacheStats() throws Exception { PrometheusMetricsGenerator.generate(pulsar, false, false, false, statsOut); String metricsStr = statsOut.toString(); - Multimap metrics = parseMetrics(metricsStr); + Multimap metrics = PrometheusMetricsClient.parseMetrics(metricsStr); metrics.entries().forEach(e -> System.out.println(e.getKey() + ": " + e.getValue()) ); - List cm = (List) metrics.get("pulsar_ml_cache_evictions"); + List cm = (List) metrics.get("pulsar_ml_cache_evictions"); assertEquals(cm.size(), 1); assertEquals(cm.get(0).tags.get("cluster"), "test"); - cm = (List) metrics.get("pulsar_ml_cache_hits_rate"); + cm = (List) metrics.get("pulsar_ml_cache_hits_rate"); assertEquals(cm.size(), 1); assertEquals(cm.get(0).tags.get("cluster"), "test"); @@ -1260,7 +1258,7 @@ public void testManagedLedgerStats() throws Exception { PrometheusMetricsGenerator.generate(pulsar, false, false, false, statsOut); String metricsStr = statsOut.toString(); - Multimap metrics = parseMetrics(metricsStr); + Multimap metrics = PrometheusMetricsClient.parseMetrics(metricsStr); metrics.entries().forEach(e -> System.out.println(e.getKey() + ": " + e.getValue()) @@ -1303,13 +1301,13 @@ public void testManagedLedgerStats() throws Exception { } }); - List cm = (List) metrics.get("pulsar_ml_AddEntryBytesRate"); + List cm = (List) metrics.get("pulsar_ml_AddEntryBytesRate"); assertEquals(cm.size(), 2); assertEquals(cm.get(0).tags.get("cluster"), "test"); String ns = cm.get(0).tags.get("namespace"); assertTrue(ns.equals("my-property/use/my-ns") || ns.equals("my-property/use/my-ns2")); - cm = (List) metrics.get("pulsar_ml_AddEntryMessagesRate"); + cm = (List) metrics.get("pulsar_ml_AddEntryMessagesRate"); assertEquals(cm.size(), 2); assertEquals(cm.get(0).tags.get("cluster"), "test"); ns = cm.get(0).tags.get("namespace"); @@ -1338,32 +1336,32 @@ public void testManagedLedgerBookieClientStats() throws Exception { PrometheusMetricsGenerator.generate(pulsar, false, false, false, statsOut); String metricsStr = statsOut.toString(); - Multimap metrics = parseMetrics(metricsStr); + Multimap metrics = PrometheusMetricsClient.parseMetrics(metricsStr); metrics.entries().forEach(e -> System.out.println(e.getKey() + ": " + e.getValue()) ); - List cm = (List) metrics.get( + List cm = (List) metrics.get( keyNameBySubstrings(metrics, "pulsar_managedLedger_client", "bookkeeper_ml_scheduler_threads")); assertEquals(cm.size(), 1); assertEquals(cm.get(0).tags.get("cluster"), "test"); - cm = (List) metrics.get( + cm = (List) metrics.get( keyNameBySubstrings(metrics, "pulsar_managedLedger_client", "bookkeeper_ml_scheduler_task_execution_sum")); assertEquals(cm.size(), 2); assertEquals(cm.get(0).tags.get("cluster"), "test"); - cm = (List) metrics.get( + cm = (List) metrics.get( keyNameBySubstrings(metrics, "pulsar_managedLedger_client", "bookkeeper_ml_scheduler_max_queue_size")); assertEquals(cm.size(), 1); assertEquals(cm.get(0).tags.get("cluster"), "test"); } - private static String keyNameBySubstrings(Multimap metrics, String... substrings) { + private static String keyNameBySubstrings(Multimap metrics, String... substrings) { for (String key: metrics.keys()) { boolean found = true; for (String s: substrings) { @@ -1418,10 +1416,10 @@ public String getCommandData() { ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); PrometheusMetricsGenerator.generate(pulsar, false, false, false, statsOut); String metricsStr = statsOut.toString(); - Multimap metrics = parseMetrics(metricsStr); - List cm = (List) metrics.get("pulsar_authentication_success_total"); + Multimap metrics = PrometheusMetricsClient.parseMetrics(metricsStr); + List cm = (List) metrics.get("pulsar_authentication_success_total"); boolean haveSucceed = false; - for (Metric metric : cm) { + for (PrometheusMetricsClient.Metric metric : cm) { if (Objects.equals(metric.tags.get("auth_method"), "token") && Objects.equals(metric.tags.get("provider_name"), provider.getClass().getSimpleName())) { haveSucceed = true; @@ -1429,10 +1427,10 @@ public String getCommandData() { } Assert.assertTrue(haveSucceed); - cm = (List) metrics.get("pulsar_authentication_failures_total"); + cm = (List) metrics.get("pulsar_authentication_failures_total"); boolean haveFailed = false; - for (Metric metric : cm) { + for (PrometheusMetricsClient.Metric metric : cm) { if (Objects.equals(metric.tags.get("auth_method"), "token") && Objects.equals(metric.tags.get("reason"), AuthenticationProviderToken.ErrorCode.INVALID_AUTH_DATA.name()) @@ -1479,8 +1477,8 @@ public String getCommandData() { ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); PrometheusMetricsGenerator.generate(pulsar, false, false, false, statsOut); String metricsStr = statsOut.toString(); - Multimap metrics = parseMetrics(metricsStr); - List cm = (List) metrics.get("pulsar_expired_token_total"); + Multimap metrics = PrometheusMetricsClient.parseMetrics(metricsStr); + List cm = (List) metrics.get("pulsar_expired_token_total"); assertEquals(cm.size(), 1); provider.close(); @@ -1520,10 +1518,10 @@ public String getCommandData() { ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); PrometheusMetricsGenerator.generate(pulsar, false, false, false, statsOut); String metricsStr = statsOut.toString(); - Multimap metrics = parseMetrics(metricsStr); - Metric countMetric = ((List) metrics.get("pulsar_expiring_token_minutes_count")).get(0); + Multimap metrics = PrometheusMetricsClient.parseMetrics(metricsStr); + PrometheusMetricsClient.Metric countMetric = ((List) metrics.get("pulsar_expiring_token_minutes_count")).get(0); assertEquals(countMetric.value, tokenRemainTime.length); - List cm = (List) metrics.get("pulsar_expiring_token_minutes_bucket"); + List cm = (List) metrics.get("pulsar_expiring_token_minutes_bucket"); assertEquals(cm.size(), 5); cm.forEach((e) -> { switch (e.tags.get("le")) { @@ -1549,8 +1547,8 @@ public String getCommandData() { @Test public void testParsingWithPositiveInfinityValue() { - Multimap metrics = parseMetrics("pulsar_broker_publish_latency{cluster=\"test\",quantile=\"0.0\"} +Inf"); - List cm = (List) metrics.get("pulsar_broker_publish_latency"); + Multimap metrics = PrometheusMetricsClient.parseMetrics("pulsar_broker_publish_latency{cluster=\"test\",quantile=\"0.0\"} +Inf"); + List cm = (List) metrics.get("pulsar_broker_publish_latency"); assertEquals(cm.size(), 1); assertEquals(cm.get(0).tags.get("cluster"), "test"); assertEquals(cm.get(0).tags.get("quantile"), "0.0"); @@ -1559,8 +1557,8 @@ public void testParsingWithPositiveInfinityValue() { @Test public void testParsingWithNegativeInfinityValue() { - Multimap metrics = parseMetrics("pulsar_broker_publish_latency{cluster=\"test\",quantile=\"0.0\"} -Inf"); - List cm = (List) metrics.get("pulsar_broker_publish_latency"); + Multimap metrics = PrometheusMetricsClient.parseMetrics("pulsar_broker_publish_latency{cluster=\"test\",quantile=\"0.0\"} -Inf"); + List cm = (List) metrics.get("pulsar_broker_publish_latency"); assertEquals(cm.size(), 1); assertEquals(cm.get(0).tags.get("cluster"), "test"); assertEquals(cm.get(0).tags.get("quantile"), "0.0"); @@ -1595,9 +1593,9 @@ public void testManagedCursorPersistStats() throws Exception { PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut); String metricsStr = statsOut.toString(); - Multimap metrics = parseMetrics(metricsStr); + Multimap metrics = PrometheusMetricsClient.parseMetrics(metricsStr); - List cm = (List) metrics.get("pulsar_ml_cursor_persistLedgerSucceed"); + List cm = (List) metrics.get("pulsar_ml_cursor_persistLedgerSucceed"); assertEquals(cm.size(), 1); assertEquals(cm.get(0).tags.get("cluster"), "test"); assertEquals(cm.get(0).tags.get("cursor_name"), subName); @@ -1607,8 +1605,8 @@ public void testManagedCursorPersistStats() throws Exception { ByteArrayOutputStream statsOut2 = new ByteArrayOutputStream(); PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut2); String metricsStr2 = statsOut2.toString(); - Multimap metrics2 = parseMetrics(metricsStr2); - List cm2 = (List) metrics2.get("pulsar_ml_cursor_persistLedgerSucceed"); + Multimap metrics2 = PrometheusMetricsClient.parseMetrics(metricsStr2); + List cm2 = (List) metrics2.get("pulsar_ml_cursor_persistLedgerSucceed"); assertEquals(cm2.size(), 0); producer.close(); @@ -1626,17 +1624,17 @@ public void testBrokerConnection() throws Exception { ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut); String metricsStr = statsOut.toString(); - Multimap metrics = parseMetrics(metricsStr); - List cm = (List) metrics.get("pulsar_connection_created_total_count"); + Multimap metrics = PrometheusMetricsClient.parseMetrics(metricsStr); + List cm = (List) metrics.get("pulsar_connection_created_total_count"); compareBrokerConnectionStateCount(cm, 1.0); - cm = (List) metrics.get("pulsar_connection_create_success_count"); + cm = (List) metrics.get("pulsar_connection_create_success_count"); compareBrokerConnectionStateCount(cm, 1.0); - cm = (List) metrics.get("pulsar_connection_closed_total_count"); + cm = (List) metrics.get("pulsar_connection_closed_total_count"); compareBrokerConnectionStateCount(cm, 0.0); - cm = (List) metrics.get("pulsar_active_connections"); + cm = (List) metrics.get("pulsar_active_connections"); compareBrokerConnectionStateCount(cm, 1.0); pulsarClient.close(); @@ -1644,8 +1642,8 @@ public void testBrokerConnection() throws Exception { PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut); metricsStr = statsOut.toString(); - metrics = parseMetrics(metricsStr); - cm = (List) metrics.get("pulsar_connection_closed_total_count"); + metrics = PrometheusMetricsClient.parseMetrics(metricsStr); + cm = (List) metrics.get("pulsar_connection_closed_total_count"); compareBrokerConnectionStateCount(cm, 1.0); pulsar.getConfiguration().setAuthenticationEnabled(true); @@ -1667,24 +1665,24 @@ public void testBrokerConnection() throws Exception { PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut); metricsStr = statsOut.toString(); - metrics = parseMetrics(metricsStr); - cm = (List) metrics.get("pulsar_connection_closed_total_count"); + metrics = PrometheusMetricsClient.parseMetrics(metricsStr); + cm = (List) metrics.get("pulsar_connection_closed_total_count"); compareBrokerConnectionStateCount(cm, 2.0); - cm = (List) metrics.get("pulsar_connection_create_fail_count"); + cm = (List) metrics.get("pulsar_connection_create_fail_count"); compareBrokerConnectionStateCount(cm, 1.0); - cm = (List) metrics.get("pulsar_connection_create_success_count"); + cm = (List) metrics.get("pulsar_connection_create_success_count"); compareBrokerConnectionStateCount(cm, 1.0); - cm = (List) metrics.get("pulsar_active_connections"); + cm = (List) metrics.get("pulsar_active_connections"); compareBrokerConnectionStateCount(cm, 0.0); - cm = (List) metrics.get("pulsar_connection_created_total_count"); + cm = (List) metrics.get("pulsar_connection_created_total_count"); compareBrokerConnectionStateCount(cm, 2.0); } - private void compareBrokerConnectionStateCount(List cm, double count) { + private void compareBrokerConnectionStateCount(List cm, double count) { assertEquals(cm.size(), 1); assertEquals(cm.get(0).tags.get("cluster"), "test"); assertEquals(cm.get(0).tags.get("broker"), "localhost"); @@ -1695,7 +1693,7 @@ private void compareBrokerConnectionStateCount(List cm, double count) { void testParseMetrics() throws IOException { String sampleMetrics = IOUtils.toString(getClass().getClassLoader() .getResourceAsStream("prometheus_metrics_sample.txt"), StandardCharsets.UTF_8); - parseMetrics(sampleMetrics); + PrometheusMetricsClient.parseMetrics(sampleMetrics); } @Test @@ -1710,22 +1708,22 @@ public void testCompaction() throws Exception { ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut); String metricsStr = statsOut.toString(); - Multimap metrics = parseMetrics(metricsStr); - List cm = (List) metrics.get("pulsar_compaction_removed_event_count"); + Multimap metrics = PrometheusMetricsClient.parseMetrics(metricsStr); + List cm = (List) metrics.get("pulsar_compaction_removed_event_count"); assertEquals(cm.size(), 0); - cm = (List) metrics.get("pulsar_compaction_succeed_count"); + cm = (List) metrics.get("pulsar_compaction_succeed_count"); assertEquals(cm.size(), 0); - cm = (List) metrics.get("pulsar_compaction_failed_count"); + cm = (List) metrics.get("pulsar_compaction_failed_count"); assertEquals(cm.size(), 0); - cm = (List) metrics.get("pulsar_compaction_duration_time_in_mills"); + cm = (List) metrics.get("pulsar_compaction_duration_time_in_mills"); assertEquals(cm.size(), 0); - cm = (List) metrics.get("pulsar_compaction_read_throughput"); + cm = (List) metrics.get("pulsar_compaction_read_throughput"); assertEquals(cm.size(), 0); - cm = (List) metrics.get("pulsar_compaction_write_throughput"); + cm = (List) metrics.get("pulsar_compaction_write_throughput"); assertEquals(cm.size(), 0); - cm = (List) metrics.get("pulsar_compaction_compacted_entries_count"); + cm = (List) metrics.get("pulsar_compaction_compacted_entries_count"); assertEquals(cm.size(), 0); - cm = (List) metrics.get("pulsar_compaction_compacted_entries_size"); + cm = (List) metrics.get("pulsar_compaction_compacted_entries_size"); assertEquals(cm.size(), 0); // final int numMessages = 1000; @@ -1745,29 +1743,29 @@ public void testCompaction() throws Exception { statsOut = new ByteArrayOutputStream(); PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut); metricsStr = statsOut.toString(); - metrics = parseMetrics(metricsStr); - cm = (List) metrics.get("pulsar_compaction_removed_event_count"); + metrics = PrometheusMetricsClient.parseMetrics(metricsStr); + cm = (List) metrics.get("pulsar_compaction_removed_event_count"); assertEquals(cm.size(), 1); assertEquals(cm.get(0).value, 990); - cm = (List) metrics.get("pulsar_compaction_succeed_count"); + cm = (List) metrics.get("pulsar_compaction_succeed_count"); assertEquals(cm.size(), 1); assertEquals(cm.get(0).value, 1); - cm = (List) metrics.get("pulsar_compaction_failed_count"); + cm = (List) metrics.get("pulsar_compaction_failed_count"); assertEquals(cm.size(), 1); assertEquals(cm.get(0).value, 0); - cm = (List) metrics.get("pulsar_compaction_duration_time_in_mills"); + cm = (List) metrics.get("pulsar_compaction_duration_time_in_mills"); assertEquals(cm.size(), 1); assertTrue(cm.get(0).value > 0); - cm = (List) metrics.get("pulsar_compaction_read_throughput"); + cm = (List) metrics.get("pulsar_compaction_read_throughput"); assertEquals(cm.size(), 1); assertTrue(cm.get(0).value > 0); - cm = (List) metrics.get("pulsar_compaction_write_throughput"); + cm = (List) metrics.get("pulsar_compaction_write_throughput"); assertEquals(cm.size(), 1); assertTrue(cm.get(0).value > 0); - cm = (List) metrics.get("pulsar_compaction_compacted_entries_count"); + cm = (List) metrics.get("pulsar_compaction_compacted_entries_count"); assertEquals(cm.size(), 1); assertEquals(cm.get(0).value, 10); - cm = (List) metrics.get("pulsar_compaction_compacted_entries_size"); + cm = (List) metrics.get("pulsar_compaction_compacted_entries_size"); assertEquals(cm.size(), 1); assertEquals(cm.get(0).value, 840); @@ -1797,7 +1795,7 @@ public void testMetricsWithCache() throws Throwable { String metricsStr1 = statsOut1.toString(); String metricsStr2 = statsOut2.toString(); assertEquals(metricsStr1, metricsStr2); - Multimap metrics = parseMetrics(metricsStr1); + Multimap metrics = PrometheusMetricsClient.parseMetrics(metricsStr1); } Thread.sleep(TimeUnit.SECONDS.toMillis(period / 2)); @@ -1830,8 +1828,8 @@ public void testSplitTopicAndPartitionLabel() throws Exception { ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); PrometheusMetricsGenerator.generate(pulsar, true, false, false, true, statsOut); String metricsStr = statsOut.toString(); - Multimap metrics = parseMetrics(metricsStr); - Collection metric = metrics.get("pulsar_consumers_count"); + Multimap metrics = PrometheusMetricsClient.parseMetrics(metricsStr); + Collection metric = metrics.get("pulsar_consumers_count"); assertTrue(metric.size() >= 15); metric.forEach(item -> { if (ns1.equals(item.tags.get("namespace"))) { @@ -1846,7 +1844,7 @@ public void testSplitTopicAndPartitionLabel() throws Exception { consumer2.close(); } - private void compareCompactionStateCount(List cm, double count) { + private void compareCompactionStateCount(List cm, double count) { assertEquals(cm.size(), 1); assertEquals(cm.get(0).tags.get("cluster"), "test"); assertEquals(cm.get(0).tags.get("broker"), "localhost"); @@ -1910,62 +1908,6 @@ public void testMetricsGroupedByTypeDefinitions() throws Exception { p2.close(); } - /** - * Hacky parsing of Prometheus text format. Should be good enough for unit tests - */ - public static Multimap parseMetrics(String metrics) { - Multimap parsed = ArrayListMultimap.create(); - - // Example of lines are - // jvm_threads_current{cluster="standalone",} 203.0 - // or - // pulsar_subscriptions_count{cluster="standalone", namespace="public/default", - // topic="persistent://public/default/test-2"} 0.0 - Pattern pattern = Pattern.compile("^(\\w+)\\{([^\\}]+)\\}\\s([+-]?[\\d\\w\\.-]+)$"); - Pattern tagsPattern = Pattern.compile("(\\w+)=\"([^\"]+)\"(,\\s?)?"); - - Splitter.on("\n").split(metrics).forEach(line -> { - if (line.isEmpty() || line.startsWith("#")) { - return; - } - - Matcher matcher = pattern.matcher(line); - assertTrue(matcher.matches(), "line " + line + " does not match pattern " + pattern); - String name = matcher.group(1); - - Metric m = new Metric(); - String numericValue = matcher.group(3); - if (numericValue.equalsIgnoreCase("-Inf")) { - m.value = Double.NEGATIVE_INFINITY; - } else if (numericValue.equalsIgnoreCase("+Inf")) { - m.value = Double.POSITIVE_INFINITY; - } else { - m.value = Double.parseDouble(numericValue); - } - String tags = matcher.group(2); - Matcher tagsMatcher = tagsPattern.matcher(tags); - while (tagsMatcher.find()) { - String tag = tagsMatcher.group(1); - String value = tagsMatcher.group(2); - m.tags.put(tag, value); - } - - parsed.put(name, m); - }); - - return parsed; - } - - public static class Metric { - public Map tags = new TreeMap<>(); - public double value; - - @Override - public String toString() { - return MoreObjects.toStringHelper(this).add("tags", tags).add("value", value).toString(); - } - } - @Test public void testEscapeLabelValue() throws Exception { String ns1 = "prop/ns-abc1"; diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/SubscriptionStatsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/SubscriptionStatsTest.java index d5e0066a86f15..8c91d75978475 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/SubscriptionStatsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/SubscriptionStatsTest.java @@ -35,6 +35,7 @@ import org.apache.pulsar.broker.service.plugin.EntryFilter; import org.apache.pulsar.broker.service.plugin.EntryFilterTest; import org.apache.pulsar.broker.service.plugin.EntryFilterWithClassLoader; +import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient; import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsGenerator; import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.api.Consumer; @@ -233,15 +234,15 @@ public void testSubscriptionStats(final String topic, final String subName, bool ByteArrayOutputStream output = new ByteArrayOutputStream(); PrometheusMetricsGenerator.generate(pulsar, enableTopicStats, false, false, output); String metricsStr = output.toString(); - Multimap metrics = PrometheusMetricsTest.parseMetrics(metricsStr); + Multimap metrics = PrometheusMetricsClient.parseMetrics(metricsStr); - Collection throughFilterMetrics = + Collection throughFilterMetrics = metrics.get("pulsar_subscription_filter_processed_msg_count"); - Collection acceptedMetrics = + Collection acceptedMetrics = metrics.get("pulsar_subscription_filter_accepted_msg_count"); - Collection rejectedMetrics = + Collection rejectedMetrics = metrics.get("pulsar_subscription_filter_rejected_msg_count"); - Collection rescheduledMetrics = + Collection rescheduledMetrics = metrics.get("pulsar_subscription_filter_rescheduled_msg_count"); if (enableTopicStats) { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/TransactionMetricsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/TransactionMetricsTest.java index 4d38f5fad5141..32cbcaed8943b 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/TransactionMetricsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/TransactionMetricsTest.java @@ -19,7 +19,7 @@ package org.apache.pulsar.broker.stats; import static com.google.common.base.Preconditions.checkArgument; -import static org.apache.pulsar.broker.stats.PrometheusMetricsTest.parseMetrics; +import static org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient.parseMetrics; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertTrue; import static org.testng.Assert.fail; @@ -39,6 +39,7 @@ import lombok.extern.slf4j.Slf4j; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.service.BrokerTestBase; +import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient; import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsGenerator; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.MessageId; @@ -119,8 +120,8 @@ public void testTransactionCoordinatorMetrics() throws Exception { ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut); String metricsStr = statsOut.toString(); - Multimap metrics = parseMetrics(metricsStr); - Collection metric = metrics.get("pulsar_txn_active_count"); + Multimap metrics = parseMetrics(metricsStr); + Collection metric = metrics.get("pulsar_txn_active_count"); assertEquals(metric.size(), 2); metric.forEach(item -> { if ("0".equals(item.tags.get("coordinator_id"))) { @@ -187,9 +188,9 @@ public void testTransactionCoordinatorRateMetrics() throws Exception { ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut); String metricsStr = statsOut.toString(); - Multimap metrics = parseMetrics(metricsStr); + Multimap metrics = parseMetrics(metricsStr); - Collection metric = metrics.get("pulsar_txn_created_total"); + Collection metric = metrics.get("pulsar_txn_created_total"); assertEquals(metric.size(), 1); metric.forEach(item -> assertEquals(item.value, txnCount)); @@ -274,9 +275,9 @@ public void testManagedLedgerMetrics() throws Exception { PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut); String metricsStr = statsOut.toString(); - Multimap metrics = parseMetrics(metricsStr); + Multimap metrics = parseMetrics(metricsStr); - Collection metric = metrics.get("pulsar_storage_size"); + Collection metric = metrics.get("pulsar_storage_size"); checkManagedLedgerMetrics(subName, 32, metric); checkManagedLedgerMetrics(MLTransactionLogImpl.TRANSACTION_SUBSCRIPTION_NAME, 252, metric); @@ -336,12 +337,12 @@ public void testManagedLedgerMetricsWhenPendingAckNotInit() throws Exception { PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut); String metricsStr = statsOut.toString(); - Multimap metrics = parseMetrics(metricsStr); + Multimap metrics = parseMetrics(metricsStr); - Collection metric = metrics.get("pulsar_storage_size"); + Collection metric = metrics.get("pulsar_storage_size"); checkManagedLedgerMetrics(subName, 32, metric); //No statistics of the pendingAck are generated when the pendingAck is not initialized. - for (PrometheusMetricsTest.Metric metric1 : metric) { + for (PrometheusMetricsClient.Metric metric1 : metric) { if (metric1.tags.containsValue(subName2)) { Assert.fail(); } @@ -431,9 +432,9 @@ public void testDuplicateMetricTypeDefinitions() throws Exception { } - private void checkManagedLedgerMetrics(String tag, double value, Collection metrics) { + private void checkManagedLedgerMetrics(String tag, double value, Collection metrics) { boolean exist = false; - for (PrometheusMetricsTest.Metric metric1 : metrics) { + for (PrometheusMetricsClient.Metric metric1 : metrics) { if (metric1.tags.containsValue(tag)) { assertEquals(metric1.value, value); exist = true; diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsClient.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsClient.java new file mode 100644 index 0000000000000..474d0d90bc3f7 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsClient.java @@ -0,0 +1,105 @@ +package org.apache.pulsar.broker.stats.prometheus; + +import static org.testng.Assert.assertTrue; +import com.google.common.base.MoreObjects; +import com.google.common.base.Splitter; +import com.google.common.collect.ArrayListMultimap; +import com.google.common.collect.Multimap; +import io.restassured.RestAssured; +import java.util.List; +import java.util.Map; +import java.util.TreeMap; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +public class PrometheusMetricsClient { + private final String host; + private final int port; + + public PrometheusMetricsClient(String host, int port) { + this.host = host; + this.port = port; + } + + @SuppressWarnings("HttpUrlsUsage") + public Metrics getMetrics() { + String metrics = RestAssured.given().baseUri("http://" + host).port(port).get("/metrics").asString(); + return new Metrics(parseMetrics(metrics)); + } + + /** + * Hacky parsing of Prometheus text format. Should be good enough for unit tests + */ + public static Multimap parseMetrics(String metrics) { + Multimap parsed = ArrayListMultimap.create(); + + // Example of lines are + // jvm_threads_current{cluster="standalone",} 203.0 + // or + // pulsar_subscriptions_count{cluster="standalone", namespace="public/default", + // topic="persistent://public/default/test-2"} 0.0 + Pattern pattern = Pattern.compile("^(\\w+)\\{([^\\}]+)\\}\\s([+-]?[\\d\\w\\.-]+)$"); + Pattern tagsPattern = Pattern.compile("(\\w+)=\"([^\"]+)\"(,\\s?)?"); + + Splitter.on("\n").split(metrics).forEach(line -> { + if (line.isEmpty() || line.startsWith("#")) { + return; + } + + Matcher matcher = pattern.matcher(line); + assertTrue(matcher.matches(), "line " + line + " does not match pattern " + pattern); + String name = matcher.group(1); + + Metric m = new Metric(); + String numericValue = matcher.group(3); + if (numericValue.equalsIgnoreCase("-Inf")) { + m.value = Double.NEGATIVE_INFINITY; + } else if (numericValue.equalsIgnoreCase("+Inf")) { + m.value = Double.POSITIVE_INFINITY; + } else { + m.value = Double.parseDouble(numericValue); + } + String tags = matcher.group(2); + Matcher tagsMatcher = tagsPattern.matcher(tags); + while (tagsMatcher.find()) { + String tag = tagsMatcher.group(1); + String value = tagsMatcher.group(2); + m.tags.put(tag, value); + } + + parsed.put(name, m); + }); + + return parsed; + } + + public static class Metric { + public Map tags = new TreeMap<>(); + public double value; + + @Override + public String toString() { + return MoreObjects.toStringHelper(this).add("tags", tags).add("value", value).toString(); + } + + public boolean contains(String labelName, String labelValue) { + String value = tags.get(labelName); + return value != null && value.equals(labelValue); + } + } + + public static class Metrics { + final Multimap nameToDataPoints; + + public Metrics(Multimap nameToDataPoints) { + this.nameToDataPoints = nameToDataPoints; + } + + public List findByNameAndLabels(String metricName, String labelName, String labelValue) { + return nameToDataPoints.get(metricName) + .stream() + .filter(metric -> metric.contains(labelName, labelValue)) + .toList(); + } + } +} diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/TransactionBufferClientTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/TransactionBufferClientTest.java index 1684b2ca138e8..f40efa1228064 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/TransactionBufferClientTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/TransactionBufferClientTest.java @@ -38,7 +38,7 @@ import org.apache.pulsar.broker.PulsarServerException; import org.apache.pulsar.broker.PulsarService; -import org.apache.pulsar.broker.stats.PrometheusMetricsTest; +import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient; import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsGenerator; import org.apache.pulsar.broker.transaction.TransactionTestBase; import org.apache.pulsar.broker.transaction.buffer.impl.TransactionBufferClientImpl; @@ -228,28 +228,28 @@ public void testTransactionBufferMetrics() throws Exception { ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); PrometheusMetricsGenerator.generate(pulsarServiceList.get(0), true, false, false, statsOut); String metricsStr = statsOut.toString(); - Multimap metricsMap = PrometheusMetricsTest.parseMetrics(metricsStr); + Multimap metricsMap = PrometheusMetricsClient.parseMetrics(metricsStr); - Collection abortFailed = metricsMap.get("pulsar_txn_tb_client_abort_failed_total"); - Collection commitFailed = metricsMap.get("pulsar_txn_tb_client_commit_failed_total"); - Collection abortLatencyCount = + Collection abortFailed = metricsMap.get("pulsar_txn_tb_client_abort_failed_total"); + Collection commitFailed = metricsMap.get("pulsar_txn_tb_client_commit_failed_total"); + Collection abortLatencyCount = metricsMap.get("pulsar_txn_tb_client_abort_latency_count"); - Collection commitLatencyCount = + Collection commitLatencyCount = metricsMap.get("pulsar_txn_tb_client_commit_latency_count"); - Collection pending = metricsMap.get("pulsar_txn_tb_client_pending_requests"); + Collection pending = metricsMap.get("pulsar_txn_tb_client_pending_requests"); assertEquals(abortFailed.stream().mapToDouble(metric -> metric.value).sum(), 0); assertEquals(commitFailed.stream().mapToDouble(metric -> metric.value).sum(), 0); for (int i = 0; i < partitions; i++) { String topic = partitionedTopicName.getPartition(i).toString(); - Optional optional = abortLatencyCount.stream() + Optional optional = abortLatencyCount.stream() .filter(metric -> metric.tags.get("topic").equals(topic)).findFirst(); assertTrue(optional.isPresent()); assertEquals(optional.get().value, 1D); - Optional optional1 = commitLatencyCount.stream() + Optional optional1 = commitLatencyCount.stream() .filter(metric -> metric.tags.get("topic").equals(topic)).findFirst(); assertTrue(optional1.isPresent()); assertEquals(optional1.get().value, 1D); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/pendingack/PendingAckPersistentTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/pendingack/PendingAckPersistentTest.java index bc537fb784f0e..ab9f7f217e7e9 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/pendingack/PendingAckPersistentTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/pendingack/PendingAckPersistentTest.java @@ -49,7 +49,7 @@ import org.apache.pulsar.broker.service.BrokerServiceException; import org.apache.pulsar.broker.service.persistent.PersistentSubscription; import org.apache.pulsar.broker.service.persistent.PersistentTopic; -import org.apache.pulsar.broker.stats.PrometheusMetricsTest; +import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient; import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsGenerator; import org.apache.pulsar.broker.transaction.TransactionTestBase; import org.apache.pulsar.broker.transaction.pendingack.impl.MLPendingAckStore; @@ -256,28 +256,28 @@ public void testPendingAckMetrics() throws Exception { ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); PrometheusMetricsGenerator.generate(pulsarServiceList.get(0), true, false, false, statsOut); String metricsStr = statsOut.toString(); - Multimap metricsMap = PrometheusMetricsTest.parseMetrics(metricsStr); + Multimap metricsMap = PrometheusMetricsClient.parseMetrics(metricsStr); - Collection abortedCount = metricsMap.get("pulsar_txn_tp_aborted_count_total"); - Collection committedCount = metricsMap.get("pulsar_txn_tp_committed_count_total"); - Collection commitLatency = metricsMap.get("pulsar_txn_tp_commit_latency"); + Collection abortedCount = metricsMap.get("pulsar_txn_tp_aborted_count_total"); + Collection committedCount = metricsMap.get("pulsar_txn_tp_committed_count_total"); + Collection commitLatency = metricsMap.get("pulsar_txn_tp_commit_latency"); Assert.assertTrue(commitLatency.size() > 0); int count = 0; - for (PrometheusMetricsTest.Metric metric : commitLatency) { + for (PrometheusMetricsClient.Metric metric : commitLatency) { if (metric.tags.get("topic").endsWith(PENDING_ACK_REPLAY_TOPIC) && metric.value > 0) { count++; } } Assert.assertTrue(count > 0); - for (PrometheusMetricsTest.Metric metric : abortedCount) { + for (PrometheusMetricsClient.Metric metric : abortedCount) { if (metric.tags.get("subscription").equals(subName) && metric.tags.get("status").equals("succeed")) { assertTrue(metric.tags.get("topic").endsWith(PENDING_ACK_REPLAY_TOPIC)); assertTrue(metric.value > 0); } } - for (PrometheusMetricsTest.Metric metric : committedCount) { + for (PrometheusMetricsClient.Metric metric : committedCount) { if (metric.tags.get("subscription").equals(subName) && metric.tags.get("status").equals("succeed")) { assertTrue(metric.tags.get("topic").endsWith(PENDING_ACK_REPLAY_TOPIC)); assertTrue(metric.value > 0); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/web/WebServiceTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/web/WebServiceTest.java index 405f3a11b5d90..6c38598f2dce2 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/web/WebServiceTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/web/WebServiceTest.java @@ -51,7 +51,7 @@ import org.apache.commons.lang3.StringUtils; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.ServiceConfiguration; -import org.apache.pulsar.broker.stats.PrometheusMetricsTest; +import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient; import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsGenerator; import org.apache.pulsar.broker.testcontext.PulsarTestContext; import org.apache.pulsar.client.admin.PulsarAdmin; @@ -104,31 +104,31 @@ public void testWebExecutorMetrics() throws Exception { ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); PrometheusMetricsGenerator.generate(pulsar, false, false, false, statsOut); String metricsStr = statsOut.toString(); - Multimap metrics = PrometheusMetricsTest.parseMetrics(metricsStr); + Multimap metrics = PrometheusMetricsClient.parseMetrics(metricsStr); - Collection maxThreads = metrics.get("pulsar_web_executor_max_threads"); - Collection minThreads = metrics.get("pulsar_web_executor_min_threads"); - Collection activeThreads = metrics.get("pulsar_web_executor_active_threads"); - Collection idleThreads = metrics.get("pulsar_web_executor_idle_threads"); - Collection currentThreads = metrics.get("pulsar_web_executor_current_threads"); + Collection maxThreads = metrics.get("pulsar_web_executor_max_threads"); + Collection minThreads = metrics.get("pulsar_web_executor_min_threads"); + Collection activeThreads = metrics.get("pulsar_web_executor_active_threads"); + Collection idleThreads = metrics.get("pulsar_web_executor_idle_threads"); + Collection currentThreads = metrics.get("pulsar_web_executor_current_threads"); - for (PrometheusMetricsTest.Metric metric : maxThreads) { + for (PrometheusMetricsClient.Metric metric : maxThreads) { Assert.assertNotNull(metric.tags.get("cluster")); Assert.assertTrue(metric.value > 0); } - for (PrometheusMetricsTest.Metric metric : minThreads) { + for (PrometheusMetricsClient.Metric metric : minThreads) { Assert.assertNotNull(metric.tags.get("cluster")); Assert.assertTrue(metric.value > 0); } - for (PrometheusMetricsTest.Metric metric : activeThreads) { + for (PrometheusMetricsClient.Metric metric : activeThreads) { Assert.assertNotNull(metric.tags.get("cluster")); Assert.assertTrue(metric.value >= 0); } - for (PrometheusMetricsTest.Metric metric : idleThreads) { + for (PrometheusMetricsClient.Metric metric : idleThreads) { Assert.assertNotNull(metric.tags.get("cluster")); Assert.assertTrue(metric.value >= 0); } - for (PrometheusMetricsTest.Metric metric : currentThreads) { + for (PrometheusMetricsClient.Metric metric : currentThreads) { Assert.assertNotNull(metric.tags.get("cluster")); Assert.assertTrue(metric.value > 0); } diff --git a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/TopicStats.java b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/TopicStats.java index 985e42b280eb9..ac50763b7e097 100644 --- a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/TopicStats.java +++ b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/TopicStats.java @@ -64,6 +64,31 @@ public interface TopicStats { /** Get the publish time of the earliest message over all the backlogs. */ long getEarliestMsgPublishTimeInBacklogs(); + /** the size in bytes of the topic backlog quota. */ + long getBacklogQuotaLimitSize(); + + /** the topic backlog age quota, in seconds. */ + long getBacklogQuotaLimitTime(); + + /** + * Age of oldest unacknowledged message, as recorded in last backlog quota check interval. + *

+ * The age of the oldest unacknowledged (i.e. backlog) message, measured by the time elapsed from its published + * time, in seconds. This value is recorded every backlog quota check interval, hence it represents the value + * seen in the last check. + *

+ */ + long getOldestBacklogMessageAgeSeconds(); + + /** + * The subscription name containing oldest unacknowledged message as recorded in last backlog quota check. + *

+ * The name of the subscription containing the oldest unacknowledged message. This value is recorded every backlog + * quota check interval, hence it represents the value seen in the last check. + *

+ */ + String getOldestBacklogMessageSubscriptionName(); + /** Space used to store the offloaded messages for the topic/. */ long getOffloadedStorageSize(); diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/TopicStatsImpl.java b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/TopicStatsImpl.java index 2381aef52c2a9..70cf4cd341484 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/TopicStatsImpl.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/TopicStatsImpl.java @@ -246,6 +246,10 @@ public void reset() { this.compaction.reset(); this.ownerBroker = null; this.bucketDelayedIndexStats.clear(); + this.backlogQuotaLimitSize = 0; + this.backlogQuotaLimitTime = 0; + this.oldestBacklogMessageAgeSeconds = -1; + this.oldestBacklogMessageSubscriptionName = null; } // if the stats are added for the 1st time, we will need to make a copy of these stats and add it to the current @@ -275,6 +279,12 @@ public TopicStatsImpl add(TopicStats ts) { this.ongoingTxnCount = stats.ongoingTxnCount; this.abortedTxnCount = stats.abortedTxnCount; this.committedTxnCount = stats.committedTxnCount; + this.backlogQuotaLimitTime = stats.backlogQuotaLimitTime; + this.backlogQuotaLimitSize = stats.backlogQuotaLimitSize; + if (stats.oldestBacklogMessageAgeSeconds > this.oldestBacklogMessageAgeSeconds) { + this.oldestBacklogMessageAgeSeconds = stats.oldestBacklogMessageAgeSeconds; + this.oldestBacklogMessageSubscriptionName = stats.oldestBacklogMessageSubscriptionName; + } stats.bucketDelayedIndexStats.forEach((k, v) -> { TopicMetricBean topicMetricBean = From 483a35d42c2eff2c38866d19989f20a20886c809 Mon Sep 17 00:00:00 2001 From: Asaf Mesika Date: Thu, 28 Dec 2023 13:15:18 +0200 Subject: [PATCH 4/9] Finished testing --- .../mledger/impl/ManagedCursorContainer.java | 26 +- .../impl/ManagedCursorContainerTest.java | 28 +- .../broker/service/BacklogQuotaManager.java | 47 ++- .../nonpersistent/NonPersistentTopic.java | 3 +- .../service/persistent/PersistentTopic.java | 50 ++- .../persistent/PersistentTopicMetrics.java | 3 +- .../broker/stats/prometheus/TopicStats.java | 3 +- .../broker/admin/TopicPoliciesTest.java | 2 +- .../service/BacklogQuotaManagerTest.java | 336 +++++++++++++----- .../broker/service/SubscriptionSeekTest.java | 24 ++ .../persistent/BucketDelayedDeliveryTest.java | 29 +- .../persistent/PersistentTopicTest.java | 9 +- .../service/schema/SchemaServiceTest.java | 23 +- .../broker/stats/ConsumerStatsTest.java | 9 +- .../broker/stats/MetadataStoreStatsTest.java | 31 +- .../broker/stats/PrometheusMetricsTest.java | 239 ++++++------- .../broker/stats/SubscriptionStatsTest.java | 15 +- .../broker/stats/TransactionMetricsTest.java | 24 +- .../prometheus/PrometheusMetricsClient.java | 37 +- .../buffer/TransactionBufferClientTest.java | 38 +- .../pendingack/PendingAckPersistentTest.java | 21 +- .../pulsar/broker/web/WebServiceTest.java | 25 +- pulsar-broker/src/test/resources/log4j2.xml | 39 ++ .../apache/pulsar/client/api/Consumer.java | 3 + 24 files changed, 660 insertions(+), 404 deletions(-) create mode 100644 pulsar-broker/src/test/resources/log4j2.xml diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorContainer.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorContainer.java index e5850d398b316..f6fb13457fd66 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorContainer.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorContainer.java @@ -29,7 +29,6 @@ import org.apache.bookkeeper.mledger.ManagedCursor; import org.apache.bookkeeper.mledger.Position; import org.apache.commons.lang3.tuple.Pair; -import org.slf4j.LoggerFactory; /** * Contains cursors for a ManagedLedger. @@ -46,8 +45,6 @@ */ public class ManagedCursorContainer implements Iterable { - private static final org.slf4j.Logger log = LoggerFactory.getLogger(ManagedCursorContainer.class); - /** * This field is incremented everytime the cursor information is updated. */ @@ -56,10 +53,14 @@ public class ManagedCursorContainer implements Iterable { @Value public static class CursorInfo { ManagedCursor cursor; - PositionImpl markDeletePosition; + PositionImpl position; - // Use {@link DataVersion#compareVersions(long, long)} to compare between two versions, - // since it rolls over to 0 once reaching Long.MAX_VALUE + /** + * Cursor info's version. + *

+ * Use {@link DataVersion#compareVersions(long, long)} to compare between two versions, + * since it rolls over to 0 once reaching Long.MAX_VALUE + */ long version; } @@ -150,7 +151,7 @@ public ManagedCursorContainer() {} * @param position position of the cursor to use for ordering, pass null if the cursor's position shouldn't be * tracked for the slowest reader. */ - @SuppressWarnings("NonAtomicOperationOnVolatileField") + @SuppressWarnings("NonAtomicOperationOnVolatileField") // We have rw lock for that public void add(ManagedCursor cursor, Position position) { long stamp = rwLock.writeLock(); try { @@ -181,7 +182,7 @@ public ManagedCursor get(String name) { } } - @SuppressWarnings("NonAtomicOperationOnVolatileField") + @SuppressWarnings("NonAtomicOperationOnVolatileField") // we have rw lock for that public boolean removeCursor(String name) { long stamp = rwLock.writeLock(); try { @@ -224,7 +225,7 @@ public boolean removeCursor(String name) { * @return a pair of positions, representing the previous slowest reader and the new slowest reader (after the * update). */ - @SuppressWarnings("NonAtomicOperationOnVolatileField") + @SuppressWarnings("NonAtomicOperationOnVolatileField") // we have rw lock for that public Pair cursorUpdated(ManagedCursor cursor, Position newPosition) { requireNonNull(cursor); @@ -236,7 +237,6 @@ public Pair cursorUpdated(ManagedCursor cursor, Posi } PositionImpl previousSlowestConsumer = heap.get(0).position; - log.info("Cursor {} change position from {} to {}", cursor.getName(), item.position, newPosition); item.position = (PositionImpl) newPosition; version = DataVersion.incrementVersion(version); @@ -282,10 +282,10 @@ public ManagedCursor getSlowestReader() { } /** - * @return Returns the CursorInfo for the cursor with oldest mark-delete position, or null if there aren't - * any tracked cursors + * @return Returns the CursorInfo for the cursor with the oldest position, + * or null if there aren't any tracked cursors */ - public CursorInfo getCursorWithOldestMarkDeletePosition() { + public CursorInfo getCursorWithOldestPosition() { long stamp = rwLock.readLock(); try { if (heap.isEmpty()) { diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorContainerTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorContainerTest.java index 5c4dbe8913632..2653f23cdcf6c 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorContainerTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorContainerTest.java @@ -456,19 +456,19 @@ public void simple() throws Exception { ManagedCursor cursor1 = new MockManagedCursor(container, "test1", new PositionImpl(5, 5)); container.add(cursor1, cursor1.getMarkDeletedPosition()); assertEquals(container.getSlowestReaderPosition(), new PositionImpl(5, 5)); - assertEqualsCursorAndMarkDelete(container.getCursorWithOldestMarkDeletePosition(), + assertEqualsCursorAndPosition(container.getCursorWithOldestPosition(), cursor1, new PositionImpl(5, 5)); ManagedCursor cursor2 = new MockManagedCursor(container, "test2", new PositionImpl(2, 2)); container.add(cursor2, cursor2.getMarkDeletedPosition()); assertEquals(container.getSlowestReaderPosition(), new PositionImpl(2, 2)); - assertEqualsCursorAndMarkDelete(container.getCursorWithOldestMarkDeletePosition(), + assertEqualsCursorAndPosition(container.getCursorWithOldestPosition(), cursor2, new PositionImpl(2, 2)); ManagedCursor cursor3 = new MockManagedCursor(container, "test3", new PositionImpl(2, 0)); container.add(cursor3, cursor3.getMarkDeletedPosition()); assertEquals(container.getSlowestReaderPosition(), new PositionImpl(2, 0)); - assertEqualsCursorAndMarkDelete(container.getCursorWithOldestMarkDeletePosition(), + assertEqualsCursorAndPosition(container.getCursorWithOldestPosition(), cursor3, new PositionImpl(2, 0)); assertEquals(container.toString(), "[test1=5:5, test2=2:2, test3=2:0]"); @@ -483,7 +483,7 @@ public void simple() throws Exception { cursor3.markDelete(new PositionImpl(3, 0)); assertEquals(container.getSlowestReaderPosition(), new PositionImpl(2, 2)); - assertEqualsCursorAndMarkDelete(container.getCursorWithOldestMarkDeletePosition(), + assertEqualsCursorAndPosition(container.getCursorWithOldestPosition(), cursor2, new PositionImpl(2, 2)); cursor2.markDelete(new PositionImpl(10, 5)); @@ -496,7 +496,7 @@ public void simple() throws Exception { container.removeCursor(cursor5.getName()); container.removeCursor(cursor1.getName()); assertEquals(container.getSlowestReaderPosition(), new PositionImpl(4, 0)); - assertEqualsCursorAndMarkDelete(container.getCursorWithOldestMarkDeletePosition(), + assertEqualsCursorAndPosition(container.getCursorWithOldestPosition(), cursor4, new PositionImpl(4, 0)); assertTrue(container.hasDurableCursors()); @@ -533,15 +533,15 @@ public void updatingCursorOutsideContainer() { container.cursorUpdated(cursor2, cursor2.position); assertEquals(container.getSlowestReaderPosition(), new PositionImpl(5, 5)); - assertEqualsCursorAndMarkDelete(container.getCursorWithOldestMarkDeletePosition(), + assertEqualsCursorAndPosition(container.getCursorWithOldestPosition(), cursor1, new PositionImpl(5, 5)); } - private void assertEqualsCursorAndMarkDelete(ManagedCursorContainer.CursorInfo cursorInfo, - ManagedCursor expectedCursor, - PositionImpl expectedMarkDeletePosition) { + private void assertEqualsCursorAndPosition(ManagedCursorContainer.CursorInfo cursorInfo, + ManagedCursor expectedCursor, + PositionImpl expectedPosition) { assertThat(cursorInfo.getCursor().getName()).isEqualTo(expectedCursor.getName()); - assertThat(cursorInfo.getMarkDeletePosition()).isEqualTo(expectedMarkDeletePosition); + assertThat(cursorInfo.getPosition()).isEqualTo(expectedPosition); } @Test @@ -777,22 +777,22 @@ public void testVersions() { MockManagedCursor c2 = new MockManagedCursor(container, "test2", new PositionImpl(5, 1)); container.add(c1, c1.getMarkDeletedPosition()); - long version = container.getCursorWithOldestMarkDeletePosition().getVersion(); + long version = container.getCursorWithOldestPosition().getVersion(); container.add(c2, c2.getMarkDeletedPosition()); - long newVersion = container.getCursorWithOldestMarkDeletePosition().getVersion(); + long newVersion = container.getCursorWithOldestPosition().getVersion(); // newVersion > version assertThat(ManagedCursorContainer.DataVersion.compareVersions(newVersion, version)).isPositive(); version = newVersion; container.cursorUpdated(c2, new PositionImpl(5, 8)); - newVersion = container.getCursorWithOldestMarkDeletePosition().getVersion(); + newVersion = container.getCursorWithOldestPosition().getVersion(); // newVersion > version assertThat(ManagedCursorContainer.DataVersion.compareVersions(newVersion, version)).isPositive(); version = newVersion; container.removeCursor("test2"); - newVersion = container.getCursorWithOldestMarkDeletePosition().getVersion(); + newVersion = container.getCursorWithOldestPosition().getVersion(); // newVersion > version assertThat(ManagedCursorContainer.DataVersion.compareVersions(newVersion, version)).isPositive(); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BacklogQuotaManager.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BacklogQuotaManager.java index 16b2b3111c513..c889062088e00 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BacklogQuotaManager.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BacklogQuotaManager.java @@ -48,8 +48,6 @@ public class BacklogQuotaManager { private final BacklogQuotaImpl defaultQuota; private final NamespaceResources namespaceResources; - - public BacklogQuotaManager(PulsarService pulsar) { double backlogQuotaGB = pulsar.getConfiguration().getBacklogQuotaDefaultLimitGB(); this.defaultQuota = BacklogQuotaImpl.builder() @@ -94,28 +92,28 @@ public void handleExceededBacklogQuota(PersistentTopic persistentTopic, BacklogQ persistentTopic.getName(), quota.getPolicy()); switch (quota.getPolicy()) { case consumer_backlog_eviction: - switch (backlogQuotaType) { - case destination_storage: - dropBacklogForSizeLimit(persistentTopic, quota); - topicBacklogQuotaMetrics.recordSizeBasedBacklogEviction(); - break; - case message_age: - dropBacklogForTimeLimit(persistentTopic, quota, preciseTimeBasedBacklogQuotaCheck); - topicBacklogQuotaMetrics.recordTimeBasedBacklogEviction(); - break; - default: - break; - } - break; - case producer_exception: - case producer_request_hold: - if (!advanceSlowestSystemCursor(persistentTopic)) { - // The slowest is not a system cursor. Disconnecting producers to put backpressure. - disconnectProducers(persistentTopic); - } - break; - default: - break; + switch (backlogQuotaType) { + case destination_storage: + dropBacklogForSizeLimit(persistentTopic, quota); + topicBacklogQuotaMetrics.recordSizeBasedBacklogEviction(); + break; + case message_age: + dropBacklogForTimeLimit(persistentTopic, quota, preciseTimeBasedBacklogQuotaCheck); + topicBacklogQuotaMetrics.recordTimeBasedBacklogEviction(); + break; + default: + break; + } + break; + case producer_exception: + case producer_request_hold: + if (!advanceSlowestSystemCursor(persistentTopic)) { + // The slowest is not a system cursor. Disconnecting producers to put backpressure. + disconnectProducers(persistentTopic); + } + break; + default: + break; } } @@ -200,7 +198,6 @@ private void dropBacklogForSizeLimit(PersistentTopic persistentTopic, BacklogQuo * @param quota * Backlog quota set for the topic */ - @SuppressWarnings("checkstyle:LineLength") private void dropBacklogForTimeLimit(PersistentTopic persistentTopic, BacklogQuota quota, boolean preciseTimeBasedBacklogQuotaCheck) { // If enabled precise time based backlog quota check, will expire message based on the timeBaseQuota diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopic.java index c98ccb9b473cd..87565fe144e99 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopic.java @@ -65,6 +65,7 @@ import org.apache.pulsar.broker.service.Topic; import org.apache.pulsar.broker.service.TopicPolicyListener; import org.apache.pulsar.broker.service.TransportCnx; +import org.apache.pulsar.broker.service.persistent.PersistentTopic; import org.apache.pulsar.broker.stats.ClusterReplicationMetrics; import org.apache.pulsar.broker.stats.NamespaceStats; import org.apache.pulsar.client.api.MessageId; @@ -1259,6 +1260,6 @@ public boolean isPersistent() { @Override public long getBestEffortOldestUnacknowledgedMessageAgeSeconds() { - return -1; + return PersistentTopic.NOT_AVAILABLE_YET; } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java index 47a41295a5b0f..46b767895daa4 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java @@ -196,13 +196,7 @@ public class PersistentTopic extends AbstractTopic implements Topic, AddEntryCallback { - private static final AtomicReferenceFieldUpdater - TIME_BASED_BACKLOG_QUOTA_CHECK_RESULT_UPDATER = AtomicReferenceFieldUpdater.newUpdater( - PersistentTopic.class, - TimeBasedBacklogQuotaCheckResult.class, - "timeBasedBacklogQuotaCheckResult"); - - private static final long NOT_AVAILABLE_YET = -1; + public static final long NOT_AVAILABLE_YET = -1; // Managed ledger associated with the topic protected final ManagedLedger ledger; @@ -284,7 +278,11 @@ protected TopicStatsHelper initialValue() { private final PersistentTopicMetrics persistentTopicMetrics = new PersistentTopicMetrics(); private volatile TimeBasedBacklogQuotaCheckResult timeBasedBacklogQuotaCheckResult; - + private static final AtomicReferenceFieldUpdater + TIME_BASED_BACKLOG_QUOTA_CHECK_RESULT_UPDATER = AtomicReferenceFieldUpdater.newUpdater( + PersistentTopic.class, + TimeBasedBacklogQuotaCheckResult.class, + "timeBasedBacklogQuotaCheckResult"); @Value private static class TimeBasedBacklogQuotaCheckResult { PositionImpl oldestCursorMarkDeletePosition; @@ -3229,16 +3227,17 @@ public CompletableFuture checkTimeBacklogExceeded() { int backlogQuotaLimitInSecond = getBacklogQuota(BacklogQuotaType.message_age).getLimitTime(); ManagedCursorContainer managedCursorContainer = (ManagedCursorContainer) ledger.getCursors(); - CursorInfo oldestMarkDeleteCursorInfo = managedCursorContainer.getCursorWithOldestMarkDeletePosition(); + CursorInfo oldestMarkDeleteCursorInfo = managedCursorContainer.getCursorWithOldestPosition(); - // If backlog quota by time is not set, and we have no durable cursor. + // If backlog quota by time is not set, and we have no durable cursor + // since `ledger.getCursors()` only managed durable cursors if (backlogQuotaLimitInSecond <= 0 || oldestMarkDeleteCursorInfo == null - || oldestMarkDeleteCursorInfo.getMarkDeletePosition() == null) { + || oldestMarkDeleteCursorInfo.getPosition() == null) { return CompletableFuture.completedFuture(false); } - PositionImpl oldestMarkDeletePosition = oldestMarkDeleteCursorInfo.getMarkDeletePosition(); + PositionImpl oldestMarkDeletePosition = oldestMarkDeleteCursorInfo.getPosition(); TimeBasedBacklogQuotaCheckResult lastCheckResult = timeBasedBacklogQuotaCheckResult; if (lastCheckResult != null @@ -3253,10 +3252,6 @@ public CompletableFuture checkTimeBacklogExceeded() { oldestMarkDeleteCursorInfo.getVersion()); updateResultIfNewer(updatedResult); - log.info("checkTimeBacklogExceeded for topic {}: Same location, different cursor. " - + "Cursor " + oldestMarkDeleteCursorInfo.getCursor().getName() - + ", markDeletePosition = " + oldestMarkDeleteCursorInfo.getMarkDeletePosition(), - topic); } long entryTimestamp = lastCheckResult.getPositionPublishTimestampInMillis(); @@ -3266,16 +3261,10 @@ public CompletableFuture checkTimeBacklogExceeded() { + "exceeded quota {}", lastCheckResult.getCursorName(), backlogQuotaLimitInSecond); } - log.info("checkTimeBacklogExceeded for topic {}: Same position, getting it from cache. " - + "Cursor " + oldestMarkDeleteCursorInfo.getCursor().getName() - + ", markDeletePosition = " + oldestMarkDeleteCursorInfo.getMarkDeletePosition(), topic); return CompletableFuture.completedFuture(expired); } if (brokerService.pulsar().getConfiguration().isPreciseTimeBasedBacklogQuotaCheck()) { - log.info("checkTimeBacklogExceeded for topic {}: Location changed. " - + "Cursor " + oldestMarkDeleteCursorInfo.getCursor().getName() - + ", markDeletePosition = " + oldestMarkDeleteCursorInfo.getMarkDeletePosition(), topic); CompletableFuture future = new CompletableFuture<>(); // Check if first unconsumed message(first message after mark delete position) // for slowest cursor's has expired. @@ -3289,7 +3278,7 @@ public void readEntryComplete(Entry entry, Object ctx) { updateResultIfNewer( new TimeBasedBacklogQuotaCheckResult( - oldestMarkDeleteCursorInfo.getMarkDeletePosition(), + oldestMarkDeleteCursorInfo.getPosition(), oldestMarkDeleteCursorInfo.getCursor().getName(), entryTimestamp, oldestMarkDeleteCursorInfo.getVersion())); @@ -3318,16 +3307,12 @@ public void readEntryFailed(ManagedLedgerException exception, Object ctx) { }, null); return future; } else { - log.info("checkTimeBacklogExceeded for topic {}: Location changed (not precise). " - + "Cursor " + oldestMarkDeleteCursorInfo.getCursor().getName() - + ", markDeletePosition = " + oldestMarkDeleteCursorInfo.getMarkDeletePosition(), topic); - try { CheckResult checkResult = estimatedTimeBasedBacklogQuotaCheck(oldestMarkDeletePosition); if (checkResult.getEstimatedOldestUnacknowledgedMessageTimestamp() != null) { updateResultIfNewer( new TimeBasedBacklogQuotaCheckResult( - oldestMarkDeleteCursorInfo.getMarkDeletePosition(), + oldestMarkDeleteCursorInfo.getPosition(), oldestMarkDeleteCursorInfo.getCursor().getName(), checkResult.getEstimatedOldestUnacknowledgedMessageTimestamp(), oldestMarkDeleteCursorInfo.getVersion())); @@ -3345,6 +3330,9 @@ private CheckResult estimatedTimeBasedBacklogQuotaCheck(PositionImpl markDeleteP throws ExecutionException, InterruptedException { int backlogQuotaLimitInSecond = getBacklogQuota(BacklogQuotaType.message_age).getLimitTime(); ManagedLedgerImpl managedLedger = (ManagedLedgerImpl) ledger; + + // The ledger timestamp is only known when ledger is closed, hence when the mark-delete + // is at active ledger (open) we can't estimate it. if (managedLedger.getLedgersInfo().lastKey().equals(markDeletePosition.getLedgerId())) { return new CheckResult(false, null); } @@ -3367,6 +3355,12 @@ private CheckResult estimatedTimeBasedBacklogQuotaCheck(PositionImpl markDeleteP && positionToCheckLedgerInfo.getTimestamp() > 0) { long estimateMsgAgeMs = managedLedger.getClock().millis() - positionToCheckLedgerInfo.getTimestamp(); boolean shouldTruncateBacklog = estimateMsgAgeMs > SECONDS.toMillis(backlogQuotaLimitInSecond); + if (log.isDebugEnabled()) { + log.debug("Time based backlog quota exceeded, quota {}, age of ledger " + + "slowest cursor currently on {}", backlogQuotaLimitInSecond * 1000, + estimateMsgAgeMs); + } + return new CheckResult(shouldTruncateBacklog, positionToCheckLedgerInfo.getTimestamp()); } else { return new CheckResult(false, null); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopicMetrics.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopicMetrics.java index 50ce1e0db7755..f89f730bcc184 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopicMetrics.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopicMetrics.java @@ -9,7 +9,6 @@ public class PersistentTopicMetrics { @Getter private final BacklogQuotaMetrics backlogQuotaMetrics = new BacklogQuotaMetrics(); - public static class BacklogQuotaMetrics { private final LongAdder timeBasedBacklogQuotaExceededEvictionCount = new LongAdder(); private final LongAdder sizeBasedBacklogQuotaExceededEvictionCount = new LongAdder(); @@ -30,4 +29,4 @@ public long getTimeBasedBacklogQuotaExceededEvictionCount() { return timeBasedBacklogQuotaExceededEvictionCount.longValue(); } } -} +} \ No newline at end of file diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/TopicStats.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/TopicStats.java index eeda0308b2144..4be006423f509 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/TopicStats.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/TopicStats.java @@ -122,6 +122,7 @@ public void reset() { timeBasedBacklogQuotaExceededEvictionCount = 0; sizeBasedBacklogQuotaExceededEvictionCount = 0; + backlogAgeSeconds = -1; } @SuppressWarnings("OptionalUsedAsFieldOrParameterType") @@ -462,7 +463,7 @@ private static void writeMetric(PrometheusMetricStreams stream, String metricNam writeTopicMetric(stream, metricName, value, cluster, namespace, topic, splitTopicAndPartitionIndexLabel); } - @SuppressWarnings({"CheckStyle", "SameParameterValue"}) + @SuppressWarnings("SameParameterValue") private static void writeBacklogQuotaMetric(PrometheusMetricStreams stream, String metricName, Number value, String cluster, String namespace, String topic, boolean splitTopicAndPartitionIndexLabel, diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicPoliciesTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicPoliciesTest.java index 2dbf12a15b10f..25ca3bf1444d2 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicPoliciesTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicPoliciesTest.java @@ -145,7 +145,7 @@ public void updatePropertiesForAutoCreatedTopicTest() throws Exception { @Cleanup Producer producer = pulsarClient.newProducer().topic(testTopic).create(); HashMap properties = new HashMap<>(); - properties.put("backlogQuotaTypeLabel", "message_age"); + properties.put("backlogQuotaType", "message_age"); admin.topics().updateProperties(testTopic, properties); admin.topics().delete(topicName.toString(), true); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BacklogQuotaManagerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BacklogQuotaManagerTest.java index f6ef30c7f3b5c..5f85dca8f24fa 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BacklogQuotaManagerTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BacklogQuotaManagerTest.java @@ -40,12 +40,13 @@ import java.util.UUID; import java.util.concurrent.CountDownLatch; import java.util.concurrent.CyclicBarrier; -import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.atomic.AtomicReference; import lombok.Cleanup; import org.apache.bookkeeper.mledger.Position; import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; +import org.apache.commons.lang3.tuple.Pair; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; @@ -206,7 +207,7 @@ private void rolloverStats() { } /** - * Readers should not effect backlog quota + * Readers should not affect backlog quota */ @Test public void testBacklogQuotaWithReader() throws Exception { @@ -218,18 +219,18 @@ public void testBacklogQuotaWithReader() throws Exception { .limitTime(TIME_TO_CHECK_BACKLOG_QUOTA) .retentionPolicy(BacklogQuota.RetentionPolicy.producer_exception) .build()); - try (PulsarClient client = PulsarClient.builder().serviceUrl(adminUrl.toString()).statsInterval(0, TimeUnit.SECONDS).build();) { + try (PulsarClient client = PulsarClient.builder().serviceUrl(adminUrl.toString()).statsInterval(0, SECONDS).build()) { final String topic1 = "persistent://prop/ns-quota/topic1" + UUID.randomUUID(); final int numMsgs = 20; Reader reader = client.newReader().topic(topic1).receiverQueueSize(1).startMessageId(MessageId.latest).create(); - org.apache.pulsar.client.api.Producer producer = createProducer(client, topic1); + Producer producer = createProducer(client, topic1); byte[] content = new byte[1024]; for (int i = 0; i < numMsgs; i++) { content[0] = (byte) (content[0] + 1); - MessageId msgId = producer.send(content); + producer.send(content); } Thread.sleep((TIME_TO_CHECK_BACKLOG_QUOTA + 1) * 1000); @@ -278,7 +279,7 @@ public void testBacklogQuotaWithReader() throws Exception { // check reader can still read with out error while (true) { - Message msg = reader.readNext(5, TimeUnit.SECONDS); + Message msg = reader.readNext(5, SECONDS); if (msg == null) { break; } @@ -303,10 +304,11 @@ public void testTriggerBacklogQuotaSizeWithReader() throws Exception { .limitTime(TIME_TO_CHECK_BACKLOG_QUOTA) .retentionPolicy(BacklogQuota.RetentionPolicy.producer_exception) .build()); - try (PulsarClient client = PulsarClient.builder().serviceUrl(adminUrl.toString()).statsInterval(0, TimeUnit.SECONDS).build();) { + try (PulsarClient client = PulsarClient.builder().serviceUrl(adminUrl.toString()).statsInterval(0, SECONDS).build();) { final String topic1 = "persistent://prop/ns-quota/topic1" + UUID.randomUUID(); final int numMsgs = 20; Reader reader = client.newReader().topic(topic1).receiverQueueSize(1).startMessageId(MessageId.latest).create(); + Producer producer = createProducer(client, topic1); byte[] content = new byte[1024]; for (int i = 0; i < numMsgs; i++) { @@ -343,13 +345,13 @@ public void testTriggerBacklogQuotaSizeWithReader() throws Exception { // check there is only one ledger left assertEquals(internalStats.ledgers.size(), 1); - // check if its the expected ledger id given MAX_ENTRIES_PER_LEDGER + // check if it's the expected ledger id given MAX_ENTRIES_PER_LEDGER assertEquals(internalStats.ledgers.get(0).ledgerId, finalMessageId.getLedgerId()); }); - // check reader can still read with out error + // check reader can still read without error while (true) { - Message msg = reader.readNext(5, TimeUnit.SECONDS); + Message msg = reader.readNext(5, SECONDS); if (msg == null) { break; } @@ -361,7 +363,8 @@ public void testTriggerBacklogQuotaSizeWithReader() throws Exception { } @Test - public void backlogsStats() throws PulsarAdminException, PulsarClientException, InterruptedException { + public void backlogsStatsPrecise() throws PulsarAdminException, PulsarClientException, InterruptedException { + config.setPreciseTimeBasedBacklogQuotaCheck(true); final String namespace = "prop/ns-quota"; assertEquals(admin.namespaces().getBacklogQuotaMap(namespace), new HashMap<>()); final int sizeLimitBytes = 15 * 1024 * 1024; @@ -382,53 +385,57 @@ public void backlogsStats() throws PulsarAdminException, PulsarClientException, message_age); try (PulsarClient client = PulsarClient.builder().serviceUrl(adminUrl.toString()) - .statsInterval(0, TimeUnit.SECONDS).build()) { + .statsInterval(0, SECONDS).build()) { final String topic1 = "persistent://prop/ns-quota/topic2" + UUID.randomUUID(); final String subName1 = "c1"; final String subName2 = "c2"; - final int numMsgs = 20; - - Consumer consumer1 = client.newConsumer().topic(topic1).subscriptionName(subName1).subscribe(); - Consumer consumer2 = client.newConsumer().topic(topic1).subscriptionName(subName2).subscribe(); - org.apache.pulsar.client.api.Producer producer = createProducer(client, topic1); + final int numMsgs = 5; + + Consumer consumer1 = client.newConsumer().topic(topic1).subscriptionName(subName1) + .acknowledgmentGroupTime(0, SECONDS) + .subscribe(); + Consumer consumer2 = client.newConsumer().topic(topic1).subscriptionName(subName2) + .acknowledgmentGroupTime(0, SECONDS) + .subscribe(); + Producer producer = createProducer(client, topic1); byte[] content = new byte[1024]; List messageIds = new ArrayList<>(numMsgs); for (int i = 0; i < numMsgs; i++) { - Thread.sleep(100); + Thread.sleep(1000); MessageId messageId = producer.send(content); messageIds.add(messageId); } - final long initialQuotaCheckCount = getQuotaCheckCount(); + String c1MarkDeletePositionBefore = + admin.topics().getInternalStats(topic1).cursors.get(subName1).markDeletePosition; + // Move subscription 1, one message, such that subscription 2 is the oldest + // S2 S1 + // 0 1 Message oldestMessage = consumer1.receive(); consumer1.acknowledge(oldestMessage); - log.info("Subscription 1 moved 1 message. Now subscription 2 is the oldest"); + log.info("Subscription 1 moved 1 message. Now subscription 2 is the oldest. Oldest message:"+ + oldestMessage.getMessageId()); - AtomicLong quotaCheckCount = new AtomicLong(); - Awaitility.await().pollInterval(1, SECONDS).atMost(TIME_TO_CHECK_BACKLOG_QUOTA*3, SECONDS).until(() -> { - quotaCheckCount.set(getQuotaCheckCount()); - return quotaCheckCount.get() > initialQuotaCheckCount + 1; - }); + c1MarkDeletePositionBefore = waitForMarkDeletePositionToChange(topic1, subName1, c1MarkDeletePositionBefore); + waitForQuotaCheckToRunTwice(); Metrics metrics = prometheusMetricsClient.getMetrics(); - TopicStats topicStats = getTopicStats(topic1); + assertThat(topicStats.getBacklogQuotaLimitSize()).isEqualTo(sizeLimitBytes); assertThat(topicStats.getBacklogQuotaLimitTime()).isEqualTo(timeLimitSeconds); - long expectedMessageAgeSeconds = MILLISECONDS.toSeconds(System.currentTimeMillis() - oldestMessage.getPublishTime()); assertThat(topicStats.getOldestBacklogMessageAgeSeconds()) .isCloseTo(expectedMessageAgeSeconds, within(1L)); assertThat(topicStats.getOldestBacklogMessageSubscriptionName()).isEqualTo(subName2); - List backlogAgeMetrics = - metrics.findByNameAndLabels("pulsar_storage_backlog_age_seconds", "topic", topic1); - assertThat(backlogAgeMetrics).hasSize(1); - Metric backlogAgeMetric = backlogAgeMetrics.get(0); + Metric backlogAgeMetric = + metrics.findSingleMetricByNameAndLabels("pulsar_storage_backlog_age_seconds", + Pair.of("topic", topic1)); assertThat(backlogAgeMetric.tags).containsExactly( entry("cluster", CLUSTER_NAME), entry("namespace", namespace), @@ -436,19 +443,20 @@ public void backlogsStats() throws PulsarAdminException, PulsarClientException, assertThat((long) backlogAgeMetric.value).isCloseTo(expectedMessageAgeSeconds, within(1L)); // Move subscription 2 away from being the oldest mark delete + // S2/S1 + // 0 1 Message secondOldestMessage = consumer2.receive(); consumer2.acknowledge(secondOldestMessage); // Switch subscription 1 to be where subscription 2 was in terms of oldest mark delete - consumer1.seek(oldestMessage.getMessageId()); + // S1 S2 + // 0 1 + consumer1.seek(MessageId.earliest); log.info("Subscription 1 moved to be the oldest"); - final long firstQuotaCheckCount = quotaCheckCount.get(); - Awaitility.await().pollInterval(1, SECONDS).atMost(TIME_TO_CHECK_BACKLOG_QUOTA*3, SECONDS).until(() -> { - quotaCheckCount.set(getQuotaCheckCount()); - return quotaCheckCount.get() > firstQuotaCheckCount+1; - }); + c1MarkDeletePositionBefore = waitForMarkDeletePositionToChange(topic1, subName1, c1MarkDeletePositionBefore); + waitForQuotaCheckToRunTwice(); metrics = prometheusMetricsClient.getMetrics(); long actualAge = (long) metrics.findByNameAndLabels( @@ -462,33 +470,153 @@ public void backlogsStats() throws PulsarAdminException, PulsarClientException, assertThat(topicStats.getOldestBacklogMessageSubscriptionName()).isEqualTo(subName1); // Move subscription 1 passed subscription 2 - List ids = new ArrayList<>(); for (int i = 0; i < 3; i++) { Message message = consumer1.receive(); - ids.add(message.getMessageId()); + log.info("Subscription 1 about to ack message ID {}", message.getMessageId()); + consumer1.acknowledge(message); } - consumer1.acknowledge(ids); log.info("Subscription 1 moved 3 messages. Now subscription 2 is the oldest"); - - final long secondQuotaCheckCount = quotaCheckCount.get(); - Awaitility.await().pollInterval(1, SECONDS).atMost(TIME_TO_CHECK_BACKLOG_QUOTA*3, SECONDS).until(() -> { - quotaCheckCount.set(getQuotaCheckCount()); - boolean result = quotaCheckCount.get() > (secondQuotaCheckCount + 1); - System.out.println("Quota Check Count = " +quotaCheckCount.get() - + ", second quota check count = "+secondQuotaCheckCount - + ", result = " + result); - return result; - }); + waitForMarkDeletePositionToChange(topic1, subName1, c1MarkDeletePositionBefore); + waitForQuotaCheckToRunTwice(); topicStats = getTopicStats(topic1); expectedMessageAgeSeconds = MILLISECONDS.toSeconds(System.currentTimeMillis() - secondOldestMessage.getPublishTime()); assertThat(topicStats.getOldestBacklogMessageAgeSeconds()).isCloseTo(expectedMessageAgeSeconds, within(1L)); assertThat(topicStats.getOldestBacklogMessageSubscriptionName()).isEqualTo(subName2); + } + } + + @Test + public void backlogsStatsNotPrecise() throws PulsarAdminException, PulsarClientException, InterruptedException { + config.setPreciseTimeBasedBacklogQuotaCheck(false); + final String namespace = "prop/ns-quota"; + assertEquals(admin.namespaces().getBacklogQuotaMap(namespace), new HashMap<>()); + final int sizeLimitBytes = 15 * 1024 * 1024; + final int timeLimitSeconds = 123; + admin.namespaces().setBacklogQuota( + namespace, + BacklogQuota.builder() + .limitSize(sizeLimitBytes) + .retentionPolicy(BacklogQuota.RetentionPolicy.producer_exception) + .build(), + destination_storage); + admin.namespaces().setBacklogQuota( + namespace, + BacklogQuota.builder() + .limitTime(timeLimitSeconds) + .retentionPolicy(BacklogQuota.RetentionPolicy.producer_exception) + .build(), + message_age); + + try (PulsarClient client = PulsarClient.builder().serviceUrl(adminUrl.toString()) + .statsInterval(0, SECONDS).build()) { + final String topic1 = "persistent://prop/ns-quota/topic2" + UUID.randomUUID(); + + final String subName1 = "brandNewC1"; + final String subName2 = "brandNewC2"; + final int numMsgs = 5; + + Consumer consumer1 = client.newConsumer().topic(topic1).subscriptionName(subName1) + .acknowledgmentGroupTime(0, SECONDS) + .isAckReceiptEnabled(true) + .subscribe(); + Consumer consumer2 = client.newConsumer().topic(topic1).subscriptionName(subName2) + .acknowledgmentGroupTime(0, SECONDS) + .isAckReceiptEnabled(true) + .subscribe(); + Producer producer = createProducer(client, topic1); + + byte[] content = new byte[1024]; + for (int i = 0; i < numMsgs; i++) { + Thread.sleep(500); + producer.send(content); + } + + String c1MarkDeletePositionBefore = + admin.topics().getInternalStats(topic1).cursors.get(subName1).markDeletePosition; + + consumer1.acknowledge(consumer1.receive()); + log.info("Moved subscription 1, by 1 message"); + c1MarkDeletePositionBefore = waitForMarkDeletePositionToChange(topic1, subName1, c1MarkDeletePositionBefore); + waitForQuotaCheckToRunTwice(); + + TopicStats topicStats = getTopicStats(topic1); + + // We have only one ledger, and it is not closed yet, so we can't tell the age until it is closed + assertThat(topicStats.getOldestBacklogMessageSubscriptionName()).isNull(); + + Metrics metrics = prometheusMetricsClient.getMetrics(); + Metric backlogAgeMetric = + metrics.findSingleMetricByNameAndLabels("pulsar_storage_backlog_age_seconds", + Pair.of("topic", topic1)); + assertThat(backlogAgeMetric.value).isEqualTo(-1); + + unloadAndLoadTopic(topic1, producer); + long unloadTime = System.currentTimeMillis(); + + waitForQuotaCheckToRunTwice(); + + topicStats = getTopicStats(topic1); + assertThat(topicStats.getOldestBacklogMessageSubscriptionName()).isEqualTo(subName2); + // age is measured against the ledger closing time + long expectedAge = MILLISECONDS.toSeconds(System.currentTimeMillis() - unloadTime); + assertThat(topicStats.getOldestBacklogMessageAgeSeconds()).isCloseTo(expectedAge, within(1L)); + + String c2MarkDeletePositionBefore = + admin.topics().getInternalStats(topic1).cursors.get(subName2).markDeletePosition; + Message message; + for (int i = 0; i < numMsgs-1; i++) { + consumer1.acknowledge(consumer1.receive()); + message = consumer2.receive(); + consumer2.acknowledge(message); + } + // At this point subscription 2 is the oldest + waitForMarkDeletePositionToChange(topic1, subName1, c1MarkDeletePositionBefore); + waitForMarkDeletePositionToChange(topic1, subName2, c2MarkDeletePositionBefore); + waitForQuotaCheckToRunTwice(); + + topicStats = getTopicStats(topic1); + assertThat(topicStats.getOldestBacklogMessageSubscriptionName()).isEqualTo(subName2); + expectedAge = MILLISECONDS.toSeconds(System.currentTimeMillis() - unloadTime); + assertThat(topicStats.getOldestBacklogMessageAgeSeconds()).isCloseTo(expectedAge, within(1L)); } } + private void unloadAndLoadTopic(String topic, Producer producer) throws PulsarAdminException, + PulsarClientException { + admin.topics().unload(topic); + // This will load the topic + producer.send("Bla".getBytes()); + Awaitility.await().pollInterval(100, MILLISECONDS).atMost(5, SECONDS) + .until(() -> admin.topics().getInternalStats(topic).numberOfEntries > 0); + } + + private void waitForQuotaCheckToRunTwice() { + final long initialQuotaCheckCount = getQuotaCheckCount(); + AtomicLong quotaCheckCount = new AtomicLong(); + Awaitility.await().pollInterval(1, SECONDS).atMost(TIME_TO_CHECK_BACKLOG_QUOTA*3, SECONDS).until(() -> { + quotaCheckCount.set(getQuotaCheckCount()); + return quotaCheckCount.get() > initialQuotaCheckCount + 1; + }); + } + + /** + * @return The new mark delete position + */ + private String waitForMarkDeletePositionToChange(String topic, + String subscriptionName, + String previousMarkDeletePosition) { + AtomicReference markDeletePosition = new AtomicReference<>(); + Awaitility.await().pollInterval(1, SECONDS).atMost(5, SECONDS).until(() -> { + markDeletePosition.set( + admin.topics().getInternalStats(topic).cursors.get(subscriptionName).markDeletePosition); + return markDeletePosition.get() != null && !markDeletePosition.get().equals(previousMarkDeletePosition); + }); + return markDeletePosition.get(); + } + private long getQuotaCheckCount() { Metrics metrics = prometheusMetricsClient.getMetrics(); return (long) metrics.findByNameAndLabels( @@ -512,7 +640,7 @@ public void testTriggerBacklogTimeQuotaWithReader() throws Exception { .limitTime(TIME_TO_CHECK_BACKLOG_QUOTA) .retentionPolicy(BacklogQuota.RetentionPolicy.producer_exception) .build()); - try (PulsarClient client = PulsarClient.builder().serviceUrl(adminUrl.toString()).statsInterval(0, TimeUnit.SECONDS).build();) { + try (PulsarClient client = PulsarClient.builder().serviceUrl(adminUrl.toString()).statsInterval(0, SECONDS).build();) { final String topic1 = "persistent://prop/ns-quota/topic2" + UUID.randomUUID(); final int numMsgs = 9; Reader reader = client.newReader().topic(topic1).receiverQueueSize(1).startMessageId(MessageId.latest).create(); @@ -558,7 +686,7 @@ public void testTriggerBacklogTimeQuotaWithReader() throws Exception { // check reader can still read without error while (true) { - Message msg = reader.readNext(5, TimeUnit.SECONDS); + Message msg = reader.readNext(5, SECONDS); if (msg == null) { break; } @@ -579,7 +707,7 @@ public void testConsumerBacklogEvictionSizeQuota() throws Exception { .retentionPolicy(BacklogQuota.RetentionPolicy.consumer_backlog_eviction) .build()); @Cleanup - PulsarClient client = PulsarClient.builder().serviceUrl(adminUrl.toString()).statsInterval(0, TimeUnit.SECONDS) + PulsarClient client = PulsarClient.builder().serviceUrl(adminUrl.toString()).statsInterval(0, SECONDS) .build(); final String topic1 = "persistent://prop/ns-quota/topic2" + UUID.randomUUID(); @@ -589,7 +717,7 @@ public void testConsumerBacklogEvictionSizeQuota() throws Exception { Consumer consumer1 = client.newConsumer().topic(topic1).subscriptionName(subName1).subscribe(); Consumer consumer2 = client.newConsumer().topic(topic1).subscriptionName(subName2).subscribe(); - org.apache.pulsar.client.api.Producer producer = createProducer(client, topic1); + Producer producer = createProducer(client, topic1); byte[] content = new byte[1024]; for (int i = 0; i < numMsgs; i++) { producer.send(content); @@ -602,6 +730,8 @@ public void testConsumerBacklogEvictionSizeQuota() throws Exception { TopicStats stats = getTopicStats(topic1); assertTrue(stats.getBacklogSize() < 10 * 1024, "Storage size is [" + stats.getStorageSize() + "]"); + assertThat(evictionCountMetric("prop/ns-quota", topic1, "size")).isEqualTo(1); + assertThat(evictionCountMetric("size")).isEqualTo(1); } @Test @@ -612,10 +742,10 @@ public void testConsumerBacklogEvictionTimeQuotaPrecise() throws Exception { BacklogQuota.builder() .limitTime(TIME_TO_CHECK_BACKLOG_QUOTA) .retentionPolicy(BacklogQuota.RetentionPolicy.consumer_backlog_eviction) - .build(), BacklogQuota.BacklogQuotaType.message_age); + .build(), message_age); config.setPreciseTimeBasedBacklogQuotaCheck(true); @Cleanup - PulsarClient client = PulsarClient.builder().serviceUrl(adminUrl.toString()).statsInterval(0, TimeUnit.SECONDS) + PulsarClient client = PulsarClient.builder().serviceUrl(adminUrl.toString()).statsInterval(0, SECONDS) .build(); final String topic1 = "persistent://prop/ns-quota/topic3" + UUID.randomUUID(); @@ -625,7 +755,7 @@ public void testConsumerBacklogEvictionTimeQuotaPrecise() throws Exception { Consumer consumer1 = client.newConsumer().topic(topic1).subscriptionName(subName1).subscribe(); Consumer consumer2 = client.newConsumer().topic(topic1).subscriptionName(subName2).subscribe(); - org.apache.pulsar.client.api.Producer producer = createProducer(client, topic1); + Producer producer = createProducer(client, topic1); byte[] content = new byte[1024]; for (int i = 0; i < numMsgs; i++) { producer.send(content); @@ -644,8 +774,32 @@ public void testConsumerBacklogEvictionTimeQuotaPrecise() throws Exception { // All messages for both subscription should be cleaned up from backlog by backlog monitor task. assertEquals(stats.getSubscriptions().get(subName1).getMsgBacklog(), 0); assertEquals(stats.getSubscriptions().get(subName2).getMsgBacklog(), 0); + assertThat(evictionCountMetric("prop/ns-quota", topic1, "time")).isEqualTo(1); + assertThat(evictionCountMetric("time")).isEqualTo(1); } + @SuppressWarnings("SameParameterValue") + private long evictionCountMetric(String namespace, String topic, String quotaType) { + Metrics metrics = prometheusMetricsClient.getMetrics(); + Metric topicEvictionsTotal = metrics.findSingleMetricByNameAndLabels( + "pulsar_storage_backlog_quota_exceeded_evictions_total", + Pair.of("topic", topic), + Pair.of("quota_type", quotaType), + Pair.of("namespace", namespace), + Pair.of("cluster", CLUSTER_NAME)); + return (long) topicEvictionsTotal.value; + } + + private long evictionCountMetric(String quotaType) { + Metrics metrics = prometheusMetricsClient.getMetrics(); + Metric topicEvictionsTotal = metrics.findSingleMetricByNameAndLabels( + "pulsar_broker_storage_backlog_quota_exceeded_evictions_total", + Pair.of("quota_type", quotaType), + Pair.of("cluster", CLUSTER_NAME)); + return (long) topicEvictionsTotal.value; + } + + @Test(timeOut = 60000) public void testConsumerBacklogEvictionTimeQuota() throws Exception { assertEquals(admin.namespaces().getBacklogQuotaMap("prop/ns-quota"), @@ -654,9 +808,9 @@ public void testConsumerBacklogEvictionTimeQuota() throws Exception { BacklogQuota.builder() .limitTime(TIME_TO_CHECK_BACKLOG_QUOTA) .retentionPolicy(BacklogQuota.RetentionPolicy.consumer_backlog_eviction) - .build(), BacklogQuota.BacklogQuotaType.message_age); + .build(), message_age); @Cleanup - PulsarClient client = PulsarClient.builder().serviceUrl(adminUrl.toString()).statsInterval(0, TimeUnit.SECONDS) + PulsarClient client = PulsarClient.builder().serviceUrl(adminUrl.toString()).statsInterval(0, SECONDS) .build(); final String topic1 = "persistent://prop/ns-quota/topic3" + UUID.randomUUID(); @@ -666,7 +820,7 @@ public void testConsumerBacklogEvictionTimeQuota() throws Exception { Consumer consumer1 = client.newConsumer().topic(topic1).subscriptionName(subName1).subscribe(); Consumer consumer2 = client.newConsumer().topic(topic1).subscriptionName(subName2).subscribe(); - org.apache.pulsar.client.api.Producer producer = createProducer(client, topic1); + Producer producer = createProducer(client, topic1); byte[] content = new byte[1024]; for (int i = 0; i < numMsgs; i++) { producer.send(content); @@ -704,9 +858,9 @@ public void testConsumerBacklogEvictionTimeQuotaWithPartEviction() throws Except BacklogQuota.builder() .limitTime(5) // set limit time as 5 seconds .retentionPolicy(BacklogQuota.RetentionPolicy.consumer_backlog_eviction) - .build(), BacklogQuota.BacklogQuotaType.message_age); + .build(), message_age); @Cleanup - PulsarClient client = PulsarClient.builder().serviceUrl(adminUrl.toString()).statsInterval(0, TimeUnit.SECONDS) + PulsarClient client = PulsarClient.builder().serviceUrl(adminUrl.toString()).statsInterval(0, SECONDS) .build(); final String topic1 = "persistent://prop/ns-quota/topic3" + UUID.randomUUID(); @@ -716,7 +870,7 @@ public void testConsumerBacklogEvictionTimeQuotaWithPartEviction() throws Except Consumer consumer1 = client.newConsumer().topic(topic1).subscriptionName(subName1).subscribe(); Consumer consumer2 = client.newConsumer().topic(topic1).subscriptionName(subName2).subscribe(); - org.apache.pulsar.client.api.Producer producer = createProducer(client, topic1); + Producer producer = createProducer(client, topic1); byte[] content = new byte[1024]; for (int i = 0; i < numMsgs; i++) { producer.send(content); @@ -758,17 +912,17 @@ public void testConsumerBacklogEvictionTimeQuotaWithEmptyLedger() throws Excepti BacklogQuota.builder() .limitTime(TIME_TO_CHECK_BACKLOG_QUOTA) .retentionPolicy(BacklogQuota.RetentionPolicy.consumer_backlog_eviction) - .build(), BacklogQuota.BacklogQuotaType.message_age); + .build(), message_age); config.setPreciseTimeBasedBacklogQuotaCheck(true); @Cleanup - PulsarClient client = PulsarClient.builder().serviceUrl(adminUrl.toString()).statsInterval(0, TimeUnit.SECONDS) + PulsarClient client = PulsarClient.builder().serviceUrl(adminUrl.toString()).statsInterval(0, SECONDS) .build(); final String topic = "persistent://prop/ns-quota/topic4" + UUID.randomUUID(); final String subName = "c1"; Consumer consumer = client.newConsumer().topic(topic).subscriptionName(subName).subscribe(); - org.apache.pulsar.client.api.Producer producer = createProducer(client, topic); + Producer producer = createProducer(client, topic); producer.send(new byte[1024]); consumer.receive(); @@ -816,7 +970,7 @@ public void testConsumerBacklogEvictionWithAckSizeQuota() throws Exception { Consumer consumer1 = client.newConsumer().topic(topic1).subscriptionName(subName1).subscribe(); Consumer consumer2 = client.newConsumer().topic(topic1).subscriptionName(subName2).subscribe(); - org.apache.pulsar.client.api.Producer producer = createProducer(client, topic1); + Producer producer = createProducer(client, topic1); byte[] content = new byte[1024]; for (int i = 0; i < numMsgs; i++) { producer.send(content); @@ -840,7 +994,7 @@ public void testConsumerBacklogEvictionWithAckTimeQuotaPrecise() throws Exceptio BacklogQuota.builder() .limitTime(TIME_TO_CHECK_BACKLOG_QUOTA) .retentionPolicy(BacklogQuota.RetentionPolicy.consumer_backlog_eviction) - .build(), BacklogQuota.BacklogQuotaType.message_age); + .build(), message_age); config.setPreciseTimeBasedBacklogQuotaCheck(true); @Cleanup PulsarClient client = PulsarClient.builder().serviceUrl(adminUrl.toString()).build(); @@ -852,7 +1006,7 @@ public void testConsumerBacklogEvictionWithAckTimeQuotaPrecise() throws Exceptio Consumer consumer1 = client.newConsumer().topic(topic1).subscriptionName(subName1).subscribe(); Consumer consumer2 = client.newConsumer().topic(topic1).subscriptionName(subName2).subscribe(); - org.apache.pulsar.client.api.Producer producer = createProducer(client, topic1); + Producer producer = createProducer(client, topic1); byte[] content = new byte[1024]; for (int i = 0; i < numMsgs; i++) { @@ -890,7 +1044,7 @@ private Producer createProducer(PulsarClient client, String topic) throws PulsarClientException { return client.newProducer() .enableBatching(false) - .sendTimeout(2, TimeUnit.SECONDS) + .sendTimeout(2, SECONDS) .topic(topic) .create(); } @@ -909,7 +1063,7 @@ public void testConsumerBacklogEvictionWithAckTimeQuota() throws Exception { Consumer consumer1 = client.newConsumer().topic(topic1).subscriptionName(subName1).subscribe(); Consumer consumer2 = client.newConsumer().topic(topic1).subscriptionName(subName2).subscribe(); - org.apache.pulsar.client.api.Producer producer = createProducer(client, topic1); + Producer producer = createProducer(client, topic1); byte[] content = new byte[1024]; List> messagesToAcknowledge = new ArrayList<>(); @@ -950,7 +1104,7 @@ public void testConsumerBacklogEvictionWithAckTimeQuota() throws Exception { BacklogQuota.builder() .limitTime(2 * TIME_TO_CHECK_BACKLOG_QUOTA) .retentionPolicy(BacklogQuota.RetentionPolicy.consumer_backlog_eviction) - .build(), BacklogQuota.BacklogQuotaType.message_age); + .build(), message_age); Awaitility.await() .pollInterval(Duration.ofSeconds(1)) @@ -984,10 +1138,10 @@ public void testConcurrentAckAndEviction() throws Exception { final CountDownLatch counter = new CountDownLatch(2); final AtomicBoolean gotException = new AtomicBoolean(false); @Cleanup - PulsarClient client = PulsarClient.builder().serviceUrl(adminUrl.toString()).statsInterval(0, TimeUnit.SECONDS) + PulsarClient client = PulsarClient.builder().serviceUrl(adminUrl.toString()).statsInterval(0, SECONDS) .build(); @Cleanup - PulsarClient client2 = PulsarClient.builder().serviceUrl(adminUrl.toString()).statsInterval(0, TimeUnit.SECONDS) + PulsarClient client2 = PulsarClient.builder().serviceUrl(adminUrl.toString()).statsInterval(0, SECONDS) .build(); Consumer consumer1 = client2.newConsumer().topic(topic1).subscriptionName(subName1).subscribe(); Consumer consumer2 = client2.newConsumer().topic(topic1).subscriptionName(subName2).subscribe(); @@ -1027,7 +1181,7 @@ public void testConcurrentAckAndEviction() throws Exception { consumerThread.start(); // test hangs without timeout since there is nothing to consume due to eviction - counter.await(20, TimeUnit.SECONDS); + counter.await(20, SECONDS); assertFalse(gotException.get()); Thread.sleep((TIME_TO_CHECK_BACKLOG_QUOTA + 1) * 1000); rolloverStats(); @@ -1056,13 +1210,13 @@ public void testNoEviction() throws Exception { final AtomicBoolean gotException = new AtomicBoolean(false); @Cleanup final PulsarClient client = PulsarClient.builder().serviceUrl(adminUrl.toString()) - .statsInterval(0, TimeUnit.SECONDS).build(); + .statsInterval(0, SECONDS).build(); final Consumer consumer1 = client.newConsumer().topic(topic1).subscriptionName(subName1).subscribe(); final Consumer consumer2 = client.newConsumer().topic(topic1).subscriptionName(subName2).subscribe(); @Cleanup final PulsarClient client2 = PulsarClient.builder().serviceUrl(adminUrl.toString()) - .statsInterval(0, TimeUnit.SECONDS).build(); + .statsInterval(0, SECONDS).build(); Thread producerThread = new Thread(() -> { try { @@ -1120,16 +1274,16 @@ public void testEvictionMulti() throws Exception { final AtomicBoolean gotException = new AtomicBoolean(false); @Cleanup final PulsarClient client = PulsarClient.builder().serviceUrl(adminUrl.toString()) - .statsInterval(0, TimeUnit.SECONDS).build(); + .statsInterval(0, SECONDS).build(); final Consumer consumer1 = client.newConsumer().topic(topic1).subscriptionName(subName1).subscribe(); final Consumer consumer2 = client.newConsumer().topic(topic1).subscriptionName(subName2).subscribe(); @Cleanup final PulsarClient client3 = PulsarClient.builder().serviceUrl(adminUrl.toString()) - .statsInterval(0, TimeUnit.SECONDS).build(); + .statsInterval(0, SECONDS).build(); @Cleanup final PulsarClient client2 = PulsarClient.builder().serviceUrl(adminUrl.toString()) - .statsInterval(0, TimeUnit.SECONDS).build(); + .statsInterval(0, SECONDS).build(); Thread producerThread1 = new Thread(() -> { try { @@ -1193,7 +1347,7 @@ public void testEvictionMulti() throws Exception { producerThread2.start(); consumerThread1.start(); consumerThread2.start(); - counter.await(20, TimeUnit.SECONDS); + counter.await(20, SECONDS); assertFalse(gotException.get()); Thread.sleep((TIME_TO_CHECK_BACKLOG_QUOTA + 1) * 1000); rolloverStats(); @@ -1213,7 +1367,7 @@ public void testAheadProducerOnHold() throws Exception { .build()); @Cleanup final PulsarClient client = PulsarClient.builder().serviceUrl(adminUrl.toString()) - .statsInterval(0, TimeUnit.SECONDS).build(); + .statsInterval(0, SECONDS).build(); final String topic1 = "persistent://prop/quotahold/hold"; final String subName1 = "c1hold"; final int numMsgs = 10; @@ -1255,7 +1409,7 @@ public void testAheadProducerOnHoldTimeout() throws Exception { .build()); @Cleanup final PulsarClient client = PulsarClient.builder().serviceUrl(adminUrl.toString()) - .statsInterval(0, TimeUnit.SECONDS).build(); + .statsInterval(0, SECONDS).build(); final String topic1 = "persistent://prop/quotahold/holdtimeout"; final String subName1 = "c1holdtimeout"; boolean gotException = false; @@ -1293,7 +1447,7 @@ public void testProducerException() throws Exception { .build()); @Cleanup final PulsarClient client = PulsarClient.builder().serviceUrl(adminUrl.toString()) - .statsInterval(0, TimeUnit.SECONDS).build(); + .statsInterval(0, SECONDS).build(); final String topic1 = "persistent://prop/quotahold/except"; final String subName1 = "c1except"; boolean gotException = false; @@ -1338,7 +1492,7 @@ public void testProducerExceptionAndThenUnblockSizeQuota(boolean dedupTestSet) t .build()); @Cleanup final PulsarClient client = PulsarClient.builder().serviceUrl(adminUrl.toString()) - .statsInterval(0, TimeUnit.SECONDS).build(); + .statsInterval(0, SECONDS).build(); final String topic1 = "persistent://prop/quotahold/exceptandunblock"; final String subName1 = "c1except"; boolean gotException = false; @@ -1422,11 +1576,11 @@ public void testProducerExceptionAndThenUnblockTimeQuotaPrecise() throws Excepti BacklogQuota.builder() .limitTime(TIME_TO_CHECK_BACKLOG_QUOTA) .retentionPolicy(BacklogQuota.RetentionPolicy.producer_exception) - .build(), BacklogQuota.BacklogQuotaType.message_age); + .build(), message_age); config.setPreciseTimeBasedBacklogQuotaCheck(true); @Cleanup final PulsarClient client = PulsarClient.builder().serviceUrl(adminUrl.toString()) - .statsInterval(0, TimeUnit.SECONDS).build(); + .statsInterval(0, SECONDS).build(); final String topic1 = "persistent://prop/quotahold/exceptandunblock2"; final String subName1 = "c1except"; boolean gotException = false; @@ -1488,10 +1642,10 @@ public void testProducerExceptionAndThenUnblockTimeQuota() throws Exception { BacklogQuota.builder() .limitTime(TIME_TO_CHECK_BACKLOG_QUOTA) .retentionPolicy(BacklogQuota.RetentionPolicy.producer_exception) - .build(), BacklogQuota.BacklogQuotaType.message_age); + .build(), message_age); @Cleanup final PulsarClient client = PulsarClient.builder().serviceUrl(adminUrl.toString()) - .statsInterval(0, TimeUnit.SECONDS).build(); + .statsInterval(0, SECONDS).build(); final String topic1 = "persistent://prop/quotahold/exceptandunblock2"; final String subName1 = "c1except"; boolean gotException = false; @@ -1565,7 +1719,7 @@ public void testBacklogQuotaInGB(boolean backlogQuotaSizeGB) throws Exception { admin = PulsarAdmin.builder().serviceHttpUrl(adminUrl.toString()).build(); @Cleanup - PulsarClient client = PulsarClient.builder().serviceUrl(pulsar.getBrokerServiceUrl()).statsInterval(0, TimeUnit.SECONDS) + PulsarClient client = PulsarClient.builder().serviceUrl(pulsar.getBrokerServiceUrl()).statsInterval(0, SECONDS) .build(); final String topic1 = "persistent://prop/ns-quota/topic2" + UUID.randomUUID(); @@ -1575,7 +1729,7 @@ public void testBacklogQuotaInGB(boolean backlogQuotaSizeGB) throws Exception { Consumer consumer1 = client.newConsumer().topic(topic1).subscriptionName(subName1).subscribe(); Consumer consumer2 = client.newConsumer().topic(topic1).subscriptionName(subName2).subscribe(); - org.apache.pulsar.client.api.Producer producer = createProducer(client, topic1); + Producer producer = createProducer(client, topic1); byte[] content = new byte[1024]; for (int i = 0; i < numMsgs; i++) { producer.send(content); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/SubscriptionSeekTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/SubscriptionSeekTest.java index b11946069c9dd..fd08f284bbf99 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/SubscriptionSeekTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/SubscriptionSeekTest.java @@ -18,6 +18,7 @@ */ package org.apache.pulsar.broker.service; +import static org.assertj.core.api.Assertions.assertThat; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertFalse; import static org.testng.Assert.assertNotNull; @@ -128,6 +129,29 @@ public void testSeek() throws Exception { assertEquals(sub.getNumberOfEntriesInBacklog(false), 0); } + @Test + public void testSeekIsByReceive() throws PulsarClientException { + final String topicName = "persistent://prop/use/ns-abc/testSeek"; + + Producer producer = pulsarClient.newProducer().topic(topicName).create(); + + String subscriptionName = "my-subscription"; + org.apache.pulsar.client.api.Consumer consumer = pulsarClient.newConsumer().topic(topicName) + .subscriptionName(subscriptionName) + .subscribe(); + + List messageIds = new ArrayList<>(); + for (int i = 0; i < 10; i++) { + String message = "my-message-" + i; + MessageId msgId = producer.send(message.getBytes()); + messageIds.add(msgId); + } + + consumer.seek(messageIds.get(5)); + Message message = consumer.receive(); + assertThat(message.getMessageId()).isEqualTo(messageIds.get(6)); + } + @Test public void testSeekForBatch() throws Exception { final String topicName = "persistent://prop/use/ns-abcd/testSeekForBatch"; diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/BucketDelayedDeliveryTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/BucketDelayedDeliveryTest.java index 40f271e0940e8..8be0aa4bc7dbd 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/BucketDelayedDeliveryTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/BucketDelayedDeliveryTest.java @@ -19,6 +19,8 @@ package org.apache.pulsar.broker.service.persistent; import static org.apache.bookkeeper.mledger.impl.ManagedCursorImpl.CURSOR_INTERNAL_PROPERTY_PREFIX; +import static org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient.Metric; +import static org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient.parseMetrics; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertNotNull; import static org.testng.Assert.assertTrue; @@ -41,7 +43,6 @@ import org.apache.pulsar.broker.BrokerTestUtil; import org.apache.pulsar.broker.delayed.BucketDelayedDeliveryTrackerFactory; import org.apache.pulsar.broker.service.Dispatcher; -import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient; import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsGenerator; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.MessageId; @@ -219,9 +220,9 @@ public void testBucketDelayedIndexMetrics() throws Exception { ByteArrayOutputStream output = new ByteArrayOutputStream(); PrometheusMetricsGenerator.generate(pulsar, true, true, true, output); String metricsStr = output.toString(StandardCharsets.UTF_8); - Multimap metricsMap = PrometheusMetricsClient.parseMetrics(metricsStr); + Multimap metricsMap = parseMetrics(metricsStr); - List bucketsMetrics = + List bucketsMetrics = metricsMap.get("pulsar_delayed_message_index_bucket_total").stream() .filter(metric -> metric.tags.get("topic").equals(topic)).toList(); MutableInt bucketsSum = new MutableInt(); @@ -230,12 +231,12 @@ public void testBucketDelayedIndexMetrics() throws Exception { bucketsSum.add(metric.value); }); assertEquals(6, bucketsSum.intValue()); - Optional bucketsTopicMetric = + Optional bucketsTopicMetric = bucketsMetrics.stream().filter(metric -> !metric.tags.containsKey("subscription")).findFirst(); assertTrue(bucketsTopicMetric.isPresent()); assertEquals(bucketsSum.intValue(), bucketsTopicMetric.get().value); - List loadedIndexMetrics = + List loadedIndexMetrics = metricsMap.get("pulsar_delayed_message_index_loaded").stream() .filter(metric -> metric.tags.get("topic").equals(topic)).toList(); MutableInt loadedIndexSum = new MutableInt(); @@ -244,12 +245,12 @@ public void testBucketDelayedIndexMetrics() throws Exception { loadedIndexSum.add(metric.value); }).count(); assertEquals(2, count); - Optional loadedIndexTopicMetrics = + Optional loadedIndexTopicMetrics = bucketsMetrics.stream().filter(metric -> !metric.tags.containsKey("subscription")).findFirst(); assertTrue(loadedIndexTopicMetrics.isPresent()); assertEquals(loadedIndexSum.intValue(), loadedIndexTopicMetrics.get().value); - List snapshotSizeBytesMetrics = + List snapshotSizeBytesMetrics = metricsMap.get("pulsar_delayed_message_index_bucket_snapshot_size_bytes").stream() .filter(metric -> metric.tags.get("topic").equals(topic)).toList(); MutableInt snapshotSizeBytesSum = new MutableInt(); @@ -259,12 +260,12 @@ public void testBucketDelayedIndexMetrics() throws Exception { snapshotSizeBytesSum.add(metric.value); }).count(); assertEquals(2, count); - Optional snapshotSizeBytesTopicMetrics = + Optional snapshotSizeBytesTopicMetrics = snapshotSizeBytesMetrics.stream().filter(metric -> !metric.tags.containsKey("subscription")).findFirst(); assertTrue(snapshotSizeBytesTopicMetrics.isPresent()); assertEquals(snapshotSizeBytesSum.intValue(), snapshotSizeBytesTopicMetrics.get().value); - List opCountMetrics = + List opCountMetrics = metricsMap.get("pulsar_delayed_message_index_bucket_op_count").stream() .filter(metric -> metric.tags.get("topic").equals(topic)).toList(); MutableInt opCountMetricsSum = new MutableInt(); @@ -276,14 +277,14 @@ public void testBucketDelayedIndexMetrics() throws Exception { opCountMetricsSum.add(metric.value); }).count(); assertEquals(2, count); - Optional opCountTopicMetrics = + Optional opCountTopicMetrics = opCountMetrics.stream() .filter(metric -> metric.tags.get("state").equals("succeed") && metric.tags.get("type") .equals("create") && !metric.tags.containsKey("subscription")).findFirst(); assertTrue(opCountTopicMetrics.isPresent()); assertEquals(opCountMetricsSum.intValue(), opCountTopicMetrics.get().value); - List opLatencyMetrics = + List opLatencyMetrics = metricsMap.get("pulsar_delayed_message_index_bucket_op_latency_ms").stream() .filter(metric -> metric.tags.get("topic").equals(topic)).toList(); MutableInt opLatencyMetricsSum = new MutableInt(); @@ -295,7 +296,7 @@ public void testBucketDelayedIndexMetrics() throws Exception { opLatencyMetricsSum.add(metric.value); }).count(); assertTrue(count >= 2); - Optional opLatencyTopicMetrics = + Optional opLatencyTopicMetrics = opCountMetrics.stream() .filter(metric -> metric.tags.get("type").equals("create") && !metric.tags.containsKey("subscription")).findFirst(); @@ -304,9 +305,9 @@ public void testBucketDelayedIndexMetrics() throws Exception { ByteArrayOutputStream namespaceOutput = new ByteArrayOutputStream(); PrometheusMetricsGenerator.generate(pulsar, false, true, true, namespaceOutput); - Multimap namespaceMetricsMap = PrometheusMetricsClient.parseMetrics(namespaceOutput.toString(StandardCharsets.UTF_8)); + Multimap namespaceMetricsMap = parseMetrics(namespaceOutput.toString(StandardCharsets.UTF_8)); - Optional namespaceMetric = + Optional namespaceMetric = namespaceMetricsMap.get("pulsar_delayed_message_index_bucket_total").stream().findFirst(); assertTrue(namespaceMetric.isPresent()); assertEquals(6, namespaceMetric.get().value); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentTopicTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentTopicTest.java index 0e2ab7e58002f..4eb2aa15fa292 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentTopicTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentTopicTest.java @@ -18,6 +18,8 @@ */ package org.apache.pulsar.broker.service.persistent; +import static org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient.Metric; +import static org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient.parseMetrics; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyInt; import static org.mockito.Mockito.doAnswer; @@ -62,7 +64,6 @@ import org.apache.pulsar.broker.service.BrokerService; import org.apache.pulsar.broker.service.BrokerTestBase; import org.apache.pulsar.broker.service.TopicPoliciesService; -import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient; import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsGenerator; import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.api.Consumer; @@ -356,14 +357,14 @@ public void testDelayedDeliveryTrackerMemoryUsageMetric(String topic, boolean ex PrometheusMetricsGenerator.generate(pulsar, exposeTopicLevelMetrics, true, true, output); String metricsStr = output.toString(StandardCharsets.UTF_8); - Multimap metricsMap = PrometheusMetricsClient.parseMetrics(metricsStr); - Collection metrics = metricsMap.get("pulsar_delayed_message_index_size_bytes"); + Multimap metricsMap = parseMetrics(metricsStr); + Collection metrics = metricsMap.get("pulsar_delayed_message_index_size_bytes"); Assert.assertTrue(metrics.size() > 0); int topicLevelNum = 0; int namespaceLevelNum = 0; int subscriptionLevelNum = 0; - for (PrometheusMetricsClient.Metric metric : metrics) { + for (Metric metric : metrics) { if (exposeTopicLevelMetrics && metric.tags.get("topic").equals(topic)) { Assert.assertTrue(metric.value > 0); topicLevelNum++; diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/schema/SchemaServiceTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/schema/SchemaServiceTest.java index 97bf6e172ddf3..a520b8c241bd1 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/schema/SchemaServiceTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/schema/SchemaServiceTest.java @@ -18,6 +18,8 @@ */ package org.apache.pulsar.broker.service.schema; +import static org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient.Metric; +import static org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient.parseMetrics; import static org.testng.Assert.assertThrows; import static org.testng.Assert.fail; import static org.testng.AssertJUnit.assertEquals; @@ -45,7 +47,6 @@ import org.apache.pulsar.broker.PulsarServerException; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; import org.apache.pulsar.broker.service.schema.SchemaRegistry.SchemaAndMetadata; -import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient; import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsGenerator; import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.impl.schema.KeyValueSchemaInfo; @@ -123,29 +124,29 @@ public void testSchemaRegistryMetrics() throws Exception { PrometheusMetricsGenerator.generate(pulsar, false, false, false, output); output.flush(); String metricsStr = output.toString(StandardCharsets.UTF_8); - Multimap metrics = PrometheusMetricsClient.parseMetrics(metricsStr); + Multimap metrics = parseMetrics(metricsStr); - Collection delMetrics = metrics.get("pulsar_schema_del_ops_failed_total"); + Collection delMetrics = metrics.get("pulsar_schema_del_ops_failed_total"); Assert.assertEquals(delMetrics.size(), 0); - Collection getMetrics = metrics.get("pulsar_schema_get_ops_failed_total"); + Collection getMetrics = metrics.get("pulsar_schema_get_ops_failed_total"); Assert.assertEquals(getMetrics.size(), 0); - Collection putMetrics = metrics.get("pulsar_schema_put_ops_failed_total"); + Collection putMetrics = metrics.get("pulsar_schema_put_ops_failed_total"); Assert.assertEquals(putMetrics.size(), 0); - Collection deleteLatency = metrics.get("pulsar_schema_del_ops_latency_count"); - for (PrometheusMetricsClient.Metric metric : deleteLatency) { + Collection deleteLatency = metrics.get("pulsar_schema_del_ops_latency_count"); + for (Metric metric : deleteLatency) { Assert.assertEquals(metric.tags.get("namespace"), namespace); Assert.assertTrue(metric.value > 0); } - Collection getLatency = metrics.get("pulsar_schema_get_ops_latency_count"); - for (PrometheusMetricsClient.Metric metric : getLatency) { + Collection getLatency = metrics.get("pulsar_schema_get_ops_latency_count"); + for (Metric metric : getLatency) { Assert.assertEquals(metric.tags.get("namespace"), namespace); Assert.assertTrue(metric.value > 0); } - Collection putLatency = metrics.get("pulsar_schema_put_ops_latency_count"); - for (PrometheusMetricsClient.Metric metric : putLatency) { + Collection putLatency = metrics.get("pulsar_schema_put_ops_latency_count"); + for (Metric metric : putLatency) { Assert.assertEquals(metric.tags.get("namespace"), namespace); Assert.assertTrue(metric.value > 0); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/ConsumerStatsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/ConsumerStatsTest.java index d04fcffd15bc8..eb4500c13667a 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/ConsumerStatsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/ConsumerStatsTest.java @@ -19,6 +19,8 @@ package org.apache.pulsar.broker.stats; import static org.apache.pulsar.broker.BrokerTestUtil.spyWithClassAndConstructorArgs; +import static org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient.Metric; +import static org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient.parseMetrics; import static org.mockito.Mockito.mock; import static org.testng.Assert.assertNotEquals; import static org.testng.AssertJUnit.assertEquals; @@ -50,7 +52,6 @@ import org.apache.pulsar.broker.service.plugin.EntryFilter; import org.apache.pulsar.broker.service.plugin.EntryFilterProducerTest; import org.apache.pulsar.broker.service.plugin.EntryFilterWithClassLoader; -import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient; import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsGenerator; import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.api.Consumer; @@ -337,11 +338,11 @@ private void testMessageAckRateMetric(String topicName, boolean exposeTopicLevel PrometheusMetricsGenerator.generate(pulsar, exposeTopicLevelMetrics, true, true, output); String metricStr = output.toString(StandardCharsets.UTF_8); - Multimap metricsMap = PrometheusMetricsClient.parseMetrics(metricStr); - Collection ackRateMetric = metricsMap.get("pulsar_consumer_msg_ack_rate"); + Multimap metricsMap = parseMetrics(metricStr); + Collection ackRateMetric = metricsMap.get("pulsar_consumer_msg_ack_rate"); String rateOutMetricName = exposeTopicLevelMetrics ? "pulsar_consumer_msg_rate_out" : "pulsar_rate_out"; - Collection rateOutMetric = metricsMap.get(rateOutMetricName); + Collection rateOutMetric = metricsMap.get(rateOutMetricName); Assert.assertTrue(ackRateMetric.size() > 0); Assert.assertTrue(rateOutMetric.size() > 0); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/MetadataStoreStatsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/MetadataStoreStatsTest.java index 71bb2c4fe07a2..15f41365da8d1 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/MetadataStoreStatsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/MetadataStoreStatsTest.java @@ -18,6 +18,8 @@ */ package org.apache.pulsar.broker.stats; +import static org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient.Metric; +import static org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient.parseMetrics; import com.google.common.collect.Multimap; import java.io.ByteArrayOutputStream; import java.util.Collection; @@ -31,7 +33,6 @@ import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.authentication.AuthenticationProviderToken; import org.apache.pulsar.broker.service.BrokerTestBase; -import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient; import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsGenerator; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.Message; @@ -102,12 +103,12 @@ public void testMetadataStoreStats() throws Exception { ByteArrayOutputStream output = new ByteArrayOutputStream(); PrometheusMetricsGenerator.generate(pulsar, false, false, false, false, output); String metricsStr = output.toString(); - Multimap metricsMap = PrometheusMetricsClient.parseMetrics(metricsStr); + Multimap metricsMap = parseMetrics(metricsStr); String metricsDebugMessage = "Assertion failed with metrics:\n" + metricsStr + "\n"; - Collection opsLatency = metricsMap.get("pulsar_metadata_store_ops_latency_ms" + "_sum"); - Collection putBytes = metricsMap.get("pulsar_metadata_store_put_bytes" + "_total"); + Collection opsLatency = metricsMap.get("pulsar_metadata_store_ops_latency_ms" + "_sum"); + Collection putBytes = metricsMap.get("pulsar_metadata_store_put_bytes" + "_total"); Assert.assertTrue(opsLatency.size() > 1, metricsDebugMessage); Assert.assertTrue(putBytes.size() > 1, metricsDebugMessage); @@ -117,7 +118,7 @@ public void testMetadataStoreStats() throws Exception { expectedMetadataStoreName.add(MetadataStoreConfig.CONFIGURATION_METADATA_STORE); AtomicInteger matchCount = new AtomicInteger(0); - for (PrometheusMetricsClient.Metric m : opsLatency) { + for (Metric m : opsLatency) { Assert.assertEquals(m.tags.get("cluster"), "test", metricsDebugMessage); String metadataStoreName = m.tags.get("name"); if (!isExpectedLabel(metadataStoreName, expectedMetadataStoreName, matchCount)) { @@ -151,7 +152,7 @@ public void testMetadataStoreStats() throws Exception { Assert.assertEquals(matchCount.get(), expectedMetadataStoreName.size() * 6); matchCount = new AtomicInteger(0); - for (PrometheusMetricsClient.Metric m : putBytes) { + for (Metric m : putBytes) { Assert.assertEquals(m.tags.get("cluster"), "test", metricsDebugMessage); String metadataStoreName = m.tags.get("name"); if (!isExpectedLabel(metadataStoreName, expectedMetadataStoreName, matchCount)) { @@ -192,12 +193,12 @@ public void testBatchMetadataStoreMetrics() throws Exception { ByteArrayOutputStream output = new ByteArrayOutputStream(); PrometheusMetricsGenerator.generate(pulsar, false, false, false, false, output); String metricsStr = output.toString(); - Multimap metricsMap = PrometheusMetricsClient.parseMetrics(metricsStr); + Multimap metricsMap = parseMetrics(metricsStr); - Collection executorQueueSize = metricsMap.get("pulsar_batch_metadata_store_executor_queue_size"); - Collection opsWaiting = metricsMap.get("pulsar_batch_metadata_store_queue_wait_time_ms" + "_sum"); - Collection batchExecuteTime = metricsMap.get("pulsar_batch_metadata_store_batch_execute_time_ms" + "_sum"); - Collection opsPerBatch = metricsMap.get("pulsar_batch_metadata_store_batch_size" + "_sum"); + Collection executorQueueSize = metricsMap.get("pulsar_batch_metadata_store_executor_queue_size"); + Collection opsWaiting = metricsMap.get("pulsar_batch_metadata_store_queue_wait_time_ms" + "_sum"); + Collection batchExecuteTime = metricsMap.get("pulsar_batch_metadata_store_batch_execute_time_ms" + "_sum"); + Collection opsPerBatch = metricsMap.get("pulsar_batch_metadata_store_batch_size" + "_sum"); String metricsDebugMessage = "Assertion failed with metrics:\n" + metricsStr + "\n"; @@ -211,7 +212,7 @@ public void testBatchMetadataStoreMetrics() throws Exception { expectedMetadataStoreName.add(MetadataStoreConfig.CONFIGURATION_METADATA_STORE); AtomicInteger matchCount = new AtomicInteger(0); - for (PrometheusMetricsClient.Metric m : executorQueueSize) { + for (Metric m : executorQueueSize) { Assert.assertEquals(m.tags.get("cluster"), "test", metricsDebugMessage); String metadataStoreName = m.tags.get("name"); if (isExpectedLabel(metadataStoreName, expectedMetadataStoreName, matchCount)) { @@ -222,7 +223,7 @@ public void testBatchMetadataStoreMetrics() throws Exception { Assert.assertEquals(matchCount.get(), expectedMetadataStoreName.size()); matchCount = new AtomicInteger(0); - for (PrometheusMetricsClient.Metric m : opsWaiting) { + for (Metric m : opsWaiting) { Assert.assertEquals(m.tags.get("cluster"), "test", metricsDebugMessage); String metadataStoreName = m.tags.get("name"); if (isExpectedLabel(metadataStoreName, expectedMetadataStoreName, matchCount)) { @@ -233,7 +234,7 @@ public void testBatchMetadataStoreMetrics() throws Exception { Assert.assertEquals(matchCount.get(), expectedMetadataStoreName.size()); matchCount = new AtomicInteger(0); - for (PrometheusMetricsClient.Metric m : batchExecuteTime) { + for (Metric m : batchExecuteTime) { Assert.assertEquals(m.tags.get("cluster"), "test", metricsDebugMessage); String metadataStoreName = m.tags.get("name"); if (isExpectedLabel(metadataStoreName, expectedMetadataStoreName, matchCount)) { @@ -244,7 +245,7 @@ public void testBatchMetadataStoreMetrics() throws Exception { Assert.assertEquals(matchCount.get(), expectedMetadataStoreName.size()); matchCount = new AtomicInteger(0); - for (PrometheusMetricsClient.Metric m : opsPerBatch) { + for (Metric m : opsPerBatch) { Assert.assertEquals(m.tags.get("cluster"), "test", metricsDebugMessage); String metadataStoreName = m.tags.get("name"); if (isExpectedLabel(metadataStoreName, expectedMetadataStoreName, matchCount)) { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/PrometheusMetricsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/PrometheusMetricsTest.java index e7312431e2927..d9cdd402c337e 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/PrometheusMetricsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/PrometheusMetricsTest.java @@ -19,6 +19,8 @@ package org.apache.pulsar.broker.stats; import static com.google.common.base.Preconditions.checkArgument; +import static org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient.Metric; +import static org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient.parseMetrics; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertNull; import static org.testng.Assert.assertTrue; @@ -70,7 +72,6 @@ import org.apache.pulsar.broker.service.persistent.PersistentMessageExpiryMonitor; import org.apache.pulsar.broker.service.persistent.PersistentSubscription; import org.apache.pulsar.broker.service.persistent.PersistentTopic; -import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient; import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsGenerator; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.MessageRoutingMode; @@ -167,7 +168,7 @@ private void checkPublishRateLimitedTimes(boolean preciseRateLimit) throws Excep ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut); String metricsStr = statsOut.toString(); - Multimap metrics = PrometheusMetricsClient.parseMetrics(metricsStr); + Multimap metrics = parseMetrics(metricsStr); assertTrue(metrics.containsKey("pulsar_publish_rate_limit_times")); metrics.get("pulsar_publish_rate_limit_times").forEach(item -> { if (ns1.equals(item.tags.get("namespace"))) { @@ -203,7 +204,7 @@ private void checkPublishRateLimitedTimes(boolean preciseRateLimit) throws Excep ByteArrayOutputStream statsOut2 = new ByteArrayOutputStream(); PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut2); String metricsStr2 = statsOut2.toString(); - Multimap metrics2 = PrometheusMetricsClient.parseMetrics(metricsStr2); + Multimap metrics2 = parseMetrics(metricsStr2); assertTrue(metrics2.containsKey("pulsar_publish_rate_limit_times")); metrics2.get("pulsar_publish_rate_limit_times").forEach(item -> { if (ns1.equals(item.tags.get("namespace"))) { @@ -235,8 +236,8 @@ public void testMetricsTopicCount() throws Exception { ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut); String metricsStr = statsOut.toString(); - Multimap metrics = PrometheusMetricsClient.parseMetrics(metricsStr); - Collection metric = metrics.get("pulsar_topics_count"); + Multimap metrics = parseMetrics(metricsStr); + Collection metric = metrics.get("pulsar_topics_count"); metric.forEach(item -> { if (ns1.equals(item.tags.get("namespace"))) { assertEquals(item.value, 6.0); @@ -245,12 +246,12 @@ public void testMetricsTopicCount() throws Exception { assertEquals(item.value, 3.0); } }); - Collection pulsarTopicLoadTimesMetrics = metrics.get("pulsar_topic_load_times"); - Collection pulsarTopicLoadTimesCountMetrics = metrics.get("pulsar_topic_load_times_count"); + Collection pulsarTopicLoadTimesMetrics = metrics.get("pulsar_topic_load_times"); + Collection pulsarTopicLoadTimesCountMetrics = metrics.get("pulsar_topic_load_times_count"); assertEquals(pulsarTopicLoadTimesMetrics.size(), 6); assertEquals(pulsarTopicLoadTimesCountMetrics.size(), 1); - Collection topicLoadTimeP999Metrics = metrics.get("pulsar_topic_load_time_99_9_percentile_ms"); - Collection topicLoadTimeFailedCountMetrics = metrics.get("pulsar_topic_load_failed_count"); + Collection topicLoadTimeP999Metrics = metrics.get("pulsar_topic_load_time_99_9_percentile_ms"); + Collection topicLoadTimeFailedCountMetrics = metrics.get("pulsar_topic_load_failed_count"); assertEquals(topicLoadTimeP999Metrics.size(), 1); assertEquals(topicLoadTimeFailedCountMetrics.size(), 1); } @@ -272,10 +273,10 @@ public void testMetricsAvgMsgSize2() throws Exception { ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); PrometheusMetricsGenerator.generate(pulsar, true, false, true, statsOut); String metricsStr = statsOut.toString(); - Multimap metrics = PrometheusMetricsClient.parseMetrics(metricsStr); + Multimap metrics = parseMetrics(metricsStr); assertTrue(metrics.containsKey("pulsar_average_msg_size")); assertEquals(metrics.get("pulsar_average_msg_size").size(), 1); - Collection avgMsgSizes = metrics.get("pulsar_average_msg_size"); + Collection avgMsgSizes = metrics.get("pulsar_average_msg_size"); avgMsgSizes.forEach(item -> { if (ns1.equals(item.tags.get("namespace"))) { assertEquals(item.value, 10); @@ -315,50 +316,50 @@ public void testPerTopicStats() throws Exception { ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut); String metricsStr = statsOut.toString(); - Multimap metrics = PrometheusMetricsClient.parseMetrics(metricsStr); + Multimap metrics = parseMetrics(metricsStr); metrics.entries().forEach(e -> { System.out.println(e.getKey() + ": " + e.getValue()); }); // There should be 2 metrics with different tags for each topic - List cm = (List) metrics.get("pulsar_storage_write_latency_le_1"); + List cm = (List) metrics.get("pulsar_storage_write_latency_le_1"); assertEquals(cm.size(), 2); assertEquals(cm.get(0).tags.get("topic"), "persistent://my-property/use/my-ns/my-topic2"); assertEquals(cm.get(0).tags.get("namespace"), "my-property/use/my-ns"); assertEquals(cm.get(1).tags.get("topic"), "persistent://my-property/use/my-ns/my-topic1"); assertEquals(cm.get(1).tags.get("namespace"), "my-property/use/my-ns"); - cm = (List) metrics.get("pulsar_producers_count"); + cm = (List) metrics.get("pulsar_producers_count"); assertEquals(cm.size(), 2); assertEquals(cm.get(1).tags.get("topic"), "persistent://my-property/use/my-ns/my-topic1"); assertEquals(cm.get(1).tags.get("namespace"), "my-property/use/my-ns"); assertEquals(cm.get(1).tags.get("topic"), "persistent://my-property/use/my-ns/my-topic1"); assertEquals(cm.get(1).tags.get("namespace"), "my-property/use/my-ns"); - cm = (List) metrics.get("pulsar_topic_load_times_count"); + cm = (List) metrics.get("pulsar_topic_load_times_count"); assertEquals(cm.size(), 1); assertEquals(cm.get(0).tags.get("cluster"), "test"); - cm = (List) metrics.get("topic_load_failed_total"); + cm = (List) metrics.get("topic_load_failed_total"); assertEquals(cm.size(), 1); assertEquals(cm.get(0).tags.get("cluster"), "test"); - cm = (List) metrics.get("pulsar_in_bytes_total"); + cm = (List) metrics.get("pulsar_in_bytes_total"); assertEquals(cm.size(), 2); assertEquals(cm.get(0).tags.get("topic"), "persistent://my-property/use/my-ns/my-topic2"); assertEquals(cm.get(0).tags.get("namespace"), "my-property/use/my-ns"); assertEquals(cm.get(1).tags.get("topic"), "persistent://my-property/use/my-ns/my-topic1"); assertEquals(cm.get(1).tags.get("namespace"), "my-property/use/my-ns"); - cm = (List) metrics.get("pulsar_in_messages_total"); + cm = (List) metrics.get("pulsar_in_messages_total"); assertEquals(cm.size(), 2); assertEquals(cm.get(0).tags.get("topic"), "persistent://my-property/use/my-ns/my-topic2"); assertEquals(cm.get(0).tags.get("namespace"), "my-property/use/my-ns"); assertEquals(cm.get(1).tags.get("topic"), "persistent://my-property/use/my-ns/my-topic1"); assertEquals(cm.get(1).tags.get("namespace"), "my-property/use/my-ns"); - cm = (List) metrics.get("pulsar_out_bytes_total"); + cm = (List) metrics.get("pulsar_out_bytes_total"); assertEquals(cm.size(), 2); assertEquals(cm.get(0).tags.get("topic"), "persistent://my-property/use/my-ns/my-topic2"); assertEquals(cm.get(0).tags.get("namespace"), "my-property/use/my-ns"); @@ -367,7 +368,7 @@ public void testPerTopicStats() throws Exception { assertEquals(cm.get(1).tags.get("namespace"), "my-property/use/my-ns"); assertEquals(cm.get(1).tags.get("subscription"), "test"); - cm = (List) metrics.get("pulsar_out_messages_total"); + cm = (List) metrics.get("pulsar_out_messages_total"); assertEquals(cm.size(), 2); assertEquals(cm.get(0).tags.get("topic"), "persistent://my-property/use/my-ns/my-topic2"); assertEquals(cm.get(0).tags.get("namespace"), "my-property/use/my-ns"); @@ -413,9 +414,9 @@ public void testPerBrokerStats() throws Exception { ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut); String metricsStr = statsOut.toString(); - Multimap metrics = PrometheusMetricsClient.parseMetrics(metricsStr); + Multimap metrics = parseMetrics(metricsStr); - Collection brokerMetrics = metrics.get("pulsar_broker_topics_count"); + Collection brokerMetrics = metrics.get("pulsar_broker_topics_count"); assertEquals(brokerMetrics.size(), 1); assertEquals(brokerMetrics.stream().toList().get(0).tags.get("cluster"), "test"); @@ -522,32 +523,32 @@ public void testPerTopicStatsReconnect() throws Exception { ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut); String metricsStr = statsOut.toString(); - Multimap metrics = PrometheusMetricsClient.parseMetrics(metricsStr); + Multimap metrics = parseMetrics(metricsStr); metrics.entries().forEach(e -> { System.out.println(e.getKey() + ": " + e.getValue()); }); - List cm = (List) metrics.get("pulsar_in_bytes_total"); + List cm = (List) metrics.get("pulsar_in_bytes_total"); assertEquals(cm.size(), 1); assertEquals(cm.get(0).value, (messageSizeBytes * messages * 2)); assertEquals(cm.get(0).tags.get("topic"), "persistent://my-property/use/my-ns/my-topic1"); assertEquals(cm.get(0).tags.get("namespace"), "my-property/use/my-ns"); - cm = (List) metrics.get("pulsar_in_messages_total"); + cm = (List) metrics.get("pulsar_in_messages_total"); assertEquals(cm.size(), 1); assertEquals(cm.get(0).value, (messages * 2)); assertEquals(cm.get(0).tags.get("topic"), "persistent://my-property/use/my-ns/my-topic1"); assertEquals(cm.get(0).tags.get("namespace"), "my-property/use/my-ns"); - cm = (List) metrics.get("pulsar_out_bytes_total"); + cm = (List) metrics.get("pulsar_out_bytes_total"); assertEquals(cm.size(), 1); assertEquals(cm.get(0).value, (messageSizeBytes * messages * 2)); assertEquals(cm.get(0).tags.get("topic"), "persistent://my-property/use/my-ns/my-topic1"); assertEquals(cm.get(0).tags.get("namespace"), "my-property/use/my-ns"); assertEquals(cm.get(0).tags.get("subscription"), "test"); - cm = (List) metrics.get("pulsar_out_messages_total"); + cm = (List) metrics.get("pulsar_out_messages_total"); assertEquals(cm.size(), 1); assertEquals(cm.get(0).value, (messages * 2)); assertEquals(cm.get(0).tags.get("topic"), "persistent://my-property/use/my-ns/my-topic1"); @@ -600,11 +601,11 @@ public void testStorageReadCacheMissesRate(boolean cacheEnable) throws Exception ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut); String metricsStr = statsOut.toString(); - Multimap metrics = PrometheusMetricsClient.parseMetrics(metricsStr); + Multimap metrics = parseMetrics(metricsStr); metrics.entries().forEach(e -> System.out.println(e.getKey() + ": " + e.getValue())); - List cm = (List) metrics.get("pulsar_storage_read_cache_misses_rate"); + List cm = (List) metrics.get("pulsar_storage_read_cache_misses_rate"); assertEquals(cm.size(), 1); if (cacheEnable) { assertEquals(cm.get(0).value, 1.0); @@ -616,7 +617,7 @@ public void testStorageReadCacheMissesRate(boolean cacheEnable) throws Exception assertEquals(cm.get(0).tags.get("namespace"), ns); assertEquals(cm.get(0).tags.get("cluster"), "test"); - List brokerMetric = (List) metrics.get("pulsar_broker_storage_read_cache_misses_rate"); + List brokerMetric = (List) metrics.get("pulsar_broker_storage_read_cache_misses_rate"); assertEquals(brokerMetric.size(), 1); if (cacheEnable) { assertEquals(brokerMetric.get(0).value, 1.0); @@ -632,11 +633,11 @@ public void testStorageReadCacheMissesRate(boolean cacheEnable) throws Exception ByteArrayOutputStream statsOut2 = new ByteArrayOutputStream(); PrometheusMetricsGenerator.generate(pulsar, false, false, false, statsOut2); String metricsStr2 = statsOut2.toString(); - Multimap metrics2 = PrometheusMetricsClient.parseMetrics(metricsStr2); + Multimap metrics2 = parseMetrics(metricsStr2); metrics2.entries().forEach(e -> System.out.println(e.getKey() + ": " + e.getValue())); - List cm2 = (List) metrics2.get("pulsar_storage_read_cache_misses_rate"); + List cm2 = (List) metrics2.get("pulsar_storage_read_cache_misses_rate"); assertEquals(cm2.size(), 1); if (cacheEnable) { assertEquals(cm2.get(0).value, 1.0); @@ -648,7 +649,7 @@ public void testStorageReadCacheMissesRate(boolean cacheEnable) throws Exception assertEquals(cm2.get(0).tags.get("namespace"), ns); assertEquals(cm2.get(0).tags.get("cluster"), "test"); - List brokerMetric2 = (List) metrics.get("pulsar_broker_storage_read_cache_misses_rate"); + List brokerMetric2 = (List) metrics.get("pulsar_broker_storage_read_cache_misses_rate"); assertEquals(brokerMetric2.size(), 1); if (cacheEnable) { assertEquals(brokerMetric2.get(0).value, 1.0); @@ -660,7 +661,7 @@ public void testStorageReadCacheMissesRate(boolean cacheEnable) throws Exception assertNull(brokerMetric2.get(0).tags.get("topic")); // test ManagedLedgerMetrics - List mlMetric = ((List) metrics.get("pulsar_ml_ReadEntriesOpsCacheMissesRate")); + List mlMetric = ((List) metrics.get("pulsar_ml_ReadEntriesOpsCacheMissesRate")); assertEquals(mlMetric.size(), 1); if (cacheEnable) { assertEquals(mlMetric.get(0).value, 1.0); @@ -716,9 +717,9 @@ public void testPerTopicExpiredStat() throws Exception { ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut); String metricsStr = statsOut.toString(); - Multimap metrics = PrometheusMetricsClient.parseMetrics(metricsStr); + Multimap metrics = parseMetrics(metricsStr); // There should be 2 metrics with different tags for each topic - List cm = (List) metrics.get("pulsar_subscription_last_expire_timestamp"); + List cm = (List) metrics.get("pulsar_subscription_last_expire_timestamp"); assertEquals(cm.size(), 2); assertEquals(cm.get(0).tags.get("topic"), topic2); assertEquals(cm.get(0).tags.get("namespace"), ns); @@ -734,7 +735,7 @@ public void testPerTopicExpiredStat() throws Exception { assertEquals((long) field.get(subscription), (long) cm.get(i).value); } - cm = (List) metrics.get("pulsar_subscription_msg_rate_expired"); + cm = (List) metrics.get("pulsar_subscription_msg_rate_expired"); assertEquals(cm.size(), 2); assertEquals(cm.get(0).tags.get("topic"), topic2); assertEquals(cm.get(0).tags.get("namespace"), ns); @@ -753,7 +754,7 @@ public void testPerTopicExpiredStat() throws Exception { assertEquals(Double.valueOf(nf.format(monitor.getMessageExpiryRate())).doubleValue(), cm.get(i).value); } - cm = (List) metrics.get("pulsar_subscription_total_msg_expired"); + cm = (List) metrics.get("pulsar_subscription_total_msg_expired"); assertEquals(cm.size(), 2); assertEquals(cm.get(0).tags.get("topic"), topic2); assertEquals(cm.get(0).tags.get("namespace"), ns); @@ -797,7 +798,7 @@ public void testBundlesMetrics() throws Exception { ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); PrometheusMetricsGenerator.generate(pulsar, false, false, false, statsOut); String metricsStr = statsOut.toString(); - Multimap metrics = PrometheusMetricsClient.parseMetrics(metricsStr); + Multimap metrics = parseMetrics(metricsStr); assertTrue(metrics.containsKey("pulsar_bundle_msg_rate_in")); assertTrue(metrics.containsKey("pulsar_bundle_msg_rate_out")); assertTrue(metrics.containsKey("pulsar_bundle_topics_count")); @@ -842,7 +843,7 @@ public void testNonPersistentSubMetrics() throws Exception { ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut); String metricsStr = statsOut.toString(); - Multimap metrics = PrometheusMetricsClient.parseMetrics(metricsStr); + Multimap metrics = parseMetrics(metricsStr); assertTrue(metrics.containsKey("pulsar_subscription_back_log")); assertTrue(metrics.containsKey("pulsar_subscription_back_log_no_delayed")); assertTrue(metrics.containsKey("pulsar_subscription_msg_throughput_out")); @@ -890,36 +891,36 @@ public void testPerNamespaceStats() throws Exception { PrometheusMetricsGenerator.generate(pulsar, false, false, false, statsOut); String metricsStr = statsOut.toString(); - Multimap metrics = PrometheusMetricsClient.parseMetrics(metricsStr); + Multimap metrics = parseMetrics(metricsStr); metrics.entries().forEach(e -> { System.out.println(e.getKey() + ": " + e.getValue()); }); // There should be 1 metric aggregated per namespace - List cm = (List) metrics.get("pulsar_storage_write_latency_le_1"); + List cm = (List) metrics.get("pulsar_storage_write_latency_le_1"); assertEquals(cm.size(), 1); assertNull(cm.get(0).tags.get("topic")); assertEquals(cm.get(0).tags.get("namespace"), "my-property/use/my-ns"); - cm = (List) metrics.get("pulsar_producers_count"); + cm = (List) metrics.get("pulsar_producers_count"); assertEquals(cm.size(), 1); assertNull(cm.get(0).tags.get("topic")); assertEquals(cm.get(0).tags.get("namespace"), "my-property/use/my-ns"); - cm = (List) metrics.get("pulsar_in_bytes_total"); + cm = (List) metrics.get("pulsar_in_bytes_total"); assertEquals(cm.size(), 1); assertEquals(cm.get(0).tags.get("namespace"), "my-property/use/my-ns"); - cm = (List) metrics.get("pulsar_in_messages_total"); + cm = (List) metrics.get("pulsar_in_messages_total"); assertEquals(cm.size(), 1); assertEquals(cm.get(0).tags.get("namespace"), "my-property/use/my-ns"); - cm = (List) metrics.get("pulsar_out_bytes_total"); + cm = (List) metrics.get("pulsar_out_bytes_total"); assertEquals(cm.size(), 1); assertEquals(cm.get(0).tags.get("namespace"), "my-property/use/my-ns"); - cm = (List) metrics.get("pulsar_out_messages_total"); + cm = (List) metrics.get("pulsar_out_messages_total"); assertEquals(cm.size(), 1); assertEquals(cm.get(0).tags.get("namespace"), "my-property/use/my-ns"); @@ -963,13 +964,13 @@ public void testPerProducerStats() throws Exception { PrometheusMetricsGenerator.generate(pulsar, true, false, true, statsOut); String metricsStr = statsOut.toString(); - Multimap metrics = PrometheusMetricsClient.parseMetrics(metricsStr); + Multimap metrics = parseMetrics(metricsStr); metrics.entries().forEach(e -> { System.out.println(e.getKey() + ": " + e.getValue()); }); - List cm = (List) metrics.get("pulsar_producer_msg_rate_in"); + List cm = (List) metrics.get("pulsar_producer_msg_rate_in"); assertEquals(cm.size(), 2); assertEquals(cm.get(0).tags.get("namespace"), "my-property/use/my-ns"); assertEquals(cm.get(0).tags.get("topic"), "persistent://my-property/use/my-ns/my-topic2"); @@ -981,7 +982,7 @@ public void testPerProducerStats() throws Exception { assertEquals(cm.get(1).tags.get("producer_name"), "producer1"); assertEquals(cm.get(1).tags.get("producer_id"), "0"); - cm = (List) metrics.get("pulsar_producer_msg_throughput_in"); + cm = (List) metrics.get("pulsar_producer_msg_throughput_in"); assertEquals(cm.size(), 2); assertEquals(cm.get(0).tags.get("namespace"), "my-property/use/my-ns"); assertEquals(cm.get(0).tags.get("topic"), "persistent://my-property/use/my-ns/my-topic2"); @@ -1031,14 +1032,14 @@ public void testPerConsumerStats() throws Exception { PrometheusMetricsGenerator.generate(pulsar, true, true, false, statsOut); String metricsStr = statsOut.toString(); - Multimap metrics = PrometheusMetricsClient.parseMetrics(metricsStr); + Multimap metrics = parseMetrics(metricsStr); metrics.entries().forEach(e -> { System.out.println(e.getKey() + ": " + e.getValue()); }); // There should be 1 metric aggregated per namespace - List cm = (List) metrics.get("pulsar_out_bytes_total"); + List cm = (List) metrics.get("pulsar_out_bytes_total"); assertEquals(cm.size(), 4); assertEquals(cm.get(0).tags.get("namespace"), "my-property/use/my-ns"); assertEquals(cm.get(0).tags.get("topic"), "persistent://my-property/use/my-ns/my-topic2"); @@ -1058,7 +1059,7 @@ public void testPerConsumerStats() throws Exception { assertEquals(cm.get(3).tags.get("subscription"), "test"); assertEquals(cm.get(3).tags.get("consumer_id"), "0"); - cm = (List) metrics.get("pulsar_out_messages_total"); + cm = (List) metrics.get("pulsar_out_messages_total"); assertEquals(cm.size(), 4); assertEquals(cm.get(0).tags.get("namespace"), "my-property/use/my-ns"); assertEquals(cm.get(0).tags.get("topic"), "persistent://my-property/use/my-ns/my-topic2"); @@ -1222,17 +1223,17 @@ public void testManagedLedgerCacheStats() throws Exception { PrometheusMetricsGenerator.generate(pulsar, false, false, false, statsOut); String metricsStr = statsOut.toString(); - Multimap metrics = PrometheusMetricsClient.parseMetrics(metricsStr); + Multimap metrics = parseMetrics(metricsStr); metrics.entries().forEach(e -> System.out.println(e.getKey() + ": " + e.getValue()) ); - List cm = (List) metrics.get("pulsar_ml_cache_evictions"); + List cm = (List) metrics.get("pulsar_ml_cache_evictions"); assertEquals(cm.size(), 1); assertEquals(cm.get(0).tags.get("cluster"), "test"); - cm = (List) metrics.get("pulsar_ml_cache_hits_rate"); + cm = (List) metrics.get("pulsar_ml_cache_hits_rate"); assertEquals(cm.size(), 1); assertEquals(cm.get(0).tags.get("cluster"), "test"); @@ -1258,7 +1259,7 @@ public void testManagedLedgerStats() throws Exception { PrometheusMetricsGenerator.generate(pulsar, false, false, false, statsOut); String metricsStr = statsOut.toString(); - Multimap metrics = PrometheusMetricsClient.parseMetrics(metricsStr); + Multimap metrics = parseMetrics(metricsStr); metrics.entries().forEach(e -> System.out.println(e.getKey() + ": " + e.getValue()) @@ -1301,13 +1302,13 @@ public void testManagedLedgerStats() throws Exception { } }); - List cm = (List) metrics.get("pulsar_ml_AddEntryBytesRate"); + List cm = (List) metrics.get("pulsar_ml_AddEntryBytesRate"); assertEquals(cm.size(), 2); assertEquals(cm.get(0).tags.get("cluster"), "test"); String ns = cm.get(0).tags.get("namespace"); assertTrue(ns.equals("my-property/use/my-ns") || ns.equals("my-property/use/my-ns2")); - cm = (List) metrics.get("pulsar_ml_AddEntryMessagesRate"); + cm = (List) metrics.get("pulsar_ml_AddEntryMessagesRate"); assertEquals(cm.size(), 2); assertEquals(cm.get(0).tags.get("cluster"), "test"); ns = cm.get(0).tags.get("namespace"); @@ -1336,32 +1337,32 @@ public void testManagedLedgerBookieClientStats() throws Exception { PrometheusMetricsGenerator.generate(pulsar, false, false, false, statsOut); String metricsStr = statsOut.toString(); - Multimap metrics = PrometheusMetricsClient.parseMetrics(metricsStr); + Multimap metrics = parseMetrics(metricsStr); metrics.entries().forEach(e -> System.out.println(e.getKey() + ": " + e.getValue()) ); - List cm = (List) metrics.get( + List cm = (List) metrics.get( keyNameBySubstrings(metrics, "pulsar_managedLedger_client", "bookkeeper_ml_scheduler_threads")); assertEquals(cm.size(), 1); assertEquals(cm.get(0).tags.get("cluster"), "test"); - cm = (List) metrics.get( + cm = (List) metrics.get( keyNameBySubstrings(metrics, "pulsar_managedLedger_client", "bookkeeper_ml_scheduler_task_execution_sum")); assertEquals(cm.size(), 2); assertEquals(cm.get(0).tags.get("cluster"), "test"); - cm = (List) metrics.get( + cm = (List) metrics.get( keyNameBySubstrings(metrics, "pulsar_managedLedger_client", "bookkeeper_ml_scheduler_max_queue_size")); assertEquals(cm.size(), 1); assertEquals(cm.get(0).tags.get("cluster"), "test"); } - private static String keyNameBySubstrings(Multimap metrics, String... substrings) { + private static String keyNameBySubstrings(Multimap metrics, String... substrings) { for (String key: metrics.keys()) { boolean found = true; for (String s: substrings) { @@ -1416,10 +1417,10 @@ public String getCommandData() { ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); PrometheusMetricsGenerator.generate(pulsar, false, false, false, statsOut); String metricsStr = statsOut.toString(); - Multimap metrics = PrometheusMetricsClient.parseMetrics(metricsStr); - List cm = (List) metrics.get("pulsar_authentication_success_total"); + Multimap metrics = parseMetrics(metricsStr); + List cm = (List) metrics.get("pulsar_authentication_success_total"); boolean haveSucceed = false; - for (PrometheusMetricsClient.Metric metric : cm) { + for (Metric metric : cm) { if (Objects.equals(metric.tags.get("auth_method"), "token") && Objects.equals(metric.tags.get("provider_name"), provider.getClass().getSimpleName())) { haveSucceed = true; @@ -1427,10 +1428,10 @@ public String getCommandData() { } Assert.assertTrue(haveSucceed); - cm = (List) metrics.get("pulsar_authentication_failures_total"); + cm = (List) metrics.get("pulsar_authentication_failures_total"); boolean haveFailed = false; - for (PrometheusMetricsClient.Metric metric : cm) { + for (Metric metric : cm) { if (Objects.equals(metric.tags.get("auth_method"), "token") && Objects.equals(metric.tags.get("reason"), AuthenticationProviderToken.ErrorCode.INVALID_AUTH_DATA.name()) @@ -1477,8 +1478,8 @@ public String getCommandData() { ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); PrometheusMetricsGenerator.generate(pulsar, false, false, false, statsOut); String metricsStr = statsOut.toString(); - Multimap metrics = PrometheusMetricsClient.parseMetrics(metricsStr); - List cm = (List) metrics.get("pulsar_expired_token_total"); + Multimap metrics = parseMetrics(metricsStr); + List cm = (List) metrics.get("pulsar_expired_token_total"); assertEquals(cm.size(), 1); provider.close(); @@ -1518,10 +1519,10 @@ public String getCommandData() { ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); PrometheusMetricsGenerator.generate(pulsar, false, false, false, statsOut); String metricsStr = statsOut.toString(); - Multimap metrics = PrometheusMetricsClient.parseMetrics(metricsStr); - PrometheusMetricsClient.Metric countMetric = ((List) metrics.get("pulsar_expiring_token_minutes_count")).get(0); + Multimap metrics = parseMetrics(metricsStr); + Metric countMetric = ((List) metrics.get("pulsar_expiring_token_minutes_count")).get(0); assertEquals(countMetric.value, tokenRemainTime.length); - List cm = (List) metrics.get("pulsar_expiring_token_minutes_bucket"); + List cm = (List) metrics.get("pulsar_expiring_token_minutes_bucket"); assertEquals(cm.size(), 5); cm.forEach((e) -> { switch (e.tags.get("le")) { @@ -1547,8 +1548,8 @@ public String getCommandData() { @Test public void testParsingWithPositiveInfinityValue() { - Multimap metrics = PrometheusMetricsClient.parseMetrics("pulsar_broker_publish_latency{cluster=\"test\",quantile=\"0.0\"} +Inf"); - List cm = (List) metrics.get("pulsar_broker_publish_latency"); + Multimap metrics = parseMetrics("pulsar_broker_publish_latency{cluster=\"test\",quantile=\"0.0\"} +Inf"); + List cm = (List) metrics.get("pulsar_broker_publish_latency"); assertEquals(cm.size(), 1); assertEquals(cm.get(0).tags.get("cluster"), "test"); assertEquals(cm.get(0).tags.get("quantile"), "0.0"); @@ -1557,8 +1558,8 @@ public void testParsingWithPositiveInfinityValue() { @Test public void testParsingWithNegativeInfinityValue() { - Multimap metrics = PrometheusMetricsClient.parseMetrics("pulsar_broker_publish_latency{cluster=\"test\",quantile=\"0.0\"} -Inf"); - List cm = (List) metrics.get("pulsar_broker_publish_latency"); + Multimap metrics = parseMetrics("pulsar_broker_publish_latency{cluster=\"test\",quantile=\"0.0\"} -Inf"); + List cm = (List) metrics.get("pulsar_broker_publish_latency"); assertEquals(cm.size(), 1); assertEquals(cm.get(0).tags.get("cluster"), "test"); assertEquals(cm.get(0).tags.get("quantile"), "0.0"); @@ -1593,9 +1594,9 @@ public void testManagedCursorPersistStats() throws Exception { PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut); String metricsStr = statsOut.toString(); - Multimap metrics = PrometheusMetricsClient.parseMetrics(metricsStr); + Multimap metrics = parseMetrics(metricsStr); - List cm = (List) metrics.get("pulsar_ml_cursor_persistLedgerSucceed"); + List cm = (List) metrics.get("pulsar_ml_cursor_persistLedgerSucceed"); assertEquals(cm.size(), 1); assertEquals(cm.get(0).tags.get("cluster"), "test"); assertEquals(cm.get(0).tags.get("cursor_name"), subName); @@ -1605,8 +1606,8 @@ public void testManagedCursorPersistStats() throws Exception { ByteArrayOutputStream statsOut2 = new ByteArrayOutputStream(); PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut2); String metricsStr2 = statsOut2.toString(); - Multimap metrics2 = PrometheusMetricsClient.parseMetrics(metricsStr2); - List cm2 = (List) metrics2.get("pulsar_ml_cursor_persistLedgerSucceed"); + Multimap metrics2 = parseMetrics(metricsStr2); + List cm2 = (List) metrics2.get("pulsar_ml_cursor_persistLedgerSucceed"); assertEquals(cm2.size(), 0); producer.close(); @@ -1624,17 +1625,17 @@ public void testBrokerConnection() throws Exception { ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut); String metricsStr = statsOut.toString(); - Multimap metrics = PrometheusMetricsClient.parseMetrics(metricsStr); - List cm = (List) metrics.get("pulsar_connection_created_total_count"); + Multimap metrics = parseMetrics(metricsStr); + List cm = (List) metrics.get("pulsar_connection_created_total_count"); compareBrokerConnectionStateCount(cm, 1.0); - cm = (List) metrics.get("pulsar_connection_create_success_count"); + cm = (List) metrics.get("pulsar_connection_create_success_count"); compareBrokerConnectionStateCount(cm, 1.0); - cm = (List) metrics.get("pulsar_connection_closed_total_count"); + cm = (List) metrics.get("pulsar_connection_closed_total_count"); compareBrokerConnectionStateCount(cm, 0.0); - cm = (List) metrics.get("pulsar_active_connections"); + cm = (List) metrics.get("pulsar_active_connections"); compareBrokerConnectionStateCount(cm, 1.0); pulsarClient.close(); @@ -1642,8 +1643,8 @@ public void testBrokerConnection() throws Exception { PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut); metricsStr = statsOut.toString(); - metrics = PrometheusMetricsClient.parseMetrics(metricsStr); - cm = (List) metrics.get("pulsar_connection_closed_total_count"); + metrics = parseMetrics(metricsStr); + cm = (List) metrics.get("pulsar_connection_closed_total_count"); compareBrokerConnectionStateCount(cm, 1.0); pulsar.getConfiguration().setAuthenticationEnabled(true); @@ -1665,24 +1666,24 @@ public void testBrokerConnection() throws Exception { PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut); metricsStr = statsOut.toString(); - metrics = PrometheusMetricsClient.parseMetrics(metricsStr); - cm = (List) metrics.get("pulsar_connection_closed_total_count"); + metrics = parseMetrics(metricsStr); + cm = (List) metrics.get("pulsar_connection_closed_total_count"); compareBrokerConnectionStateCount(cm, 2.0); - cm = (List) metrics.get("pulsar_connection_create_fail_count"); + cm = (List) metrics.get("pulsar_connection_create_fail_count"); compareBrokerConnectionStateCount(cm, 1.0); - cm = (List) metrics.get("pulsar_connection_create_success_count"); + cm = (List) metrics.get("pulsar_connection_create_success_count"); compareBrokerConnectionStateCount(cm, 1.0); - cm = (List) metrics.get("pulsar_active_connections"); + cm = (List) metrics.get("pulsar_active_connections"); compareBrokerConnectionStateCount(cm, 0.0); - cm = (List) metrics.get("pulsar_connection_created_total_count"); + cm = (List) metrics.get("pulsar_connection_created_total_count"); compareBrokerConnectionStateCount(cm, 2.0); } - private void compareBrokerConnectionStateCount(List cm, double count) { + private void compareBrokerConnectionStateCount(List cm, double count) { assertEquals(cm.size(), 1); assertEquals(cm.get(0).tags.get("cluster"), "test"); assertEquals(cm.get(0).tags.get("broker"), "localhost"); @@ -1693,7 +1694,7 @@ private void compareBrokerConnectionStateCount(List metrics = PrometheusMetricsClient.parseMetrics(metricsStr); - List cm = (List) metrics.get("pulsar_compaction_removed_event_count"); + Multimap metrics = parseMetrics(metricsStr); + List cm = (List) metrics.get("pulsar_compaction_removed_event_count"); assertEquals(cm.size(), 0); - cm = (List) metrics.get("pulsar_compaction_succeed_count"); + cm = (List) metrics.get("pulsar_compaction_succeed_count"); assertEquals(cm.size(), 0); - cm = (List) metrics.get("pulsar_compaction_failed_count"); + cm = (List) metrics.get("pulsar_compaction_failed_count"); assertEquals(cm.size(), 0); - cm = (List) metrics.get("pulsar_compaction_duration_time_in_mills"); + cm = (List) metrics.get("pulsar_compaction_duration_time_in_mills"); assertEquals(cm.size(), 0); - cm = (List) metrics.get("pulsar_compaction_read_throughput"); + cm = (List) metrics.get("pulsar_compaction_read_throughput"); assertEquals(cm.size(), 0); - cm = (List) metrics.get("pulsar_compaction_write_throughput"); + cm = (List) metrics.get("pulsar_compaction_write_throughput"); assertEquals(cm.size(), 0); - cm = (List) metrics.get("pulsar_compaction_compacted_entries_count"); + cm = (List) metrics.get("pulsar_compaction_compacted_entries_count"); assertEquals(cm.size(), 0); - cm = (List) metrics.get("pulsar_compaction_compacted_entries_size"); + cm = (List) metrics.get("pulsar_compaction_compacted_entries_size"); assertEquals(cm.size(), 0); // final int numMessages = 1000; @@ -1743,29 +1744,29 @@ public void testCompaction() throws Exception { statsOut = new ByteArrayOutputStream(); PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut); metricsStr = statsOut.toString(); - metrics = PrometheusMetricsClient.parseMetrics(metricsStr); - cm = (List) metrics.get("pulsar_compaction_removed_event_count"); + metrics = parseMetrics(metricsStr); + cm = (List) metrics.get("pulsar_compaction_removed_event_count"); assertEquals(cm.size(), 1); assertEquals(cm.get(0).value, 990); - cm = (List) metrics.get("pulsar_compaction_succeed_count"); + cm = (List) metrics.get("pulsar_compaction_succeed_count"); assertEquals(cm.size(), 1); assertEquals(cm.get(0).value, 1); - cm = (List) metrics.get("pulsar_compaction_failed_count"); + cm = (List) metrics.get("pulsar_compaction_failed_count"); assertEquals(cm.size(), 1); assertEquals(cm.get(0).value, 0); - cm = (List) metrics.get("pulsar_compaction_duration_time_in_mills"); + cm = (List) metrics.get("pulsar_compaction_duration_time_in_mills"); assertEquals(cm.size(), 1); assertTrue(cm.get(0).value > 0); - cm = (List) metrics.get("pulsar_compaction_read_throughput"); + cm = (List) metrics.get("pulsar_compaction_read_throughput"); assertEquals(cm.size(), 1); assertTrue(cm.get(0).value > 0); - cm = (List) metrics.get("pulsar_compaction_write_throughput"); + cm = (List) metrics.get("pulsar_compaction_write_throughput"); assertEquals(cm.size(), 1); assertTrue(cm.get(0).value > 0); - cm = (List) metrics.get("pulsar_compaction_compacted_entries_count"); + cm = (List) metrics.get("pulsar_compaction_compacted_entries_count"); assertEquals(cm.size(), 1); assertEquals(cm.get(0).value, 10); - cm = (List) metrics.get("pulsar_compaction_compacted_entries_size"); + cm = (List) metrics.get("pulsar_compaction_compacted_entries_size"); assertEquals(cm.size(), 1); assertEquals(cm.get(0).value, 840); @@ -1795,7 +1796,7 @@ public void testMetricsWithCache() throws Throwable { String metricsStr1 = statsOut1.toString(); String metricsStr2 = statsOut2.toString(); assertEquals(metricsStr1, metricsStr2); - Multimap metrics = PrometheusMetricsClient.parseMetrics(metricsStr1); + Multimap metrics = parseMetrics(metricsStr1); } Thread.sleep(TimeUnit.SECONDS.toMillis(period / 2)); @@ -1828,8 +1829,8 @@ public void testSplitTopicAndPartitionLabel() throws Exception { ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); PrometheusMetricsGenerator.generate(pulsar, true, false, false, true, statsOut); String metricsStr = statsOut.toString(); - Multimap metrics = PrometheusMetricsClient.parseMetrics(metricsStr); - Collection metric = metrics.get("pulsar_consumers_count"); + Multimap metrics = parseMetrics(metricsStr); + Collection metric = metrics.get("pulsar_consumers_count"); assertTrue(metric.size() >= 15); metric.forEach(item -> { if (ns1.equals(item.tags.get("namespace"))) { @@ -1844,7 +1845,7 @@ public void testSplitTopicAndPartitionLabel() throws Exception { consumer2.close(); } - private void compareCompactionStateCount(List cm, double count) { + private void compareCompactionStateCount(List cm, double count) { assertEquals(cm.size(), 1); assertEquals(cm.get(0).tags.get("cluster"), "test"); assertEquals(cm.get(0).tags.get("broker"), "localhost"); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/SubscriptionStatsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/SubscriptionStatsTest.java index 8c91d75978475..e39860274d12f 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/SubscriptionStatsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/SubscriptionStatsTest.java @@ -19,6 +19,8 @@ package org.apache.pulsar.broker.stats; import static org.apache.pulsar.broker.BrokerTestUtil.spyWithClassAndConstructorArgs; +import static org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient.Metric; +import static org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient.parseMetrics; import static org.mockito.Mockito.mock; import com.google.common.collect.Multimap; import java.io.ByteArrayOutputStream; @@ -35,7 +37,6 @@ import org.apache.pulsar.broker.service.plugin.EntryFilter; import org.apache.pulsar.broker.service.plugin.EntryFilterTest; import org.apache.pulsar.broker.service.plugin.EntryFilterWithClassLoader; -import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient; import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsGenerator; import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.api.Consumer; @@ -85,7 +86,7 @@ protected void cleanup() throws Exception { @Test public void testConsumersAfterMarkDelete() throws PulsarClientException, PulsarAdminException { final String topicName = "persistent://my-property/my-ns/testConsumersAfterMarkDelete-" - + UUID.randomUUID().toString(); + + UUID.randomUUID(); final String subName = "my-sub"; Consumer consumer1 = pulsarClient.newConsumer() @@ -234,15 +235,15 @@ public void testSubscriptionStats(final String topic, final String subName, bool ByteArrayOutputStream output = new ByteArrayOutputStream(); PrometheusMetricsGenerator.generate(pulsar, enableTopicStats, false, false, output); String metricsStr = output.toString(); - Multimap metrics = PrometheusMetricsClient.parseMetrics(metricsStr); + Multimap metrics = parseMetrics(metricsStr); - Collection throughFilterMetrics = + Collection throughFilterMetrics = metrics.get("pulsar_subscription_filter_processed_msg_count"); - Collection acceptedMetrics = + Collection acceptedMetrics = metrics.get("pulsar_subscription_filter_accepted_msg_count"); - Collection rejectedMetrics = + Collection rejectedMetrics = metrics.get("pulsar_subscription_filter_rejected_msg_count"); - Collection rescheduledMetrics = + Collection rescheduledMetrics = metrics.get("pulsar_subscription_filter_rescheduled_msg_count"); if (enableTopicStats) { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/TransactionMetricsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/TransactionMetricsTest.java index 32cbcaed8943b..723a493eca1df 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/TransactionMetricsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/TransactionMetricsTest.java @@ -19,6 +19,7 @@ package org.apache.pulsar.broker.stats; import static com.google.common.base.Preconditions.checkArgument; +import static org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient.Metric; import static org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient.parseMetrics; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertTrue; @@ -39,7 +40,6 @@ import lombok.extern.slf4j.Slf4j; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.service.BrokerTestBase; -import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient; import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsGenerator; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.MessageId; @@ -120,8 +120,8 @@ public void testTransactionCoordinatorMetrics() throws Exception { ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut); String metricsStr = statsOut.toString(); - Multimap metrics = parseMetrics(metricsStr); - Collection metric = metrics.get("pulsar_txn_active_count"); + Multimap metrics = parseMetrics(metricsStr); + Collection metric = metrics.get("pulsar_txn_active_count"); assertEquals(metric.size(), 2); metric.forEach(item -> { if ("0".equals(item.tags.get("coordinator_id"))) { @@ -188,9 +188,9 @@ public void testTransactionCoordinatorRateMetrics() throws Exception { ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut); String metricsStr = statsOut.toString(); - Multimap metrics = parseMetrics(metricsStr); + Multimap metrics = parseMetrics(metricsStr); - Collection metric = metrics.get("pulsar_txn_created_total"); + Collection metric = metrics.get("pulsar_txn_created_total"); assertEquals(metric.size(), 1); metric.forEach(item -> assertEquals(item.value, txnCount)); @@ -275,9 +275,9 @@ public void testManagedLedgerMetrics() throws Exception { PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut); String metricsStr = statsOut.toString(); - Multimap metrics = parseMetrics(metricsStr); + Multimap metrics = parseMetrics(metricsStr); - Collection metric = metrics.get("pulsar_storage_size"); + Collection metric = metrics.get("pulsar_storage_size"); checkManagedLedgerMetrics(subName, 32, metric); checkManagedLedgerMetrics(MLTransactionLogImpl.TRANSACTION_SUBSCRIPTION_NAME, 252, metric); @@ -337,12 +337,12 @@ public void testManagedLedgerMetricsWhenPendingAckNotInit() throws Exception { PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut); String metricsStr = statsOut.toString(); - Multimap metrics = parseMetrics(metricsStr); + Multimap metrics = parseMetrics(metricsStr); - Collection metric = metrics.get("pulsar_storage_size"); + Collection metric = metrics.get("pulsar_storage_size"); checkManagedLedgerMetrics(subName, 32, metric); //No statistics of the pendingAck are generated when the pendingAck is not initialized. - for (PrometheusMetricsClient.Metric metric1 : metric) { + for (Metric metric1 : metric) { if (metric1.tags.containsValue(subName2)) { Assert.fail(); } @@ -432,9 +432,9 @@ public void testDuplicateMetricTypeDefinitions() throws Exception { } - private void checkManagedLedgerMetrics(String tag, double value, Collection metrics) { + private void checkManagedLedgerMetrics(String tag, double value, Collection metrics) { boolean exist = false; - for (PrometheusMetricsClient.Metric metric1 : metrics) { + for (Metric metric1 : metrics) { if (metric1.tags.containsValue(tag)) { assertEquals(metric1.value, value); exist = true; diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsClient.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsClient.java index 474d0d90bc3f7..77538bf619c0d 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsClient.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsClient.java @@ -1,16 +1,20 @@ package org.apache.pulsar.broker.stats.prometheus; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Fail.fail; import static org.testng.Assert.assertTrue; import com.google.common.base.MoreObjects; import com.google.common.base.Splitter; import com.google.common.collect.ArrayListMultimap; import com.google.common.collect.Multimap; import io.restassured.RestAssured; +import java.util.Arrays; import java.util.List; import java.util.Map; import java.util.TreeMap; import java.util.regex.Matcher; import java.util.regex.Pattern; +import org.apache.commons.lang3.tuple.Pair; public class PrometheusMetricsClient { private final String host; @@ -38,7 +42,7 @@ public static Multimap parseMetrics(String metrics) { // or // pulsar_subscriptions_count{cluster="standalone", namespace="public/default", // topic="persistent://public/default/test-2"} 0.0 - Pattern pattern = Pattern.compile("^(\\w+)\\{([^\\}]+)\\}\\s([+-]?[\\d\\w\\.-]+)$"); + Pattern pattern = Pattern.compile("^(\\w+)\\{([^}]+)}\\s([+-]?[\\d\\w.-]+)$"); Pattern tagsPattern = Pattern.compile("(\\w+)=\"([^\"]+)\"(,\\s?)?"); Splitter.on("\n").split(metrics).forEach(line -> { @@ -101,5 +105,36 @@ public List findByNameAndLabels(String metricName, String labelName, Str .filter(metric -> metric.contains(labelName, labelValue)) .toList(); } + + @SafeVarargs + public final List findByNameAndLabels(String metricName, Pair... nameValuePairs) { + return nameToDataPoints.get(metricName) + .stream() + .filter(metric -> { + for (Pair nameValuePair : nameValuePairs) { + String labelName = nameValuePair.getLeft(); + String labelValue = nameValuePair.getRight(); + if (!metric.contains(labelName, labelValue)) { + return false; + } + } + return true; + }) + .toList(); + } + + @SafeVarargs + public final Metric findSingleMetricByNameAndLabels(String metricName, Pair... nameValuePairs) { + List metricByNameAndLabels = findByNameAndLabels(metricName, nameValuePairs); + if (metricByNameAndLabels.size() != 1) { + fail("Expected to find 1 metric, but found the following: "+metricByNameAndLabels + + ". Metrics are = "+nameToDataPoints.get(metricName)+". Labels requested = "+ Arrays.toString( + nameValuePairs)); + } + assertThat(metricByNameAndLabels) + .describedAs(metricByNameAndLabels.toString()) + .hasSize(1); + return metricByNameAndLabels.get(0); + } } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/TransactionBufferClientTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/TransactionBufferClientTest.java index f40efa1228064..380c32829b48e 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/TransactionBufferClientTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/TransactionBufferClientTest.java @@ -18,27 +18,35 @@ */ package org.apache.pulsar.broker.transaction.buffer; +import static org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient.Metric; +import static org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient.parseMetrics; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyInt; import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertTrue; +import static org.testng.Assert.fail; import com.google.common.collect.Multimap; import com.google.common.collect.Sets; import io.netty.channel.Channel; import io.netty.channel.ChannelHandlerContext; import io.netty.util.HashedWheelTimer; import io.netty.util.concurrent.DefaultThreadFactory; -import lombok.Cleanup; import java.io.ByteArrayOutputStream; +import java.lang.reflect.Field; import java.util.ArrayList; import java.util.Collection; import java.util.List; import java.util.Optional; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentSkipListMap; import java.util.concurrent.ExecutionException; - +import java.util.concurrent.TimeUnit; +import lombok.Cleanup; import org.apache.pulsar.broker.PulsarServerException; import org.apache.pulsar.broker.PulsarService; -import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient; import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsGenerator; import org.apache.pulsar.broker.transaction.TransactionTestBase; import org.apache.pulsar.broker.transaction.buffer.impl.TransactionBufferClientImpl; @@ -70,14 +78,6 @@ import org.testng.annotations.AfterClass; import org.testng.annotations.BeforeClass; import org.testng.annotations.Test; -import java.lang.reflect.Field; -import java.util.concurrent.ConcurrentSkipListMap; -import java.util.concurrent.TimeUnit; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; -import static org.testng.Assert.assertEquals; -import static org.testng.Assert.assertTrue; -import static org.testng.Assert.fail; @Test(groups = "broker") public class TransactionBufferClientTest extends TransactionTestBase { @@ -228,28 +228,28 @@ public void testTransactionBufferMetrics() throws Exception { ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); PrometheusMetricsGenerator.generate(pulsarServiceList.get(0), true, false, false, statsOut); String metricsStr = statsOut.toString(); - Multimap metricsMap = PrometheusMetricsClient.parseMetrics(metricsStr); + Multimap metricsMap = parseMetrics(metricsStr); - Collection abortFailed = metricsMap.get("pulsar_txn_tb_client_abort_failed_total"); - Collection commitFailed = metricsMap.get("pulsar_txn_tb_client_commit_failed_total"); - Collection abortLatencyCount = + Collection abortFailed = metricsMap.get("pulsar_txn_tb_client_abort_failed_total"); + Collection commitFailed = metricsMap.get("pulsar_txn_tb_client_commit_failed_total"); + Collection abortLatencyCount = metricsMap.get("pulsar_txn_tb_client_abort_latency_count"); - Collection commitLatencyCount = + Collection commitLatencyCount = metricsMap.get("pulsar_txn_tb_client_commit_latency_count"); - Collection pending = metricsMap.get("pulsar_txn_tb_client_pending_requests"); + Collection pending = metricsMap.get("pulsar_txn_tb_client_pending_requests"); assertEquals(abortFailed.stream().mapToDouble(metric -> metric.value).sum(), 0); assertEquals(commitFailed.stream().mapToDouble(metric -> metric.value).sum(), 0); for (int i = 0; i < partitions; i++) { String topic = partitionedTopicName.getPartition(i).toString(); - Optional optional = abortLatencyCount.stream() + Optional optional = abortLatencyCount.stream() .filter(metric -> metric.tags.get("topic").equals(topic)).findFirst(); assertTrue(optional.isPresent()); assertEquals(optional.get().value, 1D); - Optional optional1 = commitLatencyCount.stream() + Optional optional1 = commitLatencyCount.stream() .filter(metric -> metric.tags.get("topic").equals(topic)).findFirst(); assertTrue(optional1.isPresent()); assertEquals(optional1.get().value, 1D); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/pendingack/PendingAckPersistentTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/pendingack/PendingAckPersistentTest.java index ab9f7f217e7e9..6c24b6b3f0151 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/pendingack/PendingAckPersistentTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/pendingack/PendingAckPersistentTest.java @@ -19,6 +19,8 @@ package org.apache.pulsar.broker.transaction.pendingack; +import static org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient.Metric; +import static org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient.parseMetrics; import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.spy; @@ -26,6 +28,7 @@ import static org.testng.AssertJUnit.assertNotNull; import static org.testng.AssertJUnit.assertTrue; import static org.testng.AssertJUnit.fail; +import com.google.common.collect.Multimap; import java.io.ByteArrayOutputStream; import java.lang.reflect.Field; import java.lang.reflect.Method; @@ -38,7 +41,6 @@ import java.util.concurrent.ConcurrentSkipListMap; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; -import com.google.common.collect.Multimap; import lombok.Cleanup; import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.mledger.ManagedCursor; @@ -49,7 +51,6 @@ import org.apache.pulsar.broker.service.BrokerServiceException; import org.apache.pulsar.broker.service.persistent.PersistentSubscription; import org.apache.pulsar.broker.service.persistent.PersistentTopic; -import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient; import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsGenerator; import org.apache.pulsar.broker.transaction.TransactionTestBase; import org.apache.pulsar.broker.transaction.pendingack.impl.MLPendingAckStore; @@ -62,9 +63,9 @@ import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.api.SubscriptionType; import org.apache.pulsar.client.api.transaction.Transaction; +import org.apache.pulsar.client.api.transaction.TxnID; import org.apache.pulsar.client.impl.BatchMessageIdImpl; import org.apache.pulsar.client.impl.MessageIdImpl; -import org.apache.pulsar.client.api.transaction.TxnID; import org.apache.pulsar.client.impl.transaction.TransactionImpl; import org.apache.pulsar.common.naming.NamespaceName; import org.apache.pulsar.common.naming.SystemTopicNames; @@ -256,28 +257,28 @@ public void testPendingAckMetrics() throws Exception { ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); PrometheusMetricsGenerator.generate(pulsarServiceList.get(0), true, false, false, statsOut); String metricsStr = statsOut.toString(); - Multimap metricsMap = PrometheusMetricsClient.parseMetrics(metricsStr); + Multimap metricsMap = parseMetrics(metricsStr); - Collection abortedCount = metricsMap.get("pulsar_txn_tp_aborted_count_total"); - Collection committedCount = metricsMap.get("pulsar_txn_tp_committed_count_total"); - Collection commitLatency = metricsMap.get("pulsar_txn_tp_commit_latency"); + Collection abortedCount = metricsMap.get("pulsar_txn_tp_aborted_count_total"); + Collection committedCount = metricsMap.get("pulsar_txn_tp_committed_count_total"); + Collection commitLatency = metricsMap.get("pulsar_txn_tp_commit_latency"); Assert.assertTrue(commitLatency.size() > 0); int count = 0; - for (PrometheusMetricsClient.Metric metric : commitLatency) { + for (Metric metric : commitLatency) { if (metric.tags.get("topic").endsWith(PENDING_ACK_REPLAY_TOPIC) && metric.value > 0) { count++; } } Assert.assertTrue(count > 0); - for (PrometheusMetricsClient.Metric metric : abortedCount) { + for (Metric metric : abortedCount) { if (metric.tags.get("subscription").equals(subName) && metric.tags.get("status").equals("succeed")) { assertTrue(metric.tags.get("topic").endsWith(PENDING_ACK_REPLAY_TOPIC)); assertTrue(metric.value > 0); } } - for (PrometheusMetricsClient.Metric metric : committedCount) { + for (Metric metric : committedCount) { if (metric.tags.get("subscription").equals(subName) && metric.tags.get("status").equals("succeed")) { assertTrue(metric.tags.get("topic").endsWith(PENDING_ACK_REPLAY_TOPIC)); assertTrue(metric.value > 0); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/web/WebServiceTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/web/WebServiceTest.java index 6c38598f2dce2..8fb95eed789d5 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/web/WebServiceTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/web/WebServiceTest.java @@ -18,6 +18,8 @@ */ package org.apache.pulsar.broker.web; +import static org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient.Metric; +import static org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient.parseMetrics; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertTrue; import static org.testng.Assert.fail; @@ -51,7 +53,6 @@ import org.apache.commons.lang3.StringUtils; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.ServiceConfiguration; -import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient; import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsGenerator; import org.apache.pulsar.broker.testcontext.PulsarTestContext; import org.apache.pulsar.client.admin.PulsarAdmin; @@ -104,31 +105,31 @@ public void testWebExecutorMetrics() throws Exception { ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); PrometheusMetricsGenerator.generate(pulsar, false, false, false, statsOut); String metricsStr = statsOut.toString(); - Multimap metrics = PrometheusMetricsClient.parseMetrics(metricsStr); + Multimap metrics = parseMetrics(metricsStr); - Collection maxThreads = metrics.get("pulsar_web_executor_max_threads"); - Collection minThreads = metrics.get("pulsar_web_executor_min_threads"); - Collection activeThreads = metrics.get("pulsar_web_executor_active_threads"); - Collection idleThreads = metrics.get("pulsar_web_executor_idle_threads"); - Collection currentThreads = metrics.get("pulsar_web_executor_current_threads"); + Collection maxThreads = metrics.get("pulsar_web_executor_max_threads"); + Collection minThreads = metrics.get("pulsar_web_executor_min_threads"); + Collection activeThreads = metrics.get("pulsar_web_executor_active_threads"); + Collection idleThreads = metrics.get("pulsar_web_executor_idle_threads"); + Collection currentThreads = metrics.get("pulsar_web_executor_current_threads"); - for (PrometheusMetricsClient.Metric metric : maxThreads) { + for (Metric metric : maxThreads) { Assert.assertNotNull(metric.tags.get("cluster")); Assert.assertTrue(metric.value > 0); } - for (PrometheusMetricsClient.Metric metric : minThreads) { + for (Metric metric : minThreads) { Assert.assertNotNull(metric.tags.get("cluster")); Assert.assertTrue(metric.value > 0); } - for (PrometheusMetricsClient.Metric metric : activeThreads) { + for (Metric metric : activeThreads) { Assert.assertNotNull(metric.tags.get("cluster")); Assert.assertTrue(metric.value >= 0); } - for (PrometheusMetricsClient.Metric metric : idleThreads) { + for (Metric metric : idleThreads) { Assert.assertNotNull(metric.tags.get("cluster")); Assert.assertTrue(metric.value >= 0); } - for (PrometheusMetricsClient.Metric metric : currentThreads) { + for (Metric metric : currentThreads) { Assert.assertNotNull(metric.tags.get("cluster")); Assert.assertTrue(metric.value > 0); } diff --git a/pulsar-broker/src/test/resources/log4j2.xml b/pulsar-broker/src/test/resources/log4j2.xml new file mode 100644 index 0000000000000..0d9244e932857 --- /dev/null +++ b/pulsar-broker/src/test/resources/log4j2.xml @@ -0,0 +1,39 @@ + + + + + + + + + + + + + + + + + + diff --git a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/Consumer.java b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/Consumer.java index c67ad08c83631..f437b28fe1f8c 100644 --- a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/Consumer.java +++ b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/Consumer.java @@ -474,6 +474,9 @@ CompletableFuture reconsumeLaterCumulativeAsync(Message message, *

  • MessageId.earliest : Reset the subscription on the earliest message available in the topic *
  • MessageId.latest : Reset the subscription on the latest message in the topic * + *

    + * This effectively resets the acknowledgement state of the subscription: all messages up to and + * including messageId will be marked as acknowledged and the rest unacknowledged. * *

    Note: For multi-topics consumer, if `messageId` is a {@link TopicMessageId}, the seek operation will happen * on the owner topic of the message, which is returned by {@link TopicMessageId#getOwnerTopic()}. Otherwise, you From 9415132e1bbd40a9f51e599182aafe8d7da82b5a Mon Sep 17 00:00:00 2001 From: Asaf Mesika Date: Thu, 28 Dec 2023 15:11:55 +0200 Subject: [PATCH 5/9] Fixes build failures --- .../persistent/PersistentTopicMetrics.java | 18 ++++++++++++++++++ .../prometheus/metrics/PrometheusLabels.java | 18 ++++++++++++++++++ .../prometheus/PrometheusMetricsClient.java | 18 ++++++++++++++++++ 3 files changed, 54 insertions(+) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopicMetrics.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopicMetrics.java index f89f730bcc184..f79d053a9790d 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopicMetrics.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopicMetrics.java @@ -1,3 +1,21 @@ +/* + * 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.broker.service.persistent; import java.util.concurrent.atomic.LongAdder; diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/metrics/PrometheusLabels.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/metrics/PrometheusLabels.java index 558f8946a497e..9a2c520731468 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/metrics/PrometheusLabels.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/metrics/PrometheusLabels.java @@ -1,3 +1,21 @@ +/* + * 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.broker.stats.prometheus.metrics; import org.apache.pulsar.common.policies.data.BacklogQuota.BacklogQuotaType; diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsClient.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsClient.java index 77538bf619c0d..8d78f48b8872a 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsClient.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsClient.java @@ -1,3 +1,21 @@ +/* + * 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.broker.stats.prometheus; import static org.assertj.core.api.Assertions.assertThat; From a2349e0f73e53cf565d08a9f93e7947c51530e12 Mon Sep 17 00:00:00 2001 From: Asaf Mesika Date: Thu, 28 Dec 2023 16:47:05 +0200 Subject: [PATCH 6/9] Fixes build failures --- .../broker/stats/prometheus/NamespaceStatsAggregatorTest.java | 3 +++ 1 file changed, 3 insertions(+) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/prometheus/NamespaceStatsAggregatorTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/prometheus/NamespaceStatsAggregatorTest.java index e63f644f3d0e9..cf923df0411dd 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/prometheus/NamespaceStatsAggregatorTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/prometheus/NamespaceStatsAggregatorTest.java @@ -32,6 +32,7 @@ import org.apache.pulsar.broker.service.Subscription; import org.apache.pulsar.broker.service.Topic; import org.apache.pulsar.broker.service.persistent.PersistentTopic; +import org.apache.pulsar.broker.service.persistent.PersistentTopicMetrics; import org.apache.pulsar.common.policies.data.BacklogQuota; import org.apache.pulsar.common.policies.data.stats.ConsumerStatsImpl; import org.apache.pulsar.common.policies.data.stats.SubscriptionStatsImpl; @@ -102,6 +103,8 @@ public void testGenerateSubscriptionsStats() { when(topic.getReplicators()).thenReturn(ConcurrentOpenHashMap.newBuilder().build()); when(topic.getManagedLedger()).thenReturn(ml); when(topic.getBacklogQuota(Mockito.any())).thenReturn(Mockito.mock(BacklogQuota.class)); + PersistentTopicMetrics persistentTopicMetrics = new PersistentTopicMetrics(); + when(topic.getPersistentTopicMetrics()).thenReturn(persistentTopicMetrics); topicsMap.put("my-topic", topic); PrometheusMetricStreams metricStreams = Mockito.spy(new PrometheusMetricStreams()); From 8f4ef28cd5a83e06b7f62156afd58994553e07ec Mon Sep 17 00:00:00 2001 From: Asaf Mesika Date: Tue, 2 Jan 2024 13:59:57 +0200 Subject: [PATCH 7/9] PR Fixes --- .../mledger/impl/ManagedCursorContainer.java | 7 ++++++- .../service/persistent/PersistentTopic.java | 19 ++++++++++------- .../service/BacklogQuotaManagerTest.java | 21 +++++++------------ .../prometheus/PrometheusMetricsClient.java | 4 ---- 4 files changed, 24 insertions(+), 27 deletions(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorContainer.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorContainer.java index f6fb13457fd66..e6857e9855baf 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorContainer.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorContainer.java @@ -85,7 +85,12 @@ private DataVersion() {} /** * Compares two data versions, which either rolls overs to 0 when reaching Long.MAX_VALUE. *

    - * Use {@link DataVersion#incrementVersion(long)} to increment the versions + * Use {@link DataVersion#incrementVersion(long)} to increment the versions. The assumptions + * is that metric versios are compared with close time proximity one to another, hence, + * they are expected not close to each other in terms of distance, hence we don't + * expect the distance ever to exceed Long.MAX_VALUE / 2, otherwise we wouldn't be able + * to know which one is a later version in case the furthest rolls over to beyond 0. We + * assume the shortest distance between them dictates that. *

    * @param v1 First version to compare * @param v2 Second version to compare diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java index e78862b259bbc..22f857588484d 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java @@ -3233,13 +3233,16 @@ public CompletableFuture checkTimeBacklogExceeded() { TopicName topicName = TopicName.get(getName()); int backlogQuotaLimitInSecond = getBacklogQuota(BacklogQuotaType.message_age).getLimitTime(); + // If backlog quota by time is not set + if (backlogQuotaLimitInSecond <= 0) { + return CompletableFuture.completedFuture(false); + } + ManagedCursorContainer managedCursorContainer = (ManagedCursorContainer) ledger.getCursors(); CursorInfo oldestMarkDeleteCursorInfo = managedCursorContainer.getCursorWithOldestPosition(); - // If backlog quota by time is not set, and we have no durable cursor - // since `ledger.getCursors()` only managed durable cursors - if (backlogQuotaLimitInSecond <= 0 - || oldestMarkDeleteCursorInfo == null + // If we have no durable cursor since `ledger.getCursors()` only managed durable cursors + if (oldestMarkDeleteCursorInfo == null || oldestMarkDeleteCursorInfo.getPosition() == null) { return CompletableFuture.completedFuture(false); } @@ -3265,7 +3268,7 @@ public CompletableFuture checkTimeBacklogExceeded() { boolean expired = MessageImpl.isEntryExpired(backlogQuotaLimitInSecond, entryTimestamp); if (expired && log.isDebugEnabled()) { log.debug("(Using cache) Time based backlog quota exceeded, oldest entry in cursor {}'s backlog" - + "exceeded quota {}", lastCheckResult.getCursorName(), backlogQuotaLimitInSecond); + + " exceeded quota {}", lastCheckResult.getCursorName(), backlogQuotaLimitInSecond); } return CompletableFuture.completedFuture(expired); @@ -3293,7 +3296,7 @@ public void readEntryComplete(Entry entry, Object ctx) { boolean expired = MessageImpl.isEntryExpired(backlogQuotaLimitInSecond, entryTimestamp); if (expired && log.isDebugEnabled()) { log.debug("Time based backlog quota exceeded, oldest entry in cursor {}'s backlog" - + "exceeded quota {}", ledger.getSlowestConsumer().getName(), + + " exceeded quota {}", ledger.getSlowestConsumer().getName(), backlogQuotaLimitInSecond); } future.complete(expired); @@ -3363,8 +3366,8 @@ private CheckResult estimatedTimeBasedBacklogQuotaCheck(PositionImpl markDeleteP long estimateMsgAgeMs = managedLedger.getClock().millis() - positionToCheckLedgerInfo.getTimestamp(); boolean shouldTruncateBacklog = estimateMsgAgeMs > SECONDS.toMillis(backlogQuotaLimitInSecond); if (log.isDebugEnabled()) { - log.debug("Time based backlog quota exceeded, quota {}, age of ledger " - + "slowest cursor currently on {}", backlogQuotaLimitInSecond * 1000, + log.debug("Time based backlog quota exceeded, quota {}[ms], age of ledger " + + "slowest cursor currently on {}[ms]", backlogQuotaLimitInSecond * 1000, estimateMsgAgeMs); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BacklogQuotaManagerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BacklogQuotaManagerTest.java index 5f85dca8f24fa..0031f63d4b373 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BacklogQuotaManagerTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BacklogQuotaManagerTest.java @@ -41,8 +41,6 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.CyclicBarrier; import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicLong; -import java.util.concurrent.atomic.AtomicReference; import lombok.Cleanup; import org.apache.bookkeeper.mledger.Position; import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; @@ -595,11 +593,10 @@ private void unloadAndLoadTopic(String topic, Producer producer) throws PulsarAd private void waitForQuotaCheckToRunTwice() { final long initialQuotaCheckCount = getQuotaCheckCount(); - AtomicLong quotaCheckCount = new AtomicLong(); - Awaitility.await().pollInterval(1, SECONDS).atMost(TIME_TO_CHECK_BACKLOG_QUOTA*3, SECONDS).until(() -> { - quotaCheckCount.set(getQuotaCheckCount()); - return quotaCheckCount.get() > initialQuotaCheckCount + 1; - }); + Awaitility.await() + .pollInterval(1, SECONDS) + .atMost(TIME_TO_CHECK_BACKLOG_QUOTA*3, SECONDS) + .until(() -> getQuotaCheckCount() > initialQuotaCheckCount + 1); } /** @@ -608,13 +605,9 @@ private void waitForQuotaCheckToRunTwice() { private String waitForMarkDeletePositionToChange(String topic, String subscriptionName, String previousMarkDeletePosition) { - AtomicReference markDeletePosition = new AtomicReference<>(); - Awaitility.await().pollInterval(1, SECONDS).atMost(5, SECONDS).until(() -> { - markDeletePosition.set( - admin.topics().getInternalStats(topic).cursors.get(subscriptionName).markDeletePosition); - return markDeletePosition.get() != null && !markDeletePosition.get().equals(previousMarkDeletePosition); - }); - return markDeletePosition.get(); + return Awaitility.await().pollInterval(1, SECONDS).atMost(5, SECONDS).until( + () -> admin.topics().getInternalStats(topic).cursors.get(subscriptionName).markDeletePosition, + markDeletePosition -> markDeletePosition != null && !markDeletePosition.equals(previousMarkDeletePosition)); } private long getQuotaCheckCount() { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsClient.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsClient.java index 8d78f48b8872a..6fd509690278d 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsClient.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsClient.java @@ -18,7 +18,6 @@ */ package org.apache.pulsar.broker.stats.prometheus; -import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Fail.fail; import static org.testng.Assert.assertTrue; import com.google.common.base.MoreObjects; @@ -149,9 +148,6 @@ public final Metric findSingleMetricByNameAndLabels(String metricName, Pair cursorUpdated(ManagedCursor cursor, Position newPosition) { requireNonNull(cursor); @@ -243,7 +241,7 @@ public Pair cursorUpdated(ManagedCursor cursor, Posi PositionImpl previousSlowestConsumer = heap.get(0).position; item.position = (PositionImpl) newPosition; - version = DataVersion.incrementVersion(version); + version = DataVersion.getNextVersion(version); if (heap.size() == 1) { return Pair.of(previousSlowestConsumer, item.position); diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorContainerTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorContainerTest.java index 2653f23cdcf6c..f0b3efe39d6b7 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorContainerTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorContainerTest.java @@ -752,20 +752,20 @@ public void testDataVersion() { assertThat(ManagedCursorContainer.DataVersion.compareVersions(3L, 3L)).isZero(); long v1 = Long.MAX_VALUE - 1; - long v2 = ManagedCursorContainer.DataVersion.incrementVersion(v1); + long v2 = ManagedCursorContainer.DataVersion.getNextVersion(v1); assertThat(ManagedCursorContainer.DataVersion.compareVersions(v1, v2)).isNegative(); - v2 = ManagedCursorContainer.DataVersion.incrementVersion(v2); + v2 = ManagedCursorContainer.DataVersion.getNextVersion(v2); assertThat(ManagedCursorContainer.DataVersion.compareVersions(v1, v2)).isNegative(); - v1 = ManagedCursorContainer.DataVersion.incrementVersion(v1); + v1 = ManagedCursorContainer.DataVersion.getNextVersion(v1); assertThat(ManagedCursorContainer.DataVersion.compareVersions(v1, v2)).isNegative(); - v1 = ManagedCursorContainer.DataVersion.incrementVersion(v1); + v1 = ManagedCursorContainer.DataVersion.getNextVersion(v1); assertThat(ManagedCursorContainer.DataVersion.compareVersions(v1, v2)).isZero(); - v1 = ManagedCursorContainer.DataVersion.incrementVersion(v1); + v1 = ManagedCursorContainer.DataVersion.getNextVersion(v1); assertThat(ManagedCursorContainer.DataVersion.compareVersions(v1, v2)).isPositive(); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java index e0d972cafd8aa..1610a65b4e5a7 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java @@ -20,6 +20,7 @@ import static com.google.common.base.Preconditions.checkArgument; import static java.util.Objects.requireNonNull; +import static java.util.concurrent.TimeUnit.MILLISECONDS; import static org.apache.bookkeeper.mledger.ManagedLedgerConfig.PROPERTY_SOURCE_TOPIC_KEY; import static org.apache.commons.collections4.CollectionUtils.isEmpty; import static org.apache.commons.lang3.StringUtils.isNotBlank; @@ -845,7 +846,7 @@ CompletableFuture shutdownEventLoopGracefully(EventLoopGroup eventLoopGrou long timeout = (long) (GRACEFUL_SHUTDOWN_TIMEOUT_RATIO_OF_TOTAL_TIMEOUT * brokerShutdownTimeoutMs); return NettyFutureUtil.toCompletableFutureVoid( eventLoopGroup.shutdownGracefully(quietPeriod, - timeout, TimeUnit.MILLISECONDS)); + timeout, MILLISECONDS)); } private CompletableFuture closeChannel(Channel channel) { @@ -899,8 +900,8 @@ public void unloadNamespaceBundlesGracefully(int maxConcurrentUnload, boolean cl rateLimiter.acquire(1); } long timeout = pulsar.getConfiguration().getNamespaceBundleUnloadingTimeoutMs(); - pulsar.getNamespaceService().unloadNamespaceBundle(su, timeout, TimeUnit.MILLISECONDS, - closeWithoutWaitingClientDisconnect).get(timeout, TimeUnit.MILLISECONDS); + pulsar.getNamespaceService().unloadNamespaceBundle(su, timeout, MILLISECONDS, + closeWithoutWaitingClientDisconnect).get(timeout, MILLISECONDS); } catch (Exception e) { log.warn("Failed to unload namespace bundle {}", su, e); } @@ -2073,29 +2074,31 @@ public BacklogQuotaManager getBacklogQuotaManager() { } public void monitorBacklogQuota() { - backlogQuotaCheckDuration.time(() -> { - forEachPersistentTopic(topic -> { - if (topic.isSizeBacklogExceeded()) { - getBacklogQuotaManager().handleExceededBacklogQuota(topic, - BacklogQuota.BacklogQuotaType.destination_storage, false); - } else { - topic.checkTimeBacklogExceeded().thenAccept(isExceeded -> { - if (isExceeded) { - getBacklogQuotaManager().handleExceededBacklogQuota(topic, - BacklogQuota.BacklogQuotaType.message_age, - pulsar.getConfiguration().isPreciseTimeBasedBacklogQuotaCheck()); - } else { - if (log.isDebugEnabled()) { - log.debug("quota not exceeded for [{}]", topic.getName()); - } + long startTimeMillis = System.currentTimeMillis(); + forEachPersistentTopic(topic -> { + if (topic.isSizeBacklogExceeded()) { + getBacklogQuotaManager().handleExceededBacklogQuota(topic, + BacklogQuota.BacklogQuotaType.destination_storage, false); + } else { + topic.checkTimeBacklogExceeded().thenAccept(isExceeded -> { + if (isExceeded) { + getBacklogQuotaManager().handleExceededBacklogQuota(topic, + BacklogQuota.BacklogQuotaType.message_age, + pulsar.getConfiguration().isPreciseTimeBasedBacklogQuotaCheck()); + } else { + if (log.isDebugEnabled()) { + log.debug("quota not exceeded for [{}]", topic.getName()); } - }).exceptionally(throwable -> { - log.error("Error when checkTimeBacklogExceeded({}) in monitorBacklogQuota", - topic.getName(), throwable); - return null; - }); - } - }); + } + }).exceptionally(throwable -> { + log.error("Error when checkTimeBacklogExceeded({}) in monitorBacklogQuota", + topic.getName(), throwable); + return null; + }).whenComplete((unused, throwable) -> { + backlogQuotaCheckDuration.observe( + MILLISECONDS.toSeconds(System.currentTimeMillis() - startTimeMillis)); + }); + } }); } @@ -2589,7 +2592,7 @@ private void updateConfigurationAndRegisterListeners() { // add listener to notify broker managedLedgerCacheEvictionTimeThresholdMillis dynamic config registerConfigurationListener( "managedLedgerCacheEvictionTimeThresholdMillis", (cacheEvictionTimeThresholdMills) -> { - managedLedgerFactory.updateCacheEvictionTimeThreshold(TimeUnit.MILLISECONDS + managedLedgerFactory.updateCacheEvictionTimeThreshold(MILLISECONDS .toNanos((long) cacheEvictionTimeThresholdMills)); }); @@ -3008,7 +3011,7 @@ private void createPendingLoadTopic() { pendingTopic.getTopicFuture() .completeExceptionally((e instanceof RuntimeException && e.getCause() != null) ? e.getCause() : e); // schedule to process next pending topic - inactivityMonitor.schedule(this::createPendingLoadTopic, 100, TimeUnit.MILLISECONDS); + inactivityMonitor.schedule(this::createPendingLoadTopic, 100, MILLISECONDS); return null; }); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopic.java index ff14a34cd9983..d49777e88c00c 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopic.java @@ -65,7 +65,6 @@ import org.apache.pulsar.broker.service.Topic; import org.apache.pulsar.broker.service.TopicPolicyListener; import org.apache.pulsar.broker.service.TransportCnx; -import org.apache.pulsar.broker.service.persistent.PersistentTopic; import org.apache.pulsar.broker.stats.ClusterReplicationMetrics; import org.apache.pulsar.broker.stats.NamespaceStats; import org.apache.pulsar.client.api.MessageId; @@ -1265,6 +1264,6 @@ public boolean isPersistent() { @Override public long getBestEffortOldestUnacknowledgedMessageAgeSeconds() { - return PersistentTopic.NOT_AVAILABLE_YET; + return -1; } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java index 975de6858ba02..ec86624869f2f 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java @@ -196,8 +196,6 @@ public class PersistentTopic extends AbstractTopic implements Topic, AddEntryCallback { - public static final long NOT_AVAILABLE_YET = -1; - // Managed ledger associated with the topic protected final ManagedLedger ledger; @@ -293,7 +291,7 @@ private static class TimeBasedBacklogQuotaCheckResult { } @Value - private static class CheckResult { + private static class EstimateTimeBasedBacklogQuotaCheckResult { boolean truncateBacklogToMatchQuota; Long estimatedOldestUnacknowledgedMessageTimestamp; } @@ -2451,8 +2449,8 @@ public CompletableFuture asyncGetStats(GetStatsOptions TimeBasedBacklogQuotaCheckResult backlogQuotaCheckResult = timeBasedBacklogQuotaCheckResult; stats.oldestBacklogMessageAgeSeconds = (backlogQuotaCheckResult == null) - ? NOT_AVAILABLE_YET - : TimeUnit.MILLISECONDS.toSeconds( + ? (long) -1 + : TimeUnit.MILLISECONDS.toSeconds( Clock.systemUTC().millis() - backlogQuotaCheckResult.getPositionPublishTimestampInMillis()); stats.oldestBacklogMessageSubscriptionName = (backlogQuotaCheckResult == null) @@ -3230,7 +3228,7 @@ public boolean isSizeBacklogExceeded() { public long getBestEffortOldestUnacknowledgedMessageAgeSeconds() { TimeBasedBacklogQuotaCheckResult result = timeBasedBacklogQuotaCheckResult; if (result == null) { - return NOT_AVAILABLE_YET; + return -1; } else { return TimeUnit.MILLISECONDS.toSeconds( Clock.systemUTC().millis() - result.getPositionPublishTimestampInMillis()); @@ -3295,7 +3293,7 @@ public CompletableFuture checkTimeBacklogExceeded() { log.debug("(Using cache) Time based backlog quota exceeded, oldest entry in cursor {}'s backlog" + " exceeded quota {}", lastCheckResult.getCursorName(), backlogQuotaLimitInSecond); } - + persistentTopicMetrics.getBacklogQuotaMetrics().recordTimeBasedBacklogQuotaCheckReadFromCache(); return CompletableFuture.completedFuture(expired); } @@ -3343,7 +3341,7 @@ public void readEntryFailed(ManagedLedgerException exception, Object ctx) { return future; } else { try { - CheckResult checkResult = estimatedTimeBasedBacklogQuotaCheck(oldestMarkDeletePosition); + EstimateTimeBasedBacklogQuotaCheckResult checkResult = estimatedTimeBasedBacklogQuotaCheck(oldestMarkDeletePosition); if (checkResult.getEstimatedOldestUnacknowledgedMessageTimestamp() != null) { updateResultIfNewer( new TimeBasedBacklogQuotaCheckResult( @@ -3361,7 +3359,7 @@ public void readEntryFailed(ManagedLedgerException exception, Object ctx) { } } - private CheckResult estimatedTimeBasedBacklogQuotaCheck(PositionImpl markDeletePosition) + private EstimateTimeBasedBacklogQuotaCheckResult estimatedTimeBasedBacklogQuotaCheck(PositionImpl markDeletePosition) throws ExecutionException, InterruptedException { int backlogQuotaLimitInSecond = getBacklogQuota(BacklogQuotaType.message_age).getLimitTime(); ManagedLedgerImpl managedLedger = (ManagedLedgerImpl) ledger; @@ -3369,7 +3367,7 @@ private CheckResult estimatedTimeBasedBacklogQuotaCheck(PositionImpl markDeleteP // The ledger timestamp is only known when ledger is closed, hence when the mark-delete // is at active ledger (open) we can't estimate it. if (managedLedger.getLedgersInfo().lastKey().equals(markDeletePosition.getLedgerId())) { - return new CheckResult(false, null); + return new EstimateTimeBasedBacklogQuotaCheckResult(false, null); } org.apache.bookkeeper.mledger.proto.MLDataFormats.ManagedLedgerInfo.LedgerInfo @@ -3396,9 +3394,9 @@ private CheckResult estimatedTimeBasedBacklogQuotaCheck(PositionImpl markDeleteP estimateMsgAgeMs); } - return new CheckResult(shouldTruncateBacklog, positionToCheckLedgerInfo.getTimestamp()); + return new EstimateTimeBasedBacklogQuotaCheckResult(shouldTruncateBacklog, positionToCheckLedgerInfo.getTimestamp()); } else { - return new CheckResult(false, null); + return new EstimateTimeBasedBacklogQuotaCheckResult(false, null); } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopicMetrics.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopicMetrics.java index f79d053a9790d..afaafe748cabb 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopicMetrics.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopicMetrics.java @@ -30,6 +30,7 @@ public class PersistentTopicMetrics { public static class BacklogQuotaMetrics { private final LongAdder timeBasedBacklogQuotaExceededEvictionCount = new LongAdder(); private final LongAdder sizeBasedBacklogQuotaExceededEvictionCount = new LongAdder(); + private final LongAdder timeBasedBacklogQuotaCheckReadFromCache = new LongAdder(); public void recordTimeBasedBacklogEviction() { timeBasedBacklogQuotaExceededEvictionCount.increment(); @@ -39,6 +40,10 @@ public void recordSizeBasedBacklogEviction() { sizeBasedBacklogQuotaExceededEvictionCount.increment(); } + public void recordTimeBasedBacklogQuotaCheckReadFromCache() { + timeBasedBacklogQuotaCheckReadFromCache.increment(); + } + public long getSizeBasedBacklogQuotaExceededEvictionCount() { return sizeBasedBacklogQuotaExceededEvictionCount.longValue(); } @@ -46,5 +51,9 @@ public long getSizeBasedBacklogQuotaExceededEvictionCount() { public long getTimeBasedBacklogQuotaExceededEvictionCount() { return timeBasedBacklogQuotaExceededEvictionCount.longValue(); } + + public long getTimeBasedBacklogQuotaCheckReadFromCache() { + return timeBasedBacklogQuotaCheckReadFromCache.longValue(); + } } } \ No newline at end of file diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BacklogQuotaManagerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BacklogQuotaManagerTest.java index 0031f63d4b373..9d243aae7d41c 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BacklogQuotaManagerTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BacklogQuotaManagerTest.java @@ -399,11 +399,9 @@ public void backlogsStatsPrecise() throws PulsarAdminException, PulsarClientExce Producer producer = createProducer(client, topic1); byte[] content = new byte[1024]; - List messageIds = new ArrayList<>(numMsgs); for (int i = 0; i < numMsgs; i++) { Thread.sleep(1000); - MessageId messageId = producer.send(content); - messageIds.add(messageId); + producer.send(content); } String c1MarkDeletePositionBefore = @@ -467,6 +465,8 @@ public void backlogsStatsPrecise() throws PulsarAdminException, PulsarClientExce topicStats = getTopicStats(topic1); assertThat(topicStats.getOldestBacklogMessageSubscriptionName()).isEqualTo(subName1); + long cacheUsedCounterBefore = getCacheUsedCounter(topic1); + // Move subscription 1 passed subscription 2 for (int i = 0; i < 3; i++) { Message message = consumer1.receive(); @@ -478,13 +478,28 @@ public void backlogsStatsPrecise() throws PulsarAdminException, PulsarClientExce waitForMarkDeletePositionToChange(topic1, subName1, c1MarkDeletePositionBefore); waitForQuotaCheckToRunTwice(); + // Cache shouldn't be used, since position has changed + assertThat(getCacheUsedCounter(topic1)).isEqualTo(cacheUsedCounterBefore); + topicStats = getTopicStats(topic1); expectedMessageAgeSeconds = MILLISECONDS.toSeconds(System.currentTimeMillis() - secondOldestMessage.getPublishTime()); assertThat(topicStats.getOldestBacklogMessageAgeSeconds()).isCloseTo(expectedMessageAgeSeconds, within(1L)); assertThat(topicStats.getOldestBacklogMessageSubscriptionName()).isEqualTo(subName2); + + cacheUsedCounterBefore = getCacheUsedCounter(topic1); + + waitForQuotaCheckToRunTwice(); + + // Cache should be used, since position hasn't changed + assertThat(getCacheUsedCounter(topic1)).isGreaterThan(cacheUsedCounterBefore); } } + private long getCacheUsedCounter(String topic1) { + return ((PersistentTopic) pulsar.getBrokerService().getTopicReference(topic1).get()) + .getPersistentTopicMetrics().getBacklogQuotaMetrics().getTimeBasedBacklogQuotaCheckReadFromCache(); + } + @Test public void backlogsStatsNotPrecise() throws PulsarAdminException, PulsarClientException, InterruptedException { config.setPreciseTimeBasedBacklogQuotaCheck(false); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ServerCnxTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ServerCnxTest.java index b6dd42d702860..e195f220f87dd 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ServerCnxTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ServerCnxTest.java @@ -21,6 +21,7 @@ import static org.apache.pulsar.broker.BrokerTestUtil.spyWithClassAndConstructorArgs; import static org.apache.pulsar.broker.BrokerTestUtil.spyWithClassAndConstructorArgsRecordingInvocations; +import static org.assertj.core.api.Assertions.assertThat; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyBoolean; import static org.mockito.ArgumentMatchers.anyInt; @@ -1844,14 +1845,14 @@ public void testBrokerClosedProducerClientRecreatesProducerThenSendCommand() thr ByteBuf clientCommand1 = Commands.newProducer(successTopicName, 1 /* producer id */, 1 /* request id */, producerName, Collections.emptyMap(), false); channel.writeInbound(clientCommand1); - assertTrue(getResponse() instanceof CommandProducerSuccess); + assertThat(getResponse()).isInstanceOf(CommandProducerSuccess.class); // Call disconnect method on producer to trigger activity similar to unloading Producer producer = serverCnx.getProducers().get(1).get(); assertNotNull(producer); producer.disconnect(); channel.runPendingTasks(); - assertTrue(getResponse() instanceof CommandCloseProducer); + assertThat(getResponse()).isInstanceOf(CommandCloseProducer.class); // Send message and expect no response sendMessage(); From 8953ffa53468fd71c4832d386261dcbe40b6c269 Mon Sep 17 00:00:00 2001 From: Asaf Mesika Date: Tue, 9 Jan 2024 11:34:52 +0200 Subject: [PATCH 9/9] Found a different, less flaky way to test if cache was used. --- .../mledger/ManagedLedgerMXBean.java | 5 ++ .../mledger/impl/ManagedLedgerImpl.java | 3 +- .../mledger/impl/ManagedLedgerMBeanImpl.java | 12 ++++- .../service/persistent/PersistentTopic.java | 51 +++++++++++++++---- .../persistent/PersistentTopicMetrics.java | 9 ---- .../service/BacklogQuotaManagerTest.java | 27 +++++----- pulsar-broker/src/test/resources/log4j2.xml | 4 +- 7 files changed, 74 insertions(+), 37 deletions(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerMXBean.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerMXBean.java index 50a3ffb157961..cb6d3700afe3a 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerMXBean.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerMXBean.java @@ -90,6 +90,11 @@ public interface ManagedLedgerMXBean { */ long getAddEntryErrors(); + /** + * @return the number of entries read from the managed ledger (from cache or BK) + */ + long getEntriesReadTotalCount(); + /** * @return the number of readEntries requests that succeeded */ diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java index 4084d7004a80d..12a191dda86da 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java @@ -323,7 +323,7 @@ public enum PositionBound { /** * This variable is used for testing the tests. - * {@link ManagedLedgerTest#testManagedLedgerWithPlacementPolicyInCustomMetadata()} + * ManagedLedgerTest#testManagedLedgerWithPlacementPolicyInCustomMetadata() */ @VisibleForTesting Map createdLedgerCustomMetadata; @@ -2129,6 +2129,7 @@ private void internalReadFromLedger(ReadHandle ledger, OpReadEntry opReadEntry) } protected void asyncReadEntry(ReadHandle ledger, PositionImpl position, ReadEntryCallback callback, Object ctx) { + mbean.addEntriesRead(1); if (config.getReadEntryTimeoutSeconds() > 0) { // set readOpCount to uniquely validate if ReadEntryCallbackWrapper is already recycled long readOpCount = READ_OP_COUNT_UPDATER.incrementAndGet(this); diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerMBeanImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerMBeanImpl.java index e057dee99538e..3935828ff3d80 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerMBeanImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerMBeanImpl.java @@ -41,6 +41,7 @@ public class ManagedLedgerMBeanImpl implements ManagedLedgerMXBean { private final Rate readEntriesOpsFailed = new Rate(); private final Rate readEntriesOpsCacheMisses = new Rate(); private final Rate markDeleteOps = new Rate(); + private final Rate entriesRead = new Rate(); private final LongAdder dataLedgerOpenOp = new LongAdder(); private final LongAdder dataLedgerCloseOp = new LongAdder(); @@ -80,6 +81,7 @@ public void refreshStats(long period, TimeUnit unit) { ledgerAddEntryLatencyStatsUsec.refresh(); ledgerSwitchLatencyStatsUsec.refresh(); entryStats.refresh(); + entriesRead.calculateRate(seconds); } public void addAddEntrySample(long size) { @@ -120,6 +122,10 @@ public void addReadEntriesSample(int count, long totalSize) { readEntriesOps.recordMultipleEvents(count, totalSize); } + public void addEntriesRead(int count) { + entriesRead.recordEvent(count); + } + public void startDataLedgerOpenOp() { dataLedgerOpenOp.increment(); } @@ -189,6 +195,11 @@ public String getName() { return managedLedger.getName(); } + @Override + public long getEntriesReadTotalCount() { + return entriesRead.getTotalCount(); + } + @Override public double getAddEntryMessagesRate() { return addEntryOps.getRate(); @@ -333,5 +344,4 @@ public PendingBookieOpsStats getPendingBookieOpsStats() { result.cursorLedgerDeleteOp = cursorLedgerDeleteOp.longValue(); return result; } - } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java index ec86624869f2f..4ac55426e5d9c 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java @@ -3255,6 +3255,9 @@ private void updateResultIfNewer(TimeBasedBacklogQuotaCheckResult updatedResult) public CompletableFuture checkTimeBacklogExceeded() { TopicName topicName = TopicName.get(getName()); int backlogQuotaLimitInSecond = getBacklogQuota(BacklogQuotaType.message_age).getLimitTime(); + if (log.isDebugEnabled()) { + log.debug("[{}] Time backlog quota = [{}]. Checking if exceeded.", topicName, backlogQuotaLimitInSecond); + } // If backlog quota by time is not set if (backlogQuotaLimitInSecond <= 0) { @@ -3267,6 +3270,10 @@ public CompletableFuture checkTimeBacklogExceeded() { // If we have no durable cursor since `ledger.getCursors()` only managed durable cursors if (oldestMarkDeleteCursorInfo == null || oldestMarkDeleteCursorInfo.getPosition() == null) { + if (log.isDebugEnabled()) { + log.debug("[{}] No durable cursor found. Skipping time based backlog quota check." + + " Oldest mark-delete cursor info: {}", topicName, oldestMarkDeleteCursorInfo); + } return CompletableFuture.completedFuture(false); } @@ -3285,15 +3292,23 @@ public CompletableFuture checkTimeBacklogExceeded() { oldestMarkDeleteCursorInfo.getVersion()); updateResultIfNewer(updatedResult); + if (log.isDebugEnabled()) { + log.debug("[{}] Time-based backlog quota check. Updating cached result for position {}, " + + "since cursor causing it has changed from {} to {}", + topicName, + oldestMarkDeletePosition, + lastCheckResult.getCursorName(), + oldestMarkDeleteCursorInfo.getCursor().getName()); + } } long entryTimestamp = lastCheckResult.getPositionPublishTimestampInMillis(); boolean expired = MessageImpl.isEntryExpired(backlogQuotaLimitInSecond, entryTimestamp); - if (expired && log.isDebugEnabled()) { - log.debug("(Using cache) Time based backlog quota exceeded, oldest entry in cursor {}'s backlog" - + " exceeded quota {}", lastCheckResult.getCursorName(), backlogQuotaLimitInSecond); + if (log.isDebugEnabled()) { + log.debug("[{}] Time based backlog quota check. Using cache result for position {}. " + + "Entry timestamp: {}, expired: {}", + topicName, oldestMarkDeletePosition, entryTimestamp, expired); } - persistentTopicMetrics.getBacklogQuotaMetrics().recordTimeBasedBacklogQuotaCheckReadFromCache(); return CompletableFuture.completedFuture(expired); } @@ -3317,10 +3332,20 @@ public void readEntryComplete(Entry entry, Object ctx) { oldestMarkDeleteCursorInfo.getVersion())); boolean expired = MessageImpl.isEntryExpired(backlogQuotaLimitInSecond, entryTimestamp); - if (expired && log.isDebugEnabled()) { - log.debug("Time based backlog quota exceeded, oldest entry in cursor {}'s backlog" - + " exceeded quota {}", ledger.getSlowestConsumer().getName(), - backlogQuotaLimitInSecond); + if (log.isDebugEnabled()) { + log.debug("[{}] Time based backlog quota check. Oldest unacked entry read from BK. " + + "Oldest entry in cursor {}'s backlog: {}. " + + "Oldest mark-delete position: {}. " + + "Quota {}. Last check result position [{}]. " + + "Expired: {}, entryTimestamp: {}", + topicName, + oldestMarkDeleteCursorInfo.getCursor().getName(), + position, + oldestMarkDeletePosition, + backlogQuotaLimitInSecond, + lastCheckResult.getOldestCursorMarkDeletePosition(), + expired, + entryTimestamp); } future.complete(expired); } catch (Exception e) { @@ -3341,7 +3366,8 @@ public void readEntryFailed(ManagedLedgerException exception, Object ctx) { return future; } else { try { - EstimateTimeBasedBacklogQuotaCheckResult checkResult = estimatedTimeBasedBacklogQuotaCheck(oldestMarkDeletePosition); + EstimateTimeBasedBacklogQuotaCheckResult checkResult = + estimatedTimeBasedBacklogQuotaCheck(oldestMarkDeletePosition); if (checkResult.getEstimatedOldestUnacknowledgedMessageTimestamp() != null) { updateResultIfNewer( new TimeBasedBacklogQuotaCheckResult( @@ -3359,7 +3385,8 @@ public void readEntryFailed(ManagedLedgerException exception, Object ctx) { } } - private EstimateTimeBasedBacklogQuotaCheckResult estimatedTimeBasedBacklogQuotaCheck(PositionImpl markDeletePosition) + private EstimateTimeBasedBacklogQuotaCheckResult estimatedTimeBasedBacklogQuotaCheck( + PositionImpl markDeletePosition) throws ExecutionException, InterruptedException { int backlogQuotaLimitInSecond = getBacklogQuota(BacklogQuotaType.message_age).getLimitTime(); ManagedLedgerImpl managedLedger = (ManagedLedgerImpl) ledger; @@ -3394,7 +3421,9 @@ private EstimateTimeBasedBacklogQuotaCheckResult estimatedTimeBasedBacklogQuotaC estimateMsgAgeMs); } - return new EstimateTimeBasedBacklogQuotaCheckResult(shouldTruncateBacklog, positionToCheckLedgerInfo.getTimestamp()); + return new EstimateTimeBasedBacklogQuotaCheckResult( + shouldTruncateBacklog, + positionToCheckLedgerInfo.getTimestamp()); } else { return new EstimateTimeBasedBacklogQuotaCheckResult(false, null); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopicMetrics.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopicMetrics.java index afaafe748cabb..f79d053a9790d 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopicMetrics.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopicMetrics.java @@ -30,7 +30,6 @@ public class PersistentTopicMetrics { public static class BacklogQuotaMetrics { private final LongAdder timeBasedBacklogQuotaExceededEvictionCount = new LongAdder(); private final LongAdder sizeBasedBacklogQuotaExceededEvictionCount = new LongAdder(); - private final LongAdder timeBasedBacklogQuotaCheckReadFromCache = new LongAdder(); public void recordTimeBasedBacklogEviction() { timeBasedBacklogQuotaExceededEvictionCount.increment(); @@ -40,10 +39,6 @@ public void recordSizeBasedBacklogEviction() { sizeBasedBacklogQuotaExceededEvictionCount.increment(); } - public void recordTimeBasedBacklogQuotaCheckReadFromCache() { - timeBasedBacklogQuotaCheckReadFromCache.increment(); - } - public long getSizeBasedBacklogQuotaExceededEvictionCount() { return sizeBasedBacklogQuotaExceededEvictionCount.longValue(); } @@ -51,9 +46,5 @@ public long getSizeBasedBacklogQuotaExceededEvictionCount() { public long getTimeBasedBacklogQuotaExceededEvictionCount() { return timeBasedBacklogQuotaExceededEvictionCount.longValue(); } - - public long getTimeBasedBacklogQuotaCheckReadFromCache() { - return timeBasedBacklogQuotaCheckReadFromCache.longValue(); - } } } \ No newline at end of file diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BacklogQuotaManagerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BacklogQuotaManagerTest.java index 9d243aae7d41c..e24fb493b954a 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BacklogQuotaManagerTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BacklogQuotaManagerTest.java @@ -388,7 +388,7 @@ public void backlogsStatsPrecise() throws PulsarAdminException, PulsarClientExce final String subName1 = "c1"; final String subName2 = "c2"; - final int numMsgs = 5; + final int numMsgs = 4; Consumer consumer1 = client.newConsumer().topic(topic1).subscriptionName(subName1) .acknowledgmentGroupTime(0, SECONDS) @@ -400,7 +400,7 @@ public void backlogsStatsPrecise() throws PulsarAdminException, PulsarClientExce byte[] content = new byte[1024]; for (int i = 0; i < numMsgs; i++) { - Thread.sleep(1000); + Thread.sleep(3000); // Guarantees if we use wrong message in age, to show up in failed test producer.send(content); } @@ -436,13 +436,15 @@ public void backlogsStatsPrecise() throws PulsarAdminException, PulsarClientExce entry("cluster", CLUSTER_NAME), entry("namespace", namespace), entry("topic", topic1)); - assertThat((long) backlogAgeMetric.value).isCloseTo(expectedMessageAgeSeconds, within(1L)); + assertThat((long) backlogAgeMetric.value).isCloseTo(expectedMessageAgeSeconds, within(2L)); // Move subscription 2 away from being the oldest mark delete // S2/S1 // 0 1 + Message firstOldestMessage = consumer2.receive(); + consumer2.acknowledge(firstOldestMessage); + // We only read and not ack, since we just need its publish-timestamp for later assert Message secondOldestMessage = consumer2.receive(); - consumer2.acknowledge(secondOldestMessage); // Switch subscription 1 to be where subscription 2 was in terms of oldest mark delete // S1 S2 @@ -460,12 +462,12 @@ public void backlogsStatsPrecise() throws PulsarAdminException, PulsarClientExce .get(0).value; expectedMessageAgeSeconds = MILLISECONDS.toSeconds(System.currentTimeMillis() - oldestMessage.getPublishTime()); - assertThat(actualAge).isCloseTo(expectedMessageAgeSeconds, within(1L)); + assertThat(actualAge).isCloseTo(expectedMessageAgeSeconds, within(2L)); topicStats = getTopicStats(topic1); assertThat(topicStats.getOldestBacklogMessageSubscriptionName()).isEqualTo(subName1); - long cacheUsedCounterBefore = getCacheUsedCounter(topic1); + long entriesReadBefore = getReadEntries(topic1); // Move subscription 1 passed subscription 2 for (int i = 0; i < 3; i++) { @@ -479,25 +481,24 @@ public void backlogsStatsPrecise() throws PulsarAdminException, PulsarClientExce waitForQuotaCheckToRunTwice(); // Cache shouldn't be used, since position has changed - assertThat(getCacheUsedCounter(topic1)).isEqualTo(cacheUsedCounterBefore); + long readEntries = getReadEntries(topic1); + assertThat(readEntries).isGreaterThan(entriesReadBefore); topicStats = getTopicStats(topic1); expectedMessageAgeSeconds = MILLISECONDS.toSeconds(System.currentTimeMillis() - secondOldestMessage.getPublishTime()); - assertThat(topicStats.getOldestBacklogMessageAgeSeconds()).isCloseTo(expectedMessageAgeSeconds, within(1L)); + assertThat(topicStats.getOldestBacklogMessageAgeSeconds()).isCloseTo(expectedMessageAgeSeconds, within(2L)); assertThat(topicStats.getOldestBacklogMessageSubscriptionName()).isEqualTo(subName2); - cacheUsedCounterBefore = getCacheUsedCounter(topic1); - waitForQuotaCheckToRunTwice(); // Cache should be used, since position hasn't changed - assertThat(getCacheUsedCounter(topic1)).isGreaterThan(cacheUsedCounterBefore); + assertThat(getReadEntries(topic1)).isEqualTo(readEntries); } } - private long getCacheUsedCounter(String topic1) { + private long getReadEntries(String topic1) { return ((PersistentTopic) pulsar.getBrokerService().getTopicReference(topic1).get()) - .getPersistentTopicMetrics().getBacklogQuotaMetrics().getTimeBasedBacklogQuotaCheckReadFromCache(); + .getManagedLedger().getStats().getEntriesReadTotalCount(); } @Test diff --git a/pulsar-broker/src/test/resources/log4j2.xml b/pulsar-broker/src/test/resources/log4j2.xml index 0d9244e932857..4038dd59b1d79 100644 --- a/pulsar-broker/src/test/resources/log4j2.xml +++ b/pulsar-broker/src/test/resources/log4j2.xml @@ -28,8 +28,8 @@ - - + +