From c50fa566bce873f18502d431ec2f35ac160bf2ca Mon Sep 17 00:00:00 2001 From: Xiangying Meng <55571188+liangyepianzhou@users.noreply.github.com> Date: Thu, 28 Nov 2024 19:33:21 +0800 Subject: [PATCH 01/22] [improve][pip] PIP-389: Add Producer config compressMinMsgBodySize to improve compression performance (#23526) Co-authored-by: xiangying --- pip/pip-389.md | 110 +++++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 110 insertions(+) create mode 100644 pip/pip-389.md diff --git a/pip/pip-389.md b/pip/pip-389.md new file mode 100644 index 0000000000000..9b23ce4341e8a --- /dev/null +++ b/pip/pip-389.md @@ -0,0 +1,110 @@ +# PIP-389: Add Producer config compressMinMsgBodySize to improve compression performance + +# Background knowledge +Pulsar provide a way to compress messages before sending them to the broker[0]. This can be done by setting the `compressionType` in the producer configuration. +The compressionType can be set to one of the following values: +- LZ4 +- ZLIB +- ZSTD +- SNAPPY + +But the compressionType is applied to all messages sent by the producer. This means that even small messages are compressed. + +In our test, we found that compressing small messages can is meaningless. The compression ratio is low and spend more cpu. +The relevant description in the official documentation: +>The smaller the amount of data to compress, the more difficult it is to compress. This problem is common to all compression algorithms. [1] + +The similar configuration in RocketMQ is `compressMsgBodyOverHowmuch`[2]: +>/** +>* Compress message body threshold, namely, message body larger than 4k will be compressed on default. +>*/ +>private int compressMsgBodyOverHowmuch = 1024 * 4; + +[0] https://pulsar.apache.org/docs/4.0.x/concepts-messaging/#compression +[1] https://github.com/facebook/zstd?tab=readme-ov-file#the-case-for-small-data-compression +[2] https://github.com/apache/rocketmq/blob/dd62ed0f3b16919adec5d5eece21a1050dc9c5a0/client/src/main/java/org/apache/rocketmq/client/producer/DefaultMQProducer.java#L117 + +# Motivation + +The motivation of this PIP is to provide a way to improve the compression performance by skipping the compression of small messages. +We want to add a new configuration `compressMinMsgBodySize` to the producer configuration. +This configuration will allow the user to set the minimum size of the message body that will be compressed. +If the message body size is less than the `compressMinMsgBodySize`, the message will not be compressed. + +# Goals + +## In Scope + +Add a new configuration `compressMinMsgBodySize` to the producer configuration. + +## Out of Scope + +Solve the compression problem of small data + +# High Level Design + +# Detailed Design + +## Design & Implementation Details + +Add a new configuration `compressMinMsgBodySize` to the producer configuration. +This configuration will allow the user to set the minimum size of the message body that will be compressed. +If the message body size is less than the `compressMinMsgBodySize`, the message will not be compressed. + +## Public-facing Changes + +Add a new configuration `compressMinMsgBodySize` to the producer configuration. + + +### Public API +NA +### Binary protocol + +### Configuration + +### CLI + +### Metrics + +NA + +# Monitoring + +NA + +# Security Considerations + +NA + +# Backward & Forward Compatibility + +## Upgrade + +This is a new feature, and it does not affect the existing configuration. + +## Downgrade / Rollback + +The new configuration `compressMinMsgBodySize` will to be removed from the producer configuration. +If you used it, you need to remove it manually. + +## Pulsar Geo-Replication Upgrade & Downgrade/Rollback Considerations + + + +# Alternatives + + + +# General Notes + +# Links + + +* Mailing List discussion thread: https://lists.apache.org/thread/vxvy7h61hg9wlgby6lcpkm9osdk9sx20 +* Mailing List voting thread: https://lists.apache.org/thread/xv7x3vmycxzsrhbdo7vmssh8lxxzyxd5 From 963be2c3290a4a6ef25b549b046039afc8690d4c Mon Sep 17 00:00:00 2001 From: Baodi Shi Date: Thu, 28 Nov 2024 21:44:15 +0800 Subject: [PATCH 02/22] [improve][broker] Decouple pulsar_storage_backlog_age_seconds metric with backlogQuota check (#23619) --- .../pulsar/broker/service/BrokerService.java | 44 +++-- .../service/persistent/PersistentTopic.java | 184 +++++++++--------- .../service/persistent/SystemTopic.java | 2 +- .../service/BacklogQuotaManagerTest.java | 133 ++++++++++++- 4 files changed, 241 insertions(+), 122 deletions(-) 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 77cd52f4558ea..6afa1ae32fbcb 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 @@ -2245,29 +2245,31 @@ public BacklogQuotaManager getBacklogQuotaManager() { public void monitorBacklogQuota() { 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()); + topic.updateOldPositionInfo().thenAccept(__ -> { + if (topic.isSizeBacklogExceeded()) { + getBacklogQuotaManager().handleExceededBacklogQuota(topic, + BacklogQuota.BacklogQuotaType.destination_storage, false); + } else { + topic.checkTimeBacklogExceeded(false).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", + }); + } + }).whenComplete((unused, throwable) -> { + if (throwable != null) { + log.error("Error when checkBacklogQuota({}) in monitorBacklogQuota", topic.getName(), throwable); - return null; - }).whenComplete((unused, throwable) -> { - backlogQuotaCheckDuration.observe( - MILLISECONDS.toSeconds(System.currentTimeMillis() - startTimeMillis)); - }); - } + } + backlogQuotaCheckDuration.observe( + MILLISECONDS.toSeconds(System.currentTimeMillis() - startTimeMillis)); + }); }); } 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 651d12373628b..eb48ceee72d76 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 @@ -295,14 +295,15 @@ protected TopicStatsHelper initialValue() { PERSISTENT_TOPIC_ATTRIBUTES_FIELD_UPDATER = AtomicReferenceFieldUpdater.newUpdater( PersistentTopic.class, PersistentTopicAttributes.class, "persistentTopicAttributes"); - private volatile TimeBasedBacklogQuotaCheckResult timeBasedBacklogQuotaCheckResult; - private static final AtomicReferenceFieldUpdater + // The topic's oldest position information, if null, indicates that there is no cursor or no backlog. + private volatile OldestPositionInfo oldestPositionInfo; + private static final AtomicReferenceFieldUpdater TIME_BASED_BACKLOG_QUOTA_CHECK_RESULT_UPDATER = AtomicReferenceFieldUpdater.newUpdater( PersistentTopic.class, - TimeBasedBacklogQuotaCheckResult.class, - "timeBasedBacklogQuotaCheckResult"); + OldestPositionInfo.class, + "oldestPositionInfo"); @Value - private static class TimeBasedBacklogQuotaCheckResult { + private static class OldestPositionInfo { Position oldestCursorMarkDeletePosition; String cursorName; long positionPublishTimestampInMillis; @@ -2634,12 +2635,10 @@ public CompletableFuture asyncGetStats(GetStatsOptions stats.backlogQuotaLimitSize = getBacklogQuota(BacklogQuotaType.destination_storage).getLimitSize(); stats.backlogQuotaLimitTime = getBacklogQuota(BacklogQuotaType.message_age).getLimitTime(); - TimeBasedBacklogQuotaCheckResult backlogQuotaCheckResult = timeBasedBacklogQuotaCheckResult; stats.oldestBacklogMessageAgeSeconds = getBestEffortOldestUnacknowledgedMessageAgeSeconds(); - stats.oldestBacklogMessageSubscriptionName = (backlogQuotaCheckResult == null) - || !hasBacklogs(getStatsOptions.isGetPreciseBacklog()) + stats.oldestBacklogMessageSubscriptionName = (oldestPositionInfo == null) ? null - : backlogQuotaCheckResult.getCursorName(); + : oldestPositionInfo.getCursorName(); stats.compaction.reset(); mxBean.flatMap(bean -> bean.getCompactionRecordForTopic(topic)).map(compactionRecord -> { @@ -3425,7 +3424,7 @@ public CompletableFuture checkBacklogQuotaExceeded(String producerName, Ba return FutureUtil.failedFuture(new TopicBacklogQuotaExceededException(retentionPolicy)); } if (backlogQuotaType == BacklogQuotaType.message_age) { - return checkTimeBacklogExceeded().thenCompose(isExceeded -> { + return checkTimeBacklogExceeded(true).thenCompose(isExceeded -> { if (isExceeded) { log.debug("[{}] Time backlog quota exceeded. Cannot create producer [{}]", this.getName(), producerName); @@ -3463,19 +3462,15 @@ public boolean isSizeBacklogExceeded() { @Override public long getBestEffortOldestUnacknowledgedMessageAgeSeconds() { - if (!hasBacklogs(false)) { - return 0; - } - TimeBasedBacklogQuotaCheckResult result = timeBasedBacklogQuotaCheckResult; - if (result == null) { + if (oldestPositionInfo == null) { return -1; } else { return TimeUnit.MILLISECONDS.toSeconds( - Clock.systemUTC().millis() - result.getPositionPublishTimestampInMillis()); + Clock.systemUTC().millis() - oldestPositionInfo.getPositionPublishTimestampInMillis()); } } - private void updateResultIfNewer(TimeBasedBacklogQuotaCheckResult updatedResult) { + private void updateResultIfNewer(OldestPositionInfo updatedResult) { TIME_BASED_BACKLOG_QUOTA_CHECK_RESULT_UPDATER.updateAndGet(this, existingResult -> { if (existingResult == null @@ -3489,74 +3484,56 @@ private void updateResultIfNewer(TimeBasedBacklogQuotaCheckResult updatedResult) } - /** - * @return determine if backlog quota enforcement needs to be done for topic based on time limit - */ - public CompletableFuture checkTimeBacklogExceeded() { + public CompletableFuture updateOldPositionInfo() { 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) { - return CompletableFuture.completedFuture(false); + if (!(ledger.getCursors() instanceof ManagedCursorContainer managedCursorContainer)) { + return CompletableFuture.failedFuture(new IllegalStateException( + String.format("[%s] No valid cursors found. Skip update old position info.", topicName))); } - ManagedCursorContainer managedCursorContainer = (ManagedCursorContainer) ledger.getCursors(); - CursorInfo oldestMarkDeleteCursorInfo = managedCursorContainer.getCursorWithOldestPosition(); + if (!hasBacklogs(brokerService.pulsar().getConfiguration().isPreciseTimeBasedBacklogQuotaCheck())) { + if (log.isDebugEnabled()) { + log.debug("[{}] No backlog. Update old position info is null", topicName); + } + TIME_BASED_BACKLOG_QUOTA_CHECK_RESULT_UPDATER.set(this, null); + return CompletableFuture.completedFuture(null); + } // If we have no durable cursor since `ledger.getCursors()` only managed durable cursors - if (oldestMarkDeleteCursorInfo == null - || oldestMarkDeleteCursorInfo.getPosition() == null) { + CursorInfo oldestMarkDeleteCursorInfo = managedCursorContainer.getCursorWithOldestPosition(); + 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); + log.debug("[{}] No durable cursor found. Update old position info is null", topicName); } - return CompletableFuture.completedFuture(false); + TIME_BASED_BACKLOG_QUOTA_CHECK_RESULT_UPDATER.set(this, null); + return CompletableFuture.completedFuture(null); } Position oldestMarkDeletePosition = oldestMarkDeleteCursorInfo.getPosition(); - - TimeBasedBacklogQuotaCheckResult lastCheckResult = timeBasedBacklogQuotaCheckResult; - if (lastCheckResult != null - && oldestMarkDeletePosition.compareTo(lastCheckResult.getOldestCursorMarkDeletePosition()) == 0) { - + OldestPositionInfo lastOldestPositionInfo = oldestPositionInfo; + if (lastOldestPositionInfo != null + && oldestMarkDeletePosition.compareTo(lastOldestPositionInfo.getOldestCursorMarkDeletePosition()) == 0) { // Same position, but the cursor causing it has changed? - if (!lastCheckResult.getCursorName().equals(oldestMarkDeleteCursorInfo.getCursor().getName())) { - final TimeBasedBacklogQuotaCheckResult updatedResult = new TimeBasedBacklogQuotaCheckResult( - lastCheckResult.getOldestCursorMarkDeletePosition(), + if (!lastOldestPositionInfo.getCursorName().equals(oldestMarkDeleteCursorInfo.getCursor().getName())) { + updateResultIfNewer(new OldestPositionInfo( + lastOldestPositionInfo.getOldestCursorMarkDeletePosition(), oldestMarkDeleteCursorInfo.getCursor().getName(), - lastCheckResult.getPositionPublishTimestampInMillis(), - oldestMarkDeleteCursorInfo.getVersion()); - - updateResultIfNewer(updatedResult); + lastOldestPositionInfo.getPositionPublishTimestampInMillis(), + oldestMarkDeleteCursorInfo.getVersion())); if (log.isDebugEnabled()) { - log.debug("[{}] Time-based backlog quota check. Updating cached result for position {}, " - + "since cursor causing it has changed from {} to {}", + log.debug("[{}] Updating cached old position info {}, " + + "since cursor causing it has changed from {} to {}", topicName, oldestMarkDeletePosition, - lastCheckResult.getCursorName(), + lastOldestPositionInfo.getCursorName(), oldestMarkDeleteCursorInfo.getCursor().getName()); } } - - long entryTimestamp = lastCheckResult.getPositionPublishTimestampInMillis(); - boolean expired = MessageImpl.isEntryExpired(backlogQuotaLimitInSecond, entryTimestamp); - if (log.isDebugEnabled()) { - log.debug("[{}] Time based backlog quota check. Using cache result for position {}. " - + "Entry timestamp: {}, expired: {}", - topicName, oldestMarkDeletePosition, entryTimestamp, expired); - } - return CompletableFuture.completedFuture(expired); + return CompletableFuture.completedFuture(null); } - if (brokerService.pulsar().getConfiguration().isPreciseTimeBasedBacklogQuotaCheck()) { - if (!hasBacklogs(true)) { - return CompletableFuture.completedFuture(false); - } - CompletableFuture future = new CompletableFuture<>(); + CompletableFuture future = new CompletableFuture<>(); // Check if first unconsumed message(first message after mark delete position) // for slowest cursor's has expired. Position position = ledger.getNextValidPosition(oldestMarkDeletePosition); @@ -3566,34 +3543,28 @@ public CompletableFuture checkTimeBacklogExceeded() { public void readEntryComplete(Entry entry, Object ctx) { try { long entryTimestamp = Commands.getEntryTimestamp(entry.getDataBuffer()); - updateResultIfNewer( - new TimeBasedBacklogQuotaCheckResult( - oldestMarkDeleteCursorInfo.getPosition(), - oldestMarkDeleteCursorInfo.getCursor().getName(), - entryTimestamp, - oldestMarkDeleteCursorInfo.getVersion())); - - boolean expired = MessageImpl.isEntryExpired(backlogQuotaLimitInSecond, entryTimestamp); + new OldestPositionInfo( + oldestMarkDeleteCursorInfo.getPosition(), + oldestMarkDeleteCursorInfo.getCursor().getName(), + entryTimestamp, + oldestMarkDeleteCursorInfo.getVersion())); if (log.isDebugEnabled()) { - log.debug("[{}] Time based backlog quota check. Oldest unacked entry read from BK. " + log.debug("[{}] Precise based update oldest position info. " + + "Oldest unacked entry read from BK. " + "Oldest entry in cursor {}'s backlog: {}. " + "Oldest mark-delete position: {}. " - + "Quota {}. Last check result position [{}]. " - + "Expired: {}, entryTimestamp: {}", + + "EntryTimestamp: {}", topicName, oldestMarkDeleteCursorInfo.getCursor().getName(), position, oldestMarkDeletePosition, - backlogQuotaLimitInSecond, - lastCheckResult.getOldestCursorMarkDeletePosition(), - expired, entryTimestamp); } - future.complete(expired); + future.complete(null); } catch (Exception e) { - log.error("[{}][{}] Error deserializing message for backlog check", topicName, e); - future.complete(false); + log.error("[{}][{}] Error deserializing message for update old position", topicName, e); + future.completeExceptionally(e); } finally { entry.release(); } @@ -3601,36 +3572,61 @@ public void readEntryComplete(Entry entry, Object ctx) { @Override public void readEntryFailed(ManagedLedgerException exception, Object ctx) { - log.error("[{}][{}] Error reading entry for precise time based backlog check", + log.error("[{}][{}] Error reading entry for precise update old position", topicName, exception); - future.complete(false); + future.completeExceptionally(exception); } }, null); return future; } else { try { - if (!hasBacklogs(false)) { - return CompletableFuture.completedFuture(false); - } EstimateTimeBasedBacklogQuotaCheckResult checkResult = estimatedTimeBasedBacklogQuotaCheck(oldestMarkDeletePosition); if (checkResult.getEstimatedOldestUnacknowledgedMessageTimestamp() != null) { updateResultIfNewer( - new TimeBasedBacklogQuotaCheckResult( - oldestMarkDeleteCursorInfo.getPosition(), - oldestMarkDeleteCursorInfo.getCursor().getName(), - checkResult.getEstimatedOldestUnacknowledgedMessageTimestamp(), - oldestMarkDeleteCursorInfo.getVersion())); + new OldestPositionInfo( + oldestMarkDeleteCursorInfo.getPosition(), + oldestMarkDeleteCursorInfo.getCursor().getName(), + checkResult.getEstimatedOldestUnacknowledgedMessageTimestamp(), + oldestMarkDeleteCursorInfo.getVersion())); } - return CompletableFuture.completedFuture(checkResult.isTruncateBacklogToMatchQuota()); + return CompletableFuture.completedFuture(null); } catch (Exception e) { - log.error("[{}][{}] Error reading entry for precise time based backlog check", topicName, e); - return CompletableFuture.completedFuture(false); + log.error("[{}][{}] Error reading entry for update old position", topicName, e); + return CompletableFuture.failedFuture(e); } } } + /** + * @return determine if backlog quota enforcement needs to be done for topic based on time limit + */ + public CompletableFuture checkTimeBacklogExceeded(boolean shouldUpdateOldPositionInfo) { + 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); + } + CompletableFuture updateFuture = shouldUpdateOldPositionInfo ? updateOldPositionInfo() + : CompletableFuture.completedFuture(null); + return updateFuture.thenCompose(__ -> { + if (backlogQuotaLimitInSecond <= 0) { + return CompletableFuture.completedFuture(false); + } + if (oldestPositionInfo == null) { + return CompletableFuture.completedFuture(false); + } + long entryTimestamp = oldestPositionInfo.getPositionPublishTimestampInMillis(); + boolean expired = MessageImpl.isEntryExpired(backlogQuotaLimitInSecond, entryTimestamp); + return CompletableFuture.completedFuture(expired); + }).exceptionally(e -> { + log.error("[{}][{}] Error checking time backlog exceeded", topicName, e); + return false; + }); + } + private EstimateTimeBasedBacklogQuotaCheckResult estimatedTimeBasedBacklogQuotaCheck( Position markDeletePosition) throws ExecutionException, InterruptedException { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/SystemTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/SystemTopic.java index 8feb432a08001..a26255c9f8bad 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/SystemTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/SystemTopic.java @@ -51,7 +51,7 @@ public boolean isSizeBacklogExceeded() { } @Override - public CompletableFuture checkTimeBacklogExceeded() { + public CompletableFuture checkTimeBacklogExceeded(boolean shouldUpdateOldPositionInfo) { return CompletableFuture.completedFuture(false); } 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 56f9f4f91246e..963dc3d26b28b 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 @@ -526,7 +526,7 @@ public void backlogsStatsPrecise() throws PulsarAdminException, PulsarClientExce assertThat(topicStats.getBacklogSize()).isEqualTo(0); assertThat(topicStats.getSubscriptions().get(subName1).getMsgBacklog()).isEqualTo(0); assertThat(topicStats.getSubscriptions().get(subName2).getMsgBacklog()).isEqualTo(0); - assertThat(topicStats.getOldestBacklogMessageAgeSeconds()).isEqualTo(0); + assertThat(topicStats.getOldestBacklogMessageAgeSeconds()).isEqualTo(-1); assertThat(topicStats.getOldestBacklogMessageSubscriptionName()).isNull(); metrics = prometheusMetricsClient.getMetrics(); @@ -537,7 +537,7 @@ public void backlogsStatsPrecise() throws PulsarAdminException, PulsarClientExce entry("cluster", CLUSTER_NAME), entry("namespace", namespace), entry("topic", topic1)); - assertThat((long) backlogAgeMetric.value).isEqualTo(0); + assertThat((long) backlogAgeMetric.value).isEqualTo(-1); // producer should create success. Producer producer2 = createProducer(client, topic1); @@ -598,7 +598,7 @@ public void backlogsStatsPreciseWithNoBacklog() throws PulsarAdminException, Pul assertThat(topicStats.getBacklogQuotaLimitTime()).isEqualTo(timeLimitSeconds); assertThat(topicStats.getBacklogSize()).isEqualTo(0); assertThat(topicStats.getSubscriptions().get(subName1).getMsgBacklog()).isEqualTo(0); - assertThat(topicStats.getOldestBacklogMessageAgeSeconds()).isEqualTo(0); + assertThat(topicStats.getOldestBacklogMessageAgeSeconds()).isEqualTo(-1); assertThat(topicStats.getOldestBacklogMessageSubscriptionName()).isNull(); Metric backlogAgeMetric = @@ -608,7 +608,7 @@ public void backlogsStatsPreciseWithNoBacklog() throws PulsarAdminException, Pul entry("cluster", CLUSTER_NAME), entry("namespace", namespace), entry("topic", topic1)); - assertThat((long) backlogAgeMetric.value).isEqualTo(0); + assertThat((long) backlogAgeMetric.value).isEqualTo(-1); // producer should create success. Producer producer2 = createProducer(client, topic1); @@ -618,6 +618,118 @@ public void backlogsStatsPreciseWithNoBacklog() throws PulsarAdminException, Pul config.setExposePreciseBacklogInPrometheus(false); } + @Test + public void backlogsAgeMetricsPreciseWithoutBacklogQuota() throws Exception { + config.setPreciseTimeBasedBacklogQuotaCheck(true); + final String namespace = "prop/ns-quota"; + assertEquals(admin.namespaces().getBacklogQuotaMap(namespace), new HashMap<>()); + + 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 = "c1"; + final String subName2 = "c2"; + final int numMsgs = 4; + + 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]; + for (int i = 0; i < numMsgs; i++) { + Thread.sleep(3000); // Guarantees if we use wrong message in age, to show up in failed test + producer.send(content); + } + + 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. Oldest message:"+ + oldestMessage.getMessageId()); + + c1MarkDeletePositionBefore = waitForMarkDeletePositionToChange(topic1, subName1, c1MarkDeletePositionBefore); + waitForQuotaCheckToRunTwice(); + + Metrics metrics = prometheusMetricsClient.getMetrics(); + TopicStats topicStats = getTopicStats(topic1); + + long expectedMessageAgeSeconds = MILLISECONDS.toSeconds(System.currentTimeMillis() - oldestMessage.getPublishTime()); + assertThat(topicStats.getOldestBacklogMessageAgeSeconds()) + .isCloseTo(expectedMessageAgeSeconds, within(1L)); + + Metric backlogAgeMetric = + metrics.findSingleMetricByNameAndLabels("pulsar_storage_backlog_age_seconds", + Pair.of("topic", topic1)); + assertThat(backlogAgeMetric.tags).containsExactly( + entry("cluster", CLUSTER_NAME), + entry("namespace", namespace), + entry("topic", topic1)); + assertThat((long) backlogAgeMetric.value).isCloseTo(expectedMessageAgeSeconds, within(2L)); + } + config.setPreciseTimeBasedBacklogQuotaCheck(false); + } + + @Test + public void backlogsAgeMetricsNoPreciseWithoutBacklogQuota() throws Exception { + config.setPreciseTimeBasedBacklogQuotaCheck(false); + final String namespace = "prop/ns-quota"; + assertEquals(admin.namespaces().getBacklogQuotaMap(namespace), new HashMap<>()); + + 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 = "c1"; + final int numMsgs = 5; + + Consumer consumer1 = client.newConsumer().topic(topic1).subscriptionName(subName1) + .acknowledgmentGroupTime(0, SECONDS) + .subscribe(); + Producer producer = createProducer(client, topic1); + + byte[] content = new byte[1024]; + for (int i = 0; i < numMsgs; i++) { + Thread.sleep(3000); // Guarantees if we use wrong message in age, to show up in failed test + producer.send(content); + } + + Message oldestMessage = consumer1.receive(); + consumer1.acknowledge(oldestMessage); + log.info("Moved subscription 1, by 1 message"); + + // Unload topic to trigger the ledger close + unloadAndLoadTopic(topic1, producer); + long unloadTime = System.currentTimeMillis(); + waitForQuotaCheckToRunTwice(); + + Metrics metrics = prometheusMetricsClient.getMetrics(); + TopicStats topicStats = getTopicStats(topic1); + + long expectedMessageAgeSeconds = MILLISECONDS.toSeconds(System.currentTimeMillis() - unloadTime); + assertThat(topicStats.getOldestBacklogMessageAgeSeconds()) + .isCloseTo(expectedMessageAgeSeconds, within(1L)); + + Metric backlogAgeMetric = + metrics.findSingleMetricByNameAndLabels("pulsar_storage_backlog_age_seconds", + Pair.of("topic", topic1)); + assertThat(backlogAgeMetric.tags).containsExactly( + entry("cluster", CLUSTER_NAME), + entry("namespace", namespace), + entry("topic", topic1)); + assertThat((long) backlogAgeMetric.value).isCloseTo(expectedMessageAgeSeconds, within(2L)); + } + } + private long getReadEntries(String topic1) { return ((PersistentTopic) pulsar.getBrokerService().getTopicReference(topic1).get()) .getManagedLedger().getStats().getEntriesReadTotalCount(); @@ -718,6 +830,15 @@ public void backlogsStatsNotPrecise() throws PulsarAdminException, PulsarClientE assertThat(topicStats.getOldestBacklogMessageSubscriptionName()).isEqualTo(subName2); expectedAge = MILLISECONDS.toSeconds(System.currentTimeMillis() - unloadTime); assertThat(topicStats.getOldestBacklogMessageAgeSeconds()).isCloseTo(expectedAge, within(1L)); + + // Unsubscribe consume1 and consumer2 + consumer1.unsubscribe(); + consumer2.unsubscribe(); + waitForQuotaCheckToRunTwice(); + topicStats = getTopicStats(topic1); + assertThat(topicStats.getOldestBacklogMessageSubscriptionName()).isNull(); + assertThat(topicStats.getOldestBacklogMessageAgeSeconds()).isEqualTo(-1); + config.setManagedLedgerMaxEntriesPerLedger(MAX_ENTRIES_PER_LEDGER); } } @@ -773,11 +894,11 @@ public void backlogsStatsNotPreciseWithNoBacklog() throws PulsarAdminException, Metrics metrics = prometheusMetricsClient.getMetrics(); assertEquals(topicStats.getSubscriptions().get(subName1).getMsgBacklog(), 0); assertThat(topicStats.getOldestBacklogMessageSubscriptionName()).isNull(); - assertThat(topicStats.getOldestBacklogMessageAgeSeconds()).isEqualTo(0); + assertThat(topicStats.getOldestBacklogMessageAgeSeconds()).isEqualTo(-1); Metric backlogAgeMetric = metrics.findSingleMetricByNameAndLabels("pulsar_storage_backlog_age_seconds", Pair.of("topic", topic1)); - assertThat(backlogAgeMetric.value).isEqualTo(0); + assertThat(backlogAgeMetric.value).isEqualTo(-1); // producer should create success. Producer producer2 = createProducer(client, topic1); From 68eb8f29c50f16f0aa50ecacec16882b82230122 Mon Sep 17 00:00:00 2001 From: Wenzhi Feng <52550727+thetumbled@users.noreply.github.com> Date: Thu, 28 Nov 2024 22:58:24 +0800 Subject: [PATCH 03/22] [fix][client] Fix deadlock of NegativeAcksTracker (#23651) --- .../client/impl/NegativeAcksTracker.java | 40 ++++++++++--------- 1 file changed, 21 insertions(+), 19 deletions(-) diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/NegativeAcksTracker.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/NegativeAcksTracker.java index e1724ebb85cda..5256ebf04f43c 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/NegativeAcksTracker.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/NegativeAcksTracker.java @@ -68,36 +68,38 @@ public NegativeAcksTracker(ConsumerBase consumer, ConsumerConfigurationData messagesToRedeliver = new HashSet<>(); - long now = System.nanoTime(); - nackedMessages.forEach((ledgerId, entryId, partitionIndex, timestamp) -> { - if (timestamp < now) { - MessageId msgId = new MessageIdImpl(ledgerId, entryId, - // need to covert non-partitioned topic partition index to -1 - (int) (partitionIndex == NON_PARTITIONED_TOPIC_PARTITION_INDEX ? -1 : partitionIndex)); - addChunkedMessageIdsAndRemoveFromSequenceMap(msgId, messagesToRedeliver, this.consumer); - messagesToRedeliver.add(msgId); + synchronized (this) { + if (nackedMessages.isEmpty()) { + this.timeout = null; + return; } - }); - if (!messagesToRedeliver.isEmpty()) { + long now = System.nanoTime(); + nackedMessages.forEach((ledgerId, entryId, partitionIndex, timestamp) -> { + if (timestamp < now) { + MessageId msgId = new MessageIdImpl(ledgerId, entryId, + // need to covert non-partitioned topic partition index to -1 + (int) (partitionIndex == NON_PARTITIONED_TOPIC_PARTITION_INDEX ? -1 : partitionIndex)); + addChunkedMessageIdsAndRemoveFromSequenceMap(msgId, messagesToRedeliver, this.consumer); + messagesToRedeliver.add(msgId); + } + }); for (MessageId messageId : messagesToRedeliver) { nackedMessages.remove(((MessageIdImpl) messageId).getLedgerId(), ((MessageIdImpl) messageId).getEntryId()); } + this.timeout = timer.newTimeout(this::triggerRedelivery, timerIntervalNanos, TimeUnit.NANOSECONDS); + } + + // release the lock of NegativeAcksTracker before calling consumer.redeliverUnacknowledgedMessages, + // in which we may acquire the lock of consumer, leading to potential deadlock. + if (!messagesToRedeliver.isEmpty()) { consumer.onNegativeAcksSend(messagesToRedeliver); log.info("[{}] {} messages will be re-delivered", consumer, messagesToRedeliver.size()); consumer.redeliverUnacknowledgedMessages(messagesToRedeliver); } - - this.timeout = timer.newTimeout(this::triggerRedelivery, timerIntervalNanos, TimeUnit.NANOSECONDS); } public synchronized void add(MessageId messageId) { From 7e6fa554012b237308db1bebf3c625e103bf0db5 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Thu, 28 Nov 2024 18:01:17 +0200 Subject: [PATCH 04/22] [improve] Upgrade oxia-java to 0.4.10 and fix closing of OxiaMetadataStore (#23653) --- distribution/server/src/assemble/LICENSE.bin.txt | 4 ++-- pom.xml | 2 +- .../pulsar/metadata/impl/oxia/OxiaMetadataStore.java | 8 +++++--- 3 files changed, 8 insertions(+), 6 deletions(-) diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt index 7c66460c21656..fd393cfec9b76 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -481,8 +481,8 @@ The Apache Software License, Version 2.0 * Prometheus - io.prometheus-simpleclient_httpserver-0.16.0.jar * Oxia - - io.streamnative.oxia-oxia-client-api-0.4.9.jar - - io.streamnative.oxia-oxia-client-0.4.9.jar + - io.streamnative.oxia-oxia-client-api-0.4.10.jar + - io.streamnative.oxia-oxia-client-0.4.10.jar * OpenHFT - net.openhft-zero-allocation-hashing-0.16.jar * Java JSON WebTokens diff --git a/pom.xml b/pom.xml index 40afef1241a54..de05eae5755c0 100644 --- a/pom.xml +++ b/pom.xml @@ -251,7 +251,7 @@ flexible messaging model and an intuitive client API. 4.5.13 4.4.15 0.7.7 - 0.4.9 + 0.4.10 2.0 1.10.12 5.5.0 diff --git a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/oxia/OxiaMetadataStore.java b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/oxia/OxiaMetadataStore.java index 27cd4a2d2f60b..4073eb8dcca2a 100644 --- a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/oxia/OxiaMetadataStore.java +++ b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/oxia/OxiaMetadataStore.java @@ -297,10 +297,12 @@ private CompletableFuture createParents(String path) { @Override public void close() throws Exception { - if (client != null) { - client.close(); + if (isClosed.compareAndSet(false, true)) { + if (client != null) { + client.close(); + } + super.close(); } - super.close(); } public Optional getMetadataEventSynchronizer() { From 66a8f739bcecc4b719adc9a063d0b17ec766d76e Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 29 Nov 2024 04:57:58 +0200 Subject: [PATCH 05/22] [fix][build] Fix error "Element encoding is not allowed here" in pom.xml (#23655) --- pom.xml | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/pom.xml b/pom.xml index de05eae5755c0..e6d154c1b34d6 100644 --- a/pom.xml +++ b/pom.xml @@ -2432,7 +2432,8 @@ flexible messaging model and an intuitive client API. ${pulsar.basedir}/buildtools/src/main/resources/pulsar/checkstyle.xml ${pulsar.basedir}/buildtools/src/main/resources/pulsar/suppressions.xml - UTF-8 + UTF-8 + UTF-8 **/proto/* @@ -2496,7 +2497,7 @@ flexible messaging model and an intuitive client API. ${pulsar.basedir}/buildtools/src/main/resources/pulsar/checkstyle.xml ${pulsar.basedir}/buildtools/src/main/resources/pulsar/suppressions.xml - UTF-8 + UTF-8 **/proto/* From 32b3ccfd331a1a6093aff80b78512dfd0809992f Mon Sep 17 00:00:00 2001 From: zhou zhuohan <843520313@qq.com> Date: Fri, 29 Nov 2024 10:59:19 +0800 Subject: [PATCH 06/22] [improve][client] Replace NameUtil#generateRandomName with RandomStringUtils#randomAlphanumeric (#23645) --- .../pulsar/broker/service/ServerCnxTest.java | 10 +++---- .../pulsar/client/impl/ConsumerBase.java | 5 ++-- .../pulsar/client/impl/ConsumerImpl.java | 4 +-- .../client/impl/MultiTopicsConsumerImpl.java | 6 ++-- .../apache/pulsar/client/util/NameUtil.java | 28 ------------------- .../pulsar/client/impl/ConsumerImplTest.java | 7 +++++ 6 files changed, 20 insertions(+), 40 deletions(-) delete mode 100644 pulsar-client/src/main/java/org/apache/pulsar/client/util/NameUtil.java 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 59e9847b75a33..b1c99940827c8 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 @@ -86,6 +86,7 @@ import org.apache.bookkeeper.mledger.ManagedLedgerFactory; import org.apache.bookkeeper.mledger.Position; import org.apache.bookkeeper.mledger.PositionFactory; +import org.apache.commons.lang3.RandomStringUtils; import org.apache.commons.lang3.mutable.MutableInt; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.ServiceConfiguration; @@ -108,7 +109,6 @@ import org.apache.pulsar.broker.testcontext.PulsarTestContext; import org.apache.pulsar.client.api.ProducerAccessMode; import org.apache.pulsar.client.api.transaction.TxnID; -import org.apache.pulsar.client.util.NameUtil; import org.apache.pulsar.common.api.AuthData; import org.apache.pulsar.common.api.proto.AuthMethod; import org.apache.pulsar.common.api.proto.BaseCommand; @@ -1084,8 +1084,8 @@ public void testHandleConsumerAfterClientChannelInactive() throws Exception { final long consumerId = 1; final MutableInt requestId = new MutableInt(1); final String sName = successSubName; - final String cName1 = NameUtil.generateRandomName(); - final String cName2 = NameUtil.generateRandomName(); + final String cName1 = RandomStringUtils.randomAlphanumeric(5); + final String cName2 = RandomStringUtils.randomAlphanumeric(5); resetChannel(); setChannelConnected(); @@ -1126,8 +1126,8 @@ public void test2ndSubFailedIfDisabledConCheck() final long consumerId = 1; final MutableInt requestId = new MutableInt(1); final String sName = successSubName; - final String cName1 = NameUtil.generateRandomName(); - final String cName2 = NameUtil.generateRandomName(); + final String cName1 = RandomStringUtils.randomAlphanumeric(5); + final String cName2 = RandomStringUtils.randomAlphanumeric(5); // Disabled connection check. pulsar.getConfig().setConnectionLivenessCheckTimeoutMillis(-1); resetChannel(); diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerBase.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerBase.java index 31aef2fd25abb..1ad8c6d28f1d7 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerBase.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerBase.java @@ -43,6 +43,7 @@ import java.util.concurrent.locks.ReentrantLock; import lombok.Getter; import lombok.Setter; +import org.apache.commons.lang3.RandomStringUtils; import org.apache.pulsar.client.api.BatchReceivePolicy; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.ConsumerBuilder; @@ -61,7 +62,6 @@ import org.apache.pulsar.client.impl.conf.ConsumerConfigurationData; import org.apache.pulsar.client.impl.transaction.TransactionImpl; import org.apache.pulsar.client.util.ExecutorProvider; -import org.apache.pulsar.client.util.NameUtil; import org.apache.pulsar.client.util.NoOpLock; import org.apache.pulsar.common.api.proto.CommandAck.AckType; import org.apache.pulsar.common.api.proto.CommandSubscribe; @@ -132,7 +132,8 @@ protected ConsumerBase(PulsarClientImpl client, String topic, ConsumerConfigurat this.maxReceiverQueueSize = receiverQueueSize; this.subscription = conf.getSubscriptionName(); this.conf = conf; - this.consumerName = conf.getConsumerName() == null ? NameUtil.generateRandomName() : conf.getConsumerName(); + this.consumerName = + conf.getConsumerName() == null ? RandomStringUtils.randomAlphanumeric(5) : conf.getConsumerName(); this.subscribeFuture = subscribeFuture; this.listener = conf.getMessageListener(); this.consumerEventListener = conf.getConsumerEventListener(); diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java index d46f4af1be748..390a70095182f 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java @@ -69,6 +69,7 @@ import java.util.stream.Collectors; import lombok.AccessLevel; import lombok.Getter; +import org.apache.commons.lang3.RandomStringUtils; import org.apache.commons.lang3.StringUtils; import org.apache.commons.lang3.tuple.Triple; import org.apache.pulsar.client.api.Consumer; @@ -98,7 +99,6 @@ import org.apache.pulsar.client.impl.schema.AutoConsumeSchema; import org.apache.pulsar.client.impl.transaction.TransactionImpl; import org.apache.pulsar.client.util.ExecutorProvider; -import org.apache.pulsar.client.util.NameUtil; import org.apache.pulsar.client.util.RetryMessageUtil; import org.apache.pulsar.common.allocator.PulsarByteBufAllocator; import org.apache.pulsar.common.api.EncryptionContext; @@ -2267,7 +2267,7 @@ private void initDeadLetterProducerIfNeeded() { .initialSubscriptionName(this.deadLetterPolicy.getInitialSubscriptionName()) .topic(this.deadLetterPolicy.getDeadLetterTopic()) .producerName(String.format("%s-%s-%s-%s-DLQ", this.topicName, this.subscription, - this.consumerName, NameUtil.generateRandomName())) + this.consumerName, RandomStringUtils.randomAlphanumeric(5))) .blockIfQueueFull(false) .enableBatching(false) .enableChunking(true) diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MultiTopicsConsumerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MultiTopicsConsumerImpl.java index dd6a304d9985c..6f9c5b47c55bb 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MultiTopicsConsumerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MultiTopicsConsumerImpl.java @@ -53,6 +53,7 @@ import java.util.stream.Collectors; import java.util.stream.IntStream; import javax.annotation.Nullable; +import org.apache.commons.lang3.RandomStringUtils; import org.apache.commons.lang3.tuple.Pair; import org.apache.pulsar.client.api.BatchReceivePolicy; import org.apache.pulsar.client.api.Consumer; @@ -69,7 +70,6 @@ import org.apache.pulsar.client.impl.conf.ConsumerConfigurationData; import org.apache.pulsar.client.impl.transaction.TransactionImpl; import org.apache.pulsar.client.util.ExecutorProvider; -import org.apache.pulsar.client.util.NameUtil; import org.apache.pulsar.common.api.proto.CommandAck.AckType; import org.apache.pulsar.common.api.proto.CommandGetTopicsOfNamespace; import org.apache.pulsar.common.naming.TopicName; @@ -113,7 +113,7 @@ public class MultiTopicsConsumerImpl extends ConsumerBase { MultiTopicsConsumerImpl(PulsarClientImpl client, ConsumerConfigurationData conf, ExecutorProvider executorProvider, CompletableFuture> subscribeFuture, Schema schema, ConsumerInterceptors interceptors, boolean createTopicIfDoesNotExist) { - this(client, DUMMY_TOPIC_NAME_PREFIX + NameUtil.generateRandomName(), conf, executorProvider, + this(client, DUMMY_TOPIC_NAME_PREFIX + RandomStringUtils.randomAlphanumeric(5), conf, executorProvider, subscribeFuture, schema, interceptors, createTopicIfDoesNotExist); } @@ -121,7 +121,7 @@ public class MultiTopicsConsumerImpl extends ConsumerBase { ExecutorProvider executorProvider, CompletableFuture> subscribeFuture, Schema schema, ConsumerInterceptors interceptors, boolean createTopicIfDoesNotExist, MessageId startMessageId, long startMessageRollbackDurationInSec) { - this(client, DUMMY_TOPIC_NAME_PREFIX + NameUtil.generateRandomName(), conf, executorProvider, + this(client, DUMMY_TOPIC_NAME_PREFIX + RandomStringUtils.randomAlphanumeric(5), conf, executorProvider, subscribeFuture, schema, interceptors, createTopicIfDoesNotExist, startMessageId, startMessageRollbackDurationInSec); } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/util/NameUtil.java b/pulsar-client/src/main/java/org/apache/pulsar/client/util/NameUtil.java deleted file mode 100644 index 4c416b6152cf8..0000000000000 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/util/NameUtil.java +++ /dev/null @@ -1,28 +0,0 @@ -/* - * 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.client.util; - -import java.util.UUID; -import org.apache.commons.codec.digest.DigestUtils; - -public class NameUtil { - public static String generateRandomName() { - return DigestUtils.sha1Hex(UUID.randomUUID().toString()).substring(0, 5); - } -} diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ConsumerImplTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ConsumerImplTest.java index e62958eb96887..4831f1e384d4b 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ConsumerImplTest.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ConsumerImplTest.java @@ -38,6 +38,7 @@ import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; +import java.util.regex.Pattern; import lombok.Cleanup; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.Message; @@ -296,4 +297,10 @@ public void testSeekAsyncInternal() { assertTrue(secondResult.isCompletedExceptionally()); verify(cnx, times(1)).sendRequestWithId(any(ByteBuf.class), anyLong()); } + + @Test(invocationTimeOut = 1000) + public void testAutoGenerateConsumerName() { + Pattern consumerNamePattern = Pattern.compile("[a-zA-Z0-9]{5}"); + assertTrue(consumerNamePattern.matcher(consumer.getConsumerName()).matches()); + } } From 3a502552f5cbe3717c1039028a8e07d2abef4b06 Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Fri, 29 Nov 2024 11:03:23 +0800 Subject: [PATCH 07/22] [improve] [pip] PIP-373: Add a topic's system prop that indicates whether users have published TXN messages in before. (#23210) [improve] [pip] PIP-373: Add a topic's system prop that indicates whether users have published TXN messages in before. (#23210) --- pip/pip-373.md | 78 ++++++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 78 insertions(+) create mode 100644 pip/pip-373.md diff --git a/pip/pip-373.md b/pip/pip-373.md new file mode 100644 index 0000000000000..b40824a186a73 --- /dev/null +++ b/pip/pip-373.md @@ -0,0 +1,78 @@ +# PIP-373: Add a topic's system prop that indicates whether users have published TXN messages in before. + +# Background knowledge + +**Transaction Buffer** +- It maintains `aborted` messages indexes and `ongoing` TXN messages indexes in memory, we call them `Two Indexes` in the following sections. +- It helps to filter out the messages that have been aborted or are still binding with an `on-going` TXN, to avoid consuming aborted messages or TXN in-progress messages. + +**Transaction Buffer initializes when a topic is loading up** +- It reads messages that contain TXN state from the original topic, to re-build `Two Indexes` into memory. + +**Transaction Buffer closing when a topic is unloading** +- It takes a snapshot for `Two Indexes` to rebuild the states faster next loading up. + +# Motivation + +- Since TXN is a feature that across multiple namespaces, it can be enabled or disabled at the cluster level. +- Transaction Buffer will try to re-build `Two Indexes` for every topic if you enable TXN. +- If you have a huge number of topics in a namespace, the task that re-build `Two Indexes` costs huge resources(CPU, Memory). + +We'd better skip Transaction Buffer re-building `Two Indexes` if the topic it is related to does not contain TXN messages to save resources usage. + +# Goals + +Skip Transaction Buffer re-building `Two Indexes` if the topic it is related to does not contain TXN messages to save resources usage. + +## In Scope + +This PIP only focuses on improving the scenario in which users have never published TXN messages on a topic. + +## Out of Scope + +This PIP does not focus on the scenario that follows. +- Published TXN messages. +- Consumed all TXN messages that were sent before. +- Transaction Buffer keeps re-building `Two Indexes` even if there are no TXN messages in the topic anymore, and it costs resources. + +# Detailed Design + +## Design & Implementation Details + +- Add a topic-level system property named `__contains_txn_messages`, the default value is `false`, and it will be set to `true` when the first TXN message is publishing. +- Transaction Buffer skips re-building `Two Indexes` if the property is `false`. + +## Public-facing Changes + +The topic property `__contains_txn_messages` becomes to a system property, it can never be used by users anymore. + +### Public API +- You will get a `NotAllowedException` when you try to set/remove a topic property named `__contains_txn_messages` by the API `pulsar-admin topics update-properties/remove-properties`. +- The value of the property `__contains_txn_messages` can be queried by `pulsar-admin topics get-properties`. + + +### Metrics +| Name | Description | Attributes | Units| +|--------------------------------------------|-----------------------------------------------------| --- | --- | +| `pulsar_broker_using_txn_topics_count` | Counter. The number of topics contains TXN messages. | cluster | - | + +# Backward & Forward Compatibility + +## Upgrade + +There are `3` scenarios that the topic's property `__contains_txn_messages` is when users try to upgrade. +- `__contains_txn_messages` is empty: broker initializes it by confirming whether there is TXN messages or not. +- **(Highlight)** `__contains_txn_messages` is not empty and is not typed `boolean`, rollback to the original behavior that always re-building `Two Indexes`, but you can never modify it anymore. +- **(Highlight)** `__contains_txn_messages` is not empty and is typed `boolean`, but it is a users' property, broker assumed that it is a system property. + +## Downgrade / Rollback + +You can downgrade or roll back gracefully. + +## Pulsar Geo-Replication Upgrade & Downgrade/Rollback Considerations + +The PIP does not affect Geo-Replication. + +# Links +* Mailing List discussion thread: https://lists.apache.org/thread/7mblhyvsrw5zybo0gs5512xg8f9sm67v +* Mailing List voting thread: https://lists.apache.org/thread/s6z5gcjyw081cxf9pwz361r8dt2k8gvl From d1753ee44221cd2bb9f16f18412617ab533112f0 Mon Sep 17 00:00:00 2001 From: Jiwei Guo Date: Fri, 29 Nov 2024 14:00:16 +0800 Subject: [PATCH 08/22] [fix][broker] Revert "[improve][client] Add log when can't add message to the container (#23657) --- .../java/org/apache/pulsar/client/impl/MessagesImpl.java | 6 ------ 1 file changed, 6 deletions(-) diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MessagesImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MessagesImpl.java index 9768fd7c74b0f..d4cd36a22e15f 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MessagesImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MessagesImpl.java @@ -22,13 +22,11 @@ import java.util.ArrayList; import java.util.Iterator; import java.util.List; -import lombok.extern.slf4j.Slf4j; import net.jcip.annotations.NotThreadSafe; import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.Messages; @NotThreadSafe -@Slf4j public class MessagesImpl implements Messages { private final List> messageList; @@ -51,14 +49,10 @@ protected boolean canAdd(Message message) { return true; } if (maxNumberOfMessages > 0 && currentNumberOfMessages + 1 > maxNumberOfMessages) { - log.warn("can't add message to the container, has exceeded the maxNumberOfMessages : {} ", - maxNumberOfMessages); return false; } if (maxSizeOfMessages > 0 && currentSizeOfMessages + message.size() > maxSizeOfMessages) { - log.warn("can't add message to the container, has exceeded the maxSizeOfMessages : {} ", - maxSizeOfMessages); return false; } From eb60d0ab2022f11ebd2217a32ef60886f757cddf Mon Sep 17 00:00:00 2001 From: Qiang Zhao Date: Fri, 29 Nov 2024 16:30:48 +0800 Subject: [PATCH 09/22] [fix][broker]: support missing broker level fine-granted permissions (#23637) --- .../authorization/AuthorizationProvider.java | 10 + .../authorization/AuthorizationService.java | 23 ++ .../pulsar/broker/admin/impl/BrokersBase.java | 100 ++++++- .../BrokerEndpointsAuthorizationTest.java | 277 ++++++++++++++++++ .../client/admin/internal/BrokersImpl.java | 2 +- .../common/policies/data/BrokerOperation.java | 39 +++ 6 files changed, 437 insertions(+), 14 deletions(-) create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/BrokerEndpointsAuthorizationTest.java create mode 100644 pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/BrokerOperation.java diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/AuthorizationProvider.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/AuthorizationProvider.java index ffb38f770a9cc..48386265940a3 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/AuthorizationProvider.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/AuthorizationProvider.java @@ -32,6 +32,7 @@ import org.apache.pulsar.common.naming.NamespaceName; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.policies.data.AuthAction; +import org.apache.pulsar.common.policies.data.BrokerOperation; import org.apache.pulsar.common.policies.data.NamespaceOperation; import org.apache.pulsar.common.policies.data.PolicyName; import org.apache.pulsar.common.policies.data.PolicyOperation; @@ -383,4 +384,13 @@ default CompletableFuture>> getPermissionsAsync(Name String.format("getPermissionsAsync on namespaceName %s is not supported by the Authorization", namespaceName))); } + + default CompletableFuture allowBrokerOperationAsync(String clusterName, + String brokerId, + BrokerOperation brokerOperation, + String role, + AuthenticationDataSource authData) { + return FutureUtil.failedFuture( + new UnsupportedOperationException("allowBrokerOperationAsync is not supported yet.")); + } } diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/AuthorizationService.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/AuthorizationService.java index 2951eb1f2973f..1348a405b0dfa 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/AuthorizationService.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/AuthorizationService.java @@ -38,6 +38,7 @@ import org.apache.pulsar.common.naming.NamespaceName; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.policies.data.AuthAction; +import org.apache.pulsar.common.policies.data.BrokerOperation; import org.apache.pulsar.common.policies.data.NamespaceOperation; import org.apache.pulsar.common.policies.data.PolicyName; import org.apache.pulsar.common.policies.data.PolicyOperation; @@ -544,6 +545,28 @@ public CompletableFuture allowTenantOperationAsync(String tenantName, } } + public CompletableFuture allowBrokerOperationAsync(String clusterName, + String brokerId, + BrokerOperation brokerOperation, + String originalRole, + String role, + AuthenticationDataSource authData) { + if (!isValidOriginalPrincipal(role, originalRole, authData)) { + return CompletableFuture.completedFuture(false); + } + + if (isProxyRole(role)) { + final var isRoleAuthorizedFuture = provider.allowBrokerOperationAsync(clusterName, brokerId, + brokerOperation, role, authData); + final var isOriginalAuthorizedFuture = provider.allowBrokerOperationAsync(clusterName, brokerId, + brokerOperation, originalRole, authData); + return isRoleAuthorizedFuture.thenCombine(isOriginalAuthorizedFuture, + (isRoleAuthorized, isOriginalAuthorized) -> isRoleAuthorized && isOriginalAuthorized); + } else { + return provider.allowBrokerOperationAsync(clusterName, brokerId, brokerOperation, role, authData); + } + } + /** * @deprecated - will be removed after 2.12. Use async variant. */ diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/BrokersBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/BrokersBase.java index e397dbb64a075..a24a78d8e3102 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/BrokersBase.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/BrokersBase.java @@ -69,6 +69,7 @@ import org.apache.pulsar.common.naming.NamespaceName; import org.apache.pulsar.common.naming.TopicVersion; import org.apache.pulsar.common.policies.data.BrokerInfo; +import org.apache.pulsar.common.policies.data.BrokerOperation; import org.apache.pulsar.common.policies.data.NamespaceOwnershipStatus; import org.apache.pulsar.common.util.FutureUtil; import org.apache.pulsar.common.util.ThreadDumpUtil; @@ -107,7 +108,8 @@ public class BrokersBase extends AdminResource { @ApiResponse(code = 404, message = "Cluster does not exist: cluster={clustername}") }) public void getActiveBrokers(@Suspended final AsyncResponse asyncResponse, @PathParam("cluster") String cluster) { - validateSuperUserAccessAsync() + validateBothSuperuserAndBrokerOperation(cluster == null ? pulsar().getConfiguration().getClusterName() + : cluster, pulsar().getBrokerId(), BrokerOperation.LIST_BROKERS) .thenCompose(__ -> validateClusterOwnershipAsync(cluster)) .thenCompose(__ -> pulsar().getLoadManager().get().getAvailableBrokersAsync()) .thenAccept(activeBrokers -> { @@ -148,7 +150,9 @@ public void getActiveBrokers(@Suspended final AsyncResponse asyncResponse) throw @ApiResponse(code = 403, message = "This operation requires super-user access"), @ApiResponse(code = 404, message = "Leader broker not found") }) public void getLeaderBroker(@Suspended final AsyncResponse asyncResponse) { - validateSuperUserAccessAsync().thenAccept(__ -> { + validateBothSuperuserAndBrokerOperation(pulsar().getConfig().getClusterName(), + pulsar().getBrokerId(), BrokerOperation.GET_LEADER_BROKER) + .thenAccept(__ -> { LeaderBroker leaderBroker = pulsar().getLeaderElectionService().getCurrentLeader() .orElseThrow(() -> new RestException(Status.NOT_FOUND, "Couldn't find leader broker")); BrokerInfo brokerInfo = BrokerInfo.builder() @@ -175,7 +179,8 @@ public void getLeaderBroker(@Suspended final AsyncResponse asyncResponse) { public void getOwnedNamespaces(@Suspended final AsyncResponse asyncResponse, @PathParam("clusterName") String cluster, @PathParam("brokerId") String brokerId) { - validateSuperUserAccessAsync() + validateBothSuperuserAndBrokerOperation(pulsar().getConfig().getClusterName(), + pulsar().getBrokerId(), BrokerOperation.LIST_OWNED_NAMESPACES) .thenCompose(__ -> maybeRedirectToBroker(brokerId)) .thenCompose(__ -> validateClusterOwnershipAsync(cluster)) .thenCompose(__ -> pulsar().getNamespaceService().getOwnedNameSpacesStatusAsync()) @@ -204,7 +209,8 @@ public void getOwnedNamespaces(@Suspended final AsyncResponse asyncResponse, public void updateDynamicConfiguration(@Suspended AsyncResponse asyncResponse, @PathParam("configName") String configName, @PathParam("configValue") String configValue) { - validateSuperUserAccessAsync() + validateBothSuperuserAndBrokerOperation(pulsar().getConfig().getClusterName(), pulsar().getBrokerId(), + BrokerOperation.UPDATE_DYNAMIC_CONFIGURATION) .thenCompose(__ -> persistDynamicConfigurationAsync(configName, configValue)) .thenAccept(__ -> { LOG.info("[{}] Updated Service configuration {}/{}", clientAppId(), configName, configValue); @@ -228,7 +234,8 @@ public void updateDynamicConfiguration(@Suspended AsyncResponse asyncResponse, public void deleteDynamicConfiguration( @Suspended AsyncResponse asyncResponse, @PathParam("configName") String configName) { - validateSuperUserAccessAsync() + validateBothSuperuserAndBrokerOperation(pulsar().getConfig().getClusterName(), pulsar().getBrokerId(), + BrokerOperation.DELETE_DYNAMIC_CONFIGURATION) .thenCompose(__ -> internalDeleteDynamicConfigurationOnMetadataAsync(configName)) .thenAccept(__ -> { LOG.info("[{}] Successfully to delete dynamic configuration {}", clientAppId(), configName); @@ -249,7 +256,8 @@ public void deleteDynamicConfiguration( @ApiResponse(code = 404, message = "Configuration not found"), @ApiResponse(code = 500, message = "Internal server error")}) public void getAllDynamicConfigurations(@Suspended AsyncResponse asyncResponse) { - validateSuperUserAccessAsync() + validateBothSuperuserAndBrokerOperation(pulsar().getConfig().getClusterName(), pulsar().getBrokerId(), + BrokerOperation.LIST_DYNAMIC_CONFIGURATIONS) .thenCompose(__ -> dynamicConfigurationResources().getDynamicConfigurationAsync()) .thenAccept(configOpt -> asyncResponse.resume(configOpt.orElseGet(Collections::emptyMap))) .exceptionally(ex -> { @@ -266,7 +274,8 @@ public void getAllDynamicConfigurations(@Suspended AsyncResponse asyncResponse) @ApiResponses(value = { @ApiResponse(code = 403, message = "You don't have admin permission to get configuration")}) public void getDynamicConfigurationName(@Suspended AsyncResponse asyncResponse) { - validateSuperUserAccessAsync() + validateBothSuperuserAndBrokerOperation(pulsar().getConfig().getClusterName(), pulsar().getBrokerId(), + BrokerOperation.LIST_DYNAMIC_CONFIGURATIONS) .thenAccept(__ -> asyncResponse.resume(pulsar().getBrokerService().getDynamicConfiguration())) .exceptionally(ex -> { LOG.error("[{}] Failed to get all dynamic configuration names.", clientAppId(), ex); @@ -281,7 +290,8 @@ public void getDynamicConfigurationName(@Suspended AsyncResponse asyncResponse) response = String.class, responseContainer = "Map") @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission") }) public void getRuntimeConfiguration(@Suspended AsyncResponse asyncResponse) { - validateSuperUserAccessAsync() + validateBothSuperuserAndBrokerOperation(pulsar().getConfig().getClusterName(), pulsar().getBrokerId(), + BrokerOperation.LIST_RUNTIME_CONFIGURATIONS) .thenAccept(__ -> asyncResponse.resume(pulsar().getBrokerService().getRuntimeConfiguration())) .exceptionally(ex -> { LOG.error("[{}] Failed to get runtime configuration.", clientAppId(), ex); @@ -322,7 +332,8 @@ private synchronized CompletableFuture persistDynamicConfigurationAsync( @ApiOperation(value = "Get the internal configuration data", response = InternalConfigurationData.class) @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission") }) public void getInternalConfigurationData(@Suspended AsyncResponse asyncResponse) { - validateSuperUserAccessAsync() + validateBothSuperuserAndBrokerOperation(pulsar().getConfig().getClusterName(), pulsar().getBrokerId(), + BrokerOperation.GET_INTERNAL_CONFIGURATION_DATA) .thenAccept(__ -> asyncResponse.resume(pulsar().getInternalConfigurationData())) .exceptionally(ex -> { LOG.error("[{}] Failed to get internal configuration data.", clientAppId(), ex); @@ -339,7 +350,8 @@ public void getInternalConfigurationData(@Suspended AsyncResponse asyncResponse) @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 500, message = "Internal server error")}) public void backlogQuotaCheck(@Suspended AsyncResponse asyncResponse) { - validateSuperUserAccessAsync() + validateBothSuperuserAndBrokerOperation(pulsar().getConfig().getClusterName(), pulsar().getBrokerId(), + BrokerOperation.CHECK_BACKLOG_QUOTA) .thenAcceptAsync(__ -> { pulsar().getBrokerService().monitorBacklogQuota(); asyncResponse.resume(Response.noContent().build()); @@ -378,7 +390,8 @@ public void healthCheck(@Suspended AsyncResponse asyncResponse, @ApiParam(value = "Topic Version") @QueryParam("topicVersion") TopicVersion topicVersion, @QueryParam("brokerId") String brokerId) { - validateSuperUserAccessAsync() + validateBothSuperuserAndBrokerOperation(pulsar().getConfig().getClusterName(), StringUtils.isBlank(brokerId) + ? pulsar().getBrokerId() : brokerId, BrokerOperation.HEALTH_CHECK) .thenAccept(__ -> checkDeadlockedThreads()) .thenCompose(__ -> maybeRedirectToBroker( StringUtils.isBlank(brokerId) ? pulsar().getBrokerId() : brokerId)) @@ -596,8 +609,9 @@ public void shutDownBrokerGracefully( @QueryParam("forcedTerminateTopic") @DefaultValue("true") boolean forcedTerminateTopic, @Suspended final AsyncResponse asyncResponse ) { - validateSuperUserAccess(); - doShutDownBrokerGracefullyAsync(maxConcurrentUnloadPerSec, forcedTerminateTopic) + validateBothSuperuserAndBrokerOperation(pulsar().getConfig().getClusterName(), pulsar().getBrokerId(), + BrokerOperation.SHUTDOWN) + .thenCompose(__ -> doShutDownBrokerGracefullyAsync(maxConcurrentUnloadPerSec, forcedTerminateTopic)) .thenAccept(__ -> { LOG.info("[{}] Successfully shutdown broker gracefully", clientAppId()); asyncResponse.resume(Response.noContent().build()); @@ -614,5 +628,65 @@ private CompletableFuture doShutDownBrokerGracefullyAsync(int maxConcurren pulsar().getBrokerService().unloadNamespaceBundlesGracefully(maxConcurrentUnloadPerSec, forcedTerminateTopic); return pulsar().closeAsync(); } + + + private CompletableFuture validateBothSuperuserAndBrokerOperation(String cluster, String brokerId, + BrokerOperation operation) { + final var superUserAccessValidation = validateSuperUserAccessAsync(); + final var brokerOperationValidation = validateBrokerOperationAsync(cluster, brokerId, operation); + return FutureUtil.waitForAll(List.of(superUserAccessValidation, brokerOperationValidation)) + .handle((result, err) -> { + if (!superUserAccessValidation.isCompletedExceptionally() + || !brokerOperationValidation.isCompletedExceptionally()) { + return null; + } + if (LOG.isDebugEnabled()) { + Throwable superUserValidationException = null; + try { + superUserAccessValidation.join(); + } catch (Throwable ex) { + superUserValidationException = FutureUtil.unwrapCompletionException(ex); + } + Throwable brokerOperationValidationException = null; + try { + brokerOperationValidation.join(); + } catch (Throwable ex) { + brokerOperationValidationException = FutureUtil.unwrapCompletionException(ex); + } + LOG.debug("validateBothSuperuserAndBrokerOperation failed." + + " originalPrincipal={} clientAppId={} operation={} broker={} " + + "superuserValidationError={} brokerOperationValidationError={}", + originalPrincipal(), clientAppId(), operation.toString(), brokerId, + superUserValidationException, brokerOperationValidationException); + } + throw new RestException(Status.UNAUTHORIZED, + String.format("Unauthorized to validateBothSuperuserAndBrokerOperation for" + + " originalPrincipal [%s] and clientAppId [%s] " + + "about operation [%s] on broker [%s]", + originalPrincipal(), clientAppId(), operation.toString(), brokerId)); + }); + } + + + private CompletableFuture validateBrokerOperationAsync(String cluster, String brokerId, + BrokerOperation operation) { + final var pulsar = pulsar(); + if (pulsar.getBrokerService().isAuthenticationEnabled() + && pulsar.getBrokerService().isAuthorizationEnabled()) { + return pulsar.getBrokerService().getAuthorizationService() + .allowBrokerOperationAsync(cluster, brokerId, operation, originalPrincipal(), + clientAppId(), clientAuthData()) + .thenAccept(isAuthorized -> { + if (!isAuthorized) { + throw new RestException(Status.UNAUTHORIZED, + String.format("Unauthorized to validateBrokerOperation for" + + " originalPrincipal [%s] and clientAppId [%s] " + + "about operation [%s] on broker [%s]", + originalPrincipal(), clientAppId(), operation.toString(), brokerId)); + } + }); + } + return CompletableFuture.completedFuture(null); + } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/BrokerEndpointsAuthorizationTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/BrokerEndpointsAuthorizationTest.java new file mode 100644 index 0000000000000..ef66f005b3c58 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/BrokerEndpointsAuthorizationTest.java @@ -0,0 +1,277 @@ +/* + * 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.admin; + + +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.eq; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.verify; +import lombok.SneakyThrows; +import org.apache.commons.lang3.reflect.FieldUtils; +import org.apache.pulsar.broker.authorization.AuthorizationService; +import org.apache.pulsar.client.admin.PulsarAdmin; +import org.apache.pulsar.client.admin.PulsarAdminException; +import org.apache.pulsar.client.impl.auth.AuthenticationToken; +import org.apache.pulsar.common.naming.TopicVersion; +import org.apache.pulsar.common.policies.data.BrokerOperation; +import org.apache.pulsar.security.MockedPulsarStandalone; +import org.testng.Assert; +import org.testng.annotations.AfterClass; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; + +@Test(groups = "broker-admin") +public class BrokerEndpointsAuthorizationTest extends MockedPulsarStandalone { + private AuthorizationService orignalAuthorizationService; + private AuthorizationService spyAuthorizationService; + + private PulsarAdmin superUserAdmin; + private PulsarAdmin nobodyAdmin; + + @SneakyThrows + @BeforeClass(alwaysRun = true) + public void setup() { + configureTokenAuthentication(); + configureDefaultAuthorization(); + start(); + this.superUserAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(SUPER_USER_TOKEN)) + .build(); + this.nobodyAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(NOBODY_TOKEN)) + .build(); + } + + @BeforeMethod(alwaysRun = true) + public void before() throws IllegalAccessException { + orignalAuthorizationService = getPulsarService().getBrokerService().getAuthorizationService(); + spyAuthorizationService = spy(orignalAuthorizationService); + FieldUtils.writeField(getPulsarService().getBrokerService(), "authorizationService", + spyAuthorizationService, true); + } + + @AfterMethod(alwaysRun = true) + public void after() throws IllegalAccessException { + if (orignalAuthorizationService != null) { + FieldUtils.writeField(getPulsarService().getBrokerService(), "authorizationService", orignalAuthorizationService, true); + } + } + + @SneakyThrows + @AfterClass(alwaysRun = true) + public void cleanup() { + if (superUserAdmin != null) { + superUserAdmin.close(); + superUserAdmin = null; + } + spyAuthorizationService = null; + orignalAuthorizationService = null; + super.close(); + } + + @Test + public void testGetActiveBroker() throws PulsarAdminException { + superUserAdmin.brokers().getActiveBrokers(); + final String brokerId = getPulsarService().getBrokerId(); + final String clusterName = getPulsarService().getConfiguration().getClusterName(); + // test allow broker operation + verify(spyAuthorizationService) + .allowBrokerOperationAsync(eq(clusterName), eq(brokerId), eq(BrokerOperation.LIST_BROKERS), any(), any(), any()); + // fallback to superuser + verify(spyAuthorizationService).isSuperUser(any(), any()); + + // ---- test nobody + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, () -> nobodyAdmin.brokers().getActiveBrokers()); + } + + @Test + public void testGetActiveBrokerWithCluster() throws PulsarAdminException { + final String clusterName = getPulsarService().getConfiguration().getClusterName(); + superUserAdmin.brokers().getActiveBrokers(clusterName); + final String brokerId = getPulsarService().getBrokerId(); + // test allow broker operation + verify(spyAuthorizationService) + .allowBrokerOperationAsync(eq(clusterName), eq(brokerId), eq(BrokerOperation.LIST_BROKERS), any(), any(), any()); + // fallback to superuser + verify(spyAuthorizationService).isSuperUser(any(), any()); + + // ---- test nobody + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, () -> nobodyAdmin.brokers().getActiveBrokers(clusterName)); + } + + @Test + public void testGetLeaderBroker() throws PulsarAdminException { + superUserAdmin.brokers().getLeaderBroker(); + final String clusterName = getPulsarService().getConfiguration().getClusterName(); + final String brokerId = getPulsarService().getBrokerId(); + // test allow broker operation + verify(spyAuthorizationService) + .allowBrokerOperationAsync(eq(clusterName), eq(brokerId), eq(BrokerOperation.GET_LEADER_BROKER), any(), any(), any()); + // fallback to superuser + verify(spyAuthorizationService).isSuperUser(any(), any()); + + // ---- test nobody + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, () -> nobodyAdmin.brokers().getLeaderBroker()); + } + + @Test + public void testGetOwnedNamespaces() throws PulsarAdminException { + final String clusterName = getPulsarService().getConfiguration().getClusterName(); + final String brokerId = getPulsarService().getBrokerId(); + superUserAdmin.brokers().getOwnedNamespaces(clusterName, brokerId); + // test allow broker operation + verify(spyAuthorizationService) + .allowBrokerOperationAsync(eq(clusterName), eq(brokerId), eq(BrokerOperation.LIST_OWNED_NAMESPACES), any(), any(), any()); + // fallback to superuser + verify(spyAuthorizationService).isSuperUser(any(), any()); + + // ---- test nobody + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, () -> nobodyAdmin.brokers().getOwnedNamespaces(clusterName, brokerId)); + } + + @Test + public void testUpdateDynamicConfiguration() throws PulsarAdminException { + final String clusterName = getPulsarService().getConfiguration().getClusterName(); + final String brokerId = getPulsarService().getBrokerId(); + superUserAdmin.brokers().updateDynamicConfiguration("maxTenants", "10"); + // test allow broker operation + verify(spyAuthorizationService) + .allowBrokerOperationAsync(eq(clusterName), eq(brokerId), eq(BrokerOperation.UPDATE_DYNAMIC_CONFIGURATION), any(), any(), any()); + // fallback to superuser + verify(spyAuthorizationService).isSuperUser(any(), any()); + + // ---- test nobody + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, () -> nobodyAdmin.brokers().updateDynamicConfiguration("maxTenants", "10")); + } + + @Test + public void testDeleteDynamicConfiguration() throws PulsarAdminException { + final String clusterName = getPulsarService().getConfiguration().getClusterName(); + final String brokerId = getPulsarService().getBrokerId(); + superUserAdmin.brokers().deleteDynamicConfiguration("maxTenants"); + // test allow broker operation + verify(spyAuthorizationService) + .allowBrokerOperationAsync(eq(clusterName), eq(brokerId), eq(BrokerOperation.DELETE_DYNAMIC_CONFIGURATION), any(), any(), any()); + // fallback to superuser + verify(spyAuthorizationService).isSuperUser(any(), any()); + + // ---- test nobody + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, () -> nobodyAdmin.brokers().deleteDynamicConfiguration("maxTenants")); + } + + + @Test + public void testGetAllDynamicConfiguration() throws PulsarAdminException { + final String clusterName = getPulsarService().getConfiguration().getClusterName(); + final String brokerId = getPulsarService().getBrokerId(); + superUserAdmin.brokers().getAllDynamicConfigurations(); + // test allow broker operation + verify(spyAuthorizationService) + .allowBrokerOperationAsync(eq(clusterName), eq(brokerId), eq(BrokerOperation.LIST_DYNAMIC_CONFIGURATIONS), any(), any(), any()); + // fallback to superuser + verify(spyAuthorizationService).isSuperUser(any(), any()); + + // ---- test nobody + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, () -> nobodyAdmin.brokers().getAllDynamicConfigurations()); + } + + + @Test + public void testGetDynamicConfigurationName() throws PulsarAdminException { + final String clusterName = getPulsarService().getConfiguration().getClusterName(); + final String brokerId = getPulsarService().getBrokerId(); + superUserAdmin.brokers().getDynamicConfigurationNames(); + // test allow broker operation + verify(spyAuthorizationService) + .allowBrokerOperationAsync(eq(clusterName), eq(brokerId), eq(BrokerOperation.LIST_DYNAMIC_CONFIGURATIONS), any(), any(), any()); + // fallback to superuser + verify(spyAuthorizationService).isSuperUser(any(), any()); + + // ---- test nobody + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, () -> nobodyAdmin.brokers().getDynamicConfigurationNames()); + } + + + @Test + public void testGetRuntimeConfiguration() throws PulsarAdminException { + final String clusterName = getPulsarService().getConfiguration().getClusterName(); + final String brokerId = getPulsarService().getBrokerId(); + superUserAdmin.brokers().getRuntimeConfigurations(); + // test allow broker operation + verify(spyAuthorizationService) + .allowBrokerOperationAsync(eq(clusterName), eq(brokerId), eq(BrokerOperation.LIST_RUNTIME_CONFIGURATIONS), any(), any(), any()); + // fallback to superuser + verify(spyAuthorizationService).isSuperUser(any(), any()); + + // ---- test nobody + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, () -> nobodyAdmin.brokers().getRuntimeConfigurations()); + } + + + @Test + public void testGetInternalConfigurationData() throws PulsarAdminException { + final String clusterName = getPulsarService().getConfiguration().getClusterName(); + final String brokerId = getPulsarService().getBrokerId(); + superUserAdmin.brokers().getInternalConfigurationData(); + // test allow broker operation + verify(spyAuthorizationService) + .allowBrokerOperationAsync(eq(clusterName), eq(brokerId), eq(BrokerOperation.GET_INTERNAL_CONFIGURATION_DATA), any(), any(), any()); + // fallback to superuser + verify(spyAuthorizationService).isSuperUser(any(), any()); + + // ---- test nobody + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, () -> nobodyAdmin.brokers().getInternalConfigurationData()); + } + + + @Test + public void testBacklogQuotaCheck() throws PulsarAdminException { + final String clusterName = getPulsarService().getConfiguration().getClusterName(); + final String brokerId = getPulsarService().getBrokerId(); + superUserAdmin.brokers().backlogQuotaCheck(); + // test allow broker operation + verify(spyAuthorizationService) + .allowBrokerOperationAsync(eq(clusterName), eq(brokerId), eq(BrokerOperation.CHECK_BACKLOG_QUOTA), any(), any(), any()); + // fallback to superuser + verify(spyAuthorizationService).isSuperUser(any(), any()); + + // ---- test nobody + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, () -> nobodyAdmin.brokers().backlogQuotaCheck()); + } + + @Test + public void testHealthCheck() throws PulsarAdminException { + final String clusterName = getPulsarService().getConfiguration().getClusterName(); + final String brokerId = getPulsarService().getBrokerId(); + superUserAdmin.brokers().healthcheck(TopicVersion.V2); + // test allow broker operation + verify(spyAuthorizationService) + .allowBrokerOperationAsync(eq(clusterName), eq(brokerId), eq(BrokerOperation.HEALTH_CHECK), any(), any(), any()); + // fallback to superuser + verify(spyAuthorizationService).isSuperUser(any(), any()); + + // ---- test nobody + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, () -> nobodyAdmin.brokers().healthcheck(TopicVersion.V2)); + } +} diff --git a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/BrokersImpl.java b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/BrokersImpl.java index 35b261b196eee..b0cd3edeb21fe 100644 --- a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/BrokersImpl.java +++ b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/BrokersImpl.java @@ -162,7 +162,7 @@ public void backlogQuotaCheck() throws PulsarAdminException { @Override public CompletableFuture backlogQuotaCheckAsync() { - WebTarget path = adminBrokers.path("backlogQuotaCheck"); + WebTarget path = adminBrokers.path("backlog-quota-check"); return asyncGetRequest(path, new FutureCallback() {}); } diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/BrokerOperation.java b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/BrokerOperation.java new file mode 100644 index 0000000000000..de053fea6ad4a --- /dev/null +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/BrokerOperation.java @@ -0,0 +1,39 @@ +/* + * 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.common.policies.data; + +public enum BrokerOperation { + LIST_BROKERS, + GET_BROKER, + + GET_LEADER_BROKER, + LIST_OWNED_NAMESPACES, + + LIST_DYNAMIC_CONFIGURATIONS, + UPDATE_DYNAMIC_CONFIGURATION, + DELETE_DYNAMIC_CONFIGURATION, + + LIST_RUNTIME_CONFIGURATIONS, + + GET_INTERNAL_CONFIGURATION_DATA, + + CHECK_BACKLOG_QUOTA, + HEALTH_CHECK, + SHUTDOWN +} From 9ed44dd77f81abeb63fd5ed2877a1601afdc0bcc Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 29 Nov 2024 11:05:29 +0200 Subject: [PATCH 10/22] [improve] Upgrade OpenTelemetry library to 1.44.1 version (#23656) --- .../server/src/assemble/LICENSE.bin.txt | 41 +++++++++---------- .../shell/src/assemble/LICENSE.bin.txt | 6 +-- pom.xml | 2 +- .../stats/BrokerOpenTelemetryTestUtil.java | 13 ++++-- .../client/metrics/ClientMetricsTest.java | 18 ++++++-- 5 files changed, 47 insertions(+), 33 deletions(-) diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt index fd393cfec9b76..10899bc0ae7aa 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -338,12 +338,11 @@ The Apache Software License, Version 2.0 - io.prometheus-simpleclient_tracer_otel-0.16.0.jar - io.prometheus-simpleclient_tracer_otel_agent-0.16.0.jar * Prometheus exporter - - io.prometheus-prometheus-metrics-config-1.3.1.jar - - io.prometheus-prometheus-metrics-exporter-common-1.3.1.jar - - io.prometheus-prometheus-metrics-exporter-httpserver-1.3.1.jar - - io.prometheus-prometheus-metrics-exposition-formats-1.3.1.jar - - io.prometheus-prometheus-metrics-model-1.3.1.jar - - io.prometheus-prometheus-metrics-shaded-protobuf-1.3.1.jar + - io.prometheus-prometheus-metrics-config-1.3.3.jar + - io.prometheus-prometheus-metrics-exporter-common-1.3.3.jar + - io.prometheus-prometheus-metrics-exporter-httpserver-1.3.3.jar + - io.prometheus-prometheus-metrics-exposition-formats-1.3.3.jar + - io.prometheus-prometheus-metrics-model-1.3.3.jar * Jakarta Bean Validation API - jakarta.validation-jakarta.validation-api-2.0.2.jar - javax.validation-validation-api-1.1.0.Final.jar @@ -516,21 +515,21 @@ The Apache Software License, Version 2.0 * RoaringBitmap - org.roaringbitmap-RoaringBitmap-1.2.0.jar * OpenTelemetry - - io.opentelemetry-opentelemetry-api-1.41.0.jar - - io.opentelemetry-opentelemetry-api-incubator-1.41.0-alpha.jar - - io.opentelemetry-opentelemetry-context-1.41.0.jar - - io.opentelemetry-opentelemetry-exporter-common-1.41.0.jar - - io.opentelemetry-opentelemetry-exporter-otlp-1.41.0.jar - - io.opentelemetry-opentelemetry-exporter-otlp-common-1.41.0.jar - - io.opentelemetry-opentelemetry-exporter-prometheus-1.41.0-alpha.jar - - io.opentelemetry-opentelemetry-exporter-sender-okhttp-1.41.0.jar - - io.opentelemetry-opentelemetry-sdk-1.41.0.jar - - io.opentelemetry-opentelemetry-sdk-common-1.41.0.jar - - io.opentelemetry-opentelemetry-sdk-extension-autoconfigure-1.41.0.jar - - io.opentelemetry-opentelemetry-sdk-extension-autoconfigure-spi-1.41.0.jar - - io.opentelemetry-opentelemetry-sdk-logs-1.41.0.jar - - io.opentelemetry-opentelemetry-sdk-metrics-1.41.0.jar - - io.opentelemetry-opentelemetry-sdk-trace-1.41.0.jar + - io.opentelemetry-opentelemetry-api-1.44.1.jar + - io.opentelemetry-opentelemetry-api-incubator-1.44.1-alpha.jar + - io.opentelemetry-opentelemetry-context-1.44.1.jar + - io.opentelemetry-opentelemetry-exporter-common-1.44.1.jar + - io.opentelemetry-opentelemetry-exporter-otlp-1.44.1.jar + - io.opentelemetry-opentelemetry-exporter-otlp-common-1.44.1.jar + - io.opentelemetry-opentelemetry-exporter-prometheus-1.44.1-alpha.jar + - io.opentelemetry-opentelemetry-exporter-sender-okhttp-1.44.1.jar + - io.opentelemetry-opentelemetry-sdk-1.44.1.jar + - io.opentelemetry-opentelemetry-sdk-common-1.44.1.jar + - io.opentelemetry-opentelemetry-sdk-extension-autoconfigure-1.44.1.jar + - io.opentelemetry-opentelemetry-sdk-extension-autoconfigure-spi-1.44.1.jar + - io.opentelemetry-opentelemetry-sdk-logs-1.44.1.jar + - io.opentelemetry-opentelemetry-sdk-metrics-1.44.1.jar + - io.opentelemetry-opentelemetry-sdk-trace-1.44.1.jar - io.opentelemetry.instrumentation-opentelemetry-instrumentation-api-1.33.6.jar - io.opentelemetry.instrumentation-opentelemetry-instrumentation-api-semconv-1.33.6-alpha.jar - io.opentelemetry.instrumentation-opentelemetry-resources-1.33.6-alpha.jar diff --git a/distribution/shell/src/assemble/LICENSE.bin.txt b/distribution/shell/src/assemble/LICENSE.bin.txt index 1601f32bb2b34..07a40d3bc1bc8 100644 --- a/distribution/shell/src/assemble/LICENSE.bin.txt +++ b/distribution/shell/src/assemble/LICENSE.bin.txt @@ -388,9 +388,9 @@ The Apache Software License, Version 2.0 - log4j-slf4j2-impl-2.23.1.jar - log4j-web-2.23.1.jar * OpenTelemetry - - opentelemetry-api-1.41.0.jar - - opentelemetry-api-incubator-1.41.0-alpha.jar - - opentelemetry-context-1.41.0.jar + - opentelemetry-api-1.44.1.jar + - opentelemetry-api-incubator-1.44.1-alpha.jar + - opentelemetry-context-1.44.1.jar * BookKeeper - bookkeeper-common-allocator-4.17.1.jar diff --git a/pom.xml b/pom.xml index e6d154c1b34d6..4830358f5ea72 100644 --- a/pom.xml +++ b/pom.xml @@ -258,7 +258,7 @@ flexible messaging model and an intuitive client API. 3.4.3 1.5.2-3 2.0.6 - 1.41.0 + 1.44.1 ${opentelemetry.version}-alpha 1.33.6 ${opentelemetry.instrumentation.version}-alpha diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/BrokerOpenTelemetryTestUtil.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/BrokerOpenTelemetryTestUtil.java index 0d46e80a70302..3bfbf2064e156 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/BrokerOpenTelemetryTestUtil.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/BrokerOpenTelemetryTestUtil.java @@ -19,6 +19,7 @@ package org.apache.pulsar.broker.stats; import static io.opentelemetry.sdk.testing.assertj.OpenTelemetryAssertions.assertThat; +import io.opentelemetry.api.common.AttributeKey; import io.opentelemetry.api.common.Attributes; import io.opentelemetry.sdk.autoconfigure.AutoConfiguredOpenTelemetrySdkBuilder; import io.opentelemetry.sdk.metrics.data.MetricData; @@ -52,13 +53,14 @@ public static void disableExporters(AutoConfiguredOpenTelemetrySdkBuilder sdkBui public static void assertMetricDoubleSumValue(Collection metrics, String metricName, Attributes attributes, Consumer valueConsumer) { + Map, Object> attributesMap = attributes.asMap(); assertThat(metrics) .anySatisfy(metric -> assertThat(metric) .hasName(metricName) .hasDoubleSumSatisfying(sum -> sum.satisfies( sumData -> assertThat(sumData.getPoints()).anySatisfy( point -> { - assertThat(point.getAttributes()).isEqualTo(attributes); + assertThat(point.getAttributes().asMap()).isEqualTo(attributesMap); valueConsumer.accept(point.getValue()); })))); } @@ -70,13 +72,14 @@ public static void assertMetricLongSumValue(Collection metrics, Stri public static void assertMetricLongSumValue(Collection metrics, String metricName, Attributes attributes, Consumer valueConsumer) { + Map, Object> attributesMap = attributes.asMap(); assertThat(metrics) .anySatisfy(metric -> assertThat(metric) .hasName(metricName) .hasLongSumSatisfying(sum -> sum.satisfies( sumData -> assertThat(sumData.getPoints()).anySatisfy( point -> { - assertThat(point.getAttributes()).isEqualTo(attributes); + assertThat(point.getAttributes().asMap()).isEqualTo(attributesMap); valueConsumer.accept(point.getValue()); })))); } @@ -88,13 +91,14 @@ public static void assertMetricLongGaugeValue(Collection metrics, St public static void assertMetricLongGaugeValue(Collection metrics, String metricName, Attributes attributes, Consumer valueConsumer) { + Map, Object> attributesMap = attributes.asMap(); assertThat(metrics) .anySatisfy(metric -> assertThat(metric) .hasName(metricName) .hasLongGaugeSatisfying(gauge -> gauge.satisfies( pointData -> assertThat(pointData.getPoints()).anySatisfy( point -> { - assertThat(point.getAttributes()).isEqualTo(attributes); + assertThat(point.getAttributes().asMap()).isEqualTo(attributesMap); valueConsumer.accept(point.getValue()); })))); } @@ -106,13 +110,14 @@ public static void assertMetricDoubleGaugeValue(Collection metrics, public static void assertMetricDoubleGaugeValue(Collection metrics, String metricName, Attributes attributes, Consumer valueConsumer) { + Map, Object> attributesMap = attributes.asMap(); assertThat(metrics) .anySatisfy(metric -> assertThat(metric) .hasName(metricName) .hasDoubleGaugeSatisfying(gauge -> gauge.satisfies( pointData -> assertThat(pointData.getPoints()).anySatisfy( point -> { - assertThat(point.getAttributes()).isEqualTo(attributes); + assertThat(point.getAttributes().asMap()).isEqualTo(attributesMap); valueConsumer.accept(point.getValue()); })))); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/metrics/ClientMetricsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/metrics/ClientMetricsTest.java index 31305123c4148..02b38acf865d4 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/metrics/ClientMetricsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/metrics/ClientMetricsTest.java @@ -21,7 +21,7 @@ import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertNotNull; import static org.testng.Assert.fail; -import io.opentelemetry.api.OpenTelemetry; +import io.opentelemetry.api.common.AttributeKey; import io.opentelemetry.api.common.Attributes; import io.opentelemetry.sdk.OpenTelemetrySdk; import io.opentelemetry.sdk.metrics.SdkMeterProvider; @@ -48,7 +48,7 @@ public class ClientMetricsTest extends ProducerConsumerBase { InMemoryMetricReader reader; - OpenTelemetry otel; + OpenTelemetrySdk otel; @BeforeMethod @Override @@ -67,6 +67,14 @@ protected void setup() throws Exception { @Override protected void cleanup() throws Exception { super.internalCleanup(); + if (otel != null) { + otel.close(); + otel = null; + } + if (reader != null) { + reader.close(); + reader = null; + } } private Map collectMetrics() { @@ -88,8 +96,9 @@ private long getCounterValue(Map metrics, String name, assertNotNull(md, "metric not found: " + name); assertEquals(md.getType(), MetricDataType.LONG_SUM); + Map, Object> expectedAttributesMap = expectedAttributes.asMap(); for (var ex : md.getLongSumData().getPoints()) { - if (ex.getAttributes().equals(expectedAttributes)) { + if (ex.getAttributes().asMap().equals(expectedAttributesMap)) { return ex.getValue(); } } @@ -109,8 +118,9 @@ private long getHistoCountValue(Map metrics, String name, assertNotNull(md, "metric not found: " + name); assertEquals(md.getType(), MetricDataType.HISTOGRAM); + Map, Object> expectedAttributesMap = expectedAttributes.asMap(); for (var ex : md.getHistogramData().getPoints()) { - if (ex.getAttributes().equals(expectedAttributes)) { + if (ex.getAttributes().asMap().equals(expectedAttributesMap)) { return ex.getCount(); } } From 429f7de96826d20bc90eb54a3b40df0e31f4b42f Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 29 Nov 2024 11:17:08 +0200 Subject: [PATCH 11/22] [improve][client] Reduce unshaded dependencies and shading warnings in shaded Java client modules (#23647) --- .../server/src/assemble/LICENSE.bin.txt | 2 - .../shell/src/assemble/LICENSE.bin.txt | 3 - jetcd-core-shaded/pom.xml | 4 + pom.xml | 30 +- pulsar-broker-auth-oidc/pom.xml | 4 + pulsar-broker-common/pom.xml | 4 +- pulsar-broker/pom.xml | 4 +- pulsar-client-admin-shaded/pom.xml | 311 ++++++++++------- ...sar.shade.javax.ws.rs.client.ClientBuilder | 1 + ...lsar.shade.javax.ws.rs.ext.RuntimeDelegate | 1 + pulsar-client-admin/pom.xml | 4 +- pulsar-client-all/pom.xml | 312 ++++++++++-------- ...sar.shade.javax.ws.rs.client.ClientBuilder | 1 + ...lsar.shade.javax.ws.rs.ext.RuntimeDelegate | 1 + pulsar-client-auth-athenz/pom.xml | 5 + pulsar-client-auth-sasl/pom.xml | 4 +- pulsar-client-shaded/pom.xml | 240 ++++++++------ pulsar-common/pom.xml | 4 +- pulsar-functions/proto/pom.xml | 6 +- pulsar-functions/runtime/pom.xml | 6 +- pulsar-functions/secrets/pom.xml | 6 +- pulsar-proxy/pom.xml | 4 +- tiered-storage/jcloud/pom.xml | 4 +- 23 files changed, 580 insertions(+), 381 deletions(-) create mode 100644 pulsar-client-admin-shaded/src/main/resources/META-INF/services/org.apache.pulsar.shade.javax.ws.rs.client.ClientBuilder create mode 100644 pulsar-client-admin-shaded/src/main/resources/META-INF/services/org.apache.pulsar.shade.javax.ws.rs.ext.RuntimeDelegate create mode 100644 pulsar-client-all/src/main/resources/META-INF/services/org.apache.pulsar.shade.javax.ws.rs.client.ClientBuilder create mode 100644 pulsar-client-all/src/main/resources/META-INF/services/org.apache.pulsar.shade.javax.ws.rs.ext.RuntimeDelegate diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt index 10899bc0ae7aa..21422a41dcef9 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -571,10 +571,8 @@ Protocol Buffers License CDDL-1.1 -- ../licenses/LICENSE-CDDL-1.1.txt * Java Annotations API - com.sun.activation-javax.activation-1.2.0.jar - - javax.xml.bind-jaxb-api-2.3.1.jar * Java Servlet API -- javax.servlet-javax.servlet-api-3.1.0.jar * WebSocket Server API -- javax.websocket-javax.websocket-client-api-1.0.jar - * Java Web Service REST API -- javax.ws.rs-javax.ws.rs-api-2.1.jar * HK2 - Dependency Injection Kernel - org.glassfish.hk2-hk2-api-2.6.1.jar - org.glassfish.hk2-hk2-locator-2.6.1.jar diff --git a/distribution/shell/src/assemble/LICENSE.bin.txt b/distribution/shell/src/assemble/LICENSE.bin.txt index 07a40d3bc1bc8..a21c272f91b1d 100644 --- a/distribution/shell/src/assemble/LICENSE.bin.txt +++ b/distribution/shell/src/assemble/LICENSE.bin.txt @@ -431,11 +431,8 @@ MIT License CDDL-1.1 -- ../licenses/LICENSE-CDDL-1.1.txt * Java Annotations API - - javax.annotation-api-1.3.2.jar - javax.activation-1.2.0.jar - - jaxb-api-2.3.1.jar * WebSocket Server API -- javax.websocket-client-api-1.0.jar - * Java Web Service REST API -- javax.ws.rs-api-2.1.jar * HK2 - Dependency Injection Kernel - hk2-api-2.6.1.jar - hk2-locator-2.6.1.jar diff --git a/jetcd-core-shaded/pom.xml b/jetcd-core-shaded/pom.xml index 0b79df8278f81..2a5536987cd42 100644 --- a/jetcd-core-shaded/pom.xml +++ b/jetcd-core-shaded/pom.xml @@ -45,6 +45,10 @@ io.netty * + + javax.annotation + javax.annotation-api + diff --git a/pom.xml b/pom.xml index 4830358f5ea72..54744a253bb2b 100644 --- a/pom.xml +++ b/pom.xml @@ -220,7 +220,7 @@ flexible messaging model and an intuitive client API. 1.10 2.14.0 1.15 - 2.1 + 2.1.6 2.1.9 3.1.0 2.9.1 @@ -231,7 +231,7 @@ flexible messaging model and an intuitive client API. 2.1.0 3.24.2 1.18.32 - 1.3.2 + 1.3.5 2.3.1 1.2.0 1.2.2 @@ -854,9 +854,9 @@ flexible messaging model and an intuitive client API. - javax.ws.rs - javax.ws.rs-api - ${javax.ws.rs-api.version} + jakarta.ws.rs + jakarta.ws.rs-api + ${jakarta.ws.rs-api.version} @@ -990,6 +990,12 @@ flexible messaging model and an intuitive client API. com.yahoo.athenz athenz-zts-java-client-core ${athenz.version} + + + javax.ws.rs + javax.ws.rs-api + + @@ -1418,9 +1424,9 @@ flexible messaging model and an intuitive client API. - javax.annotation - javax.annotation-api - ${javax.annotation-api.version} + jakarta.annotation + jakarta.annotation-api + ${jakarta.annotation-api.version} @@ -1723,11 +1729,6 @@ flexible messaging model and an intuitive client API. lombok provided - - javax.annotation - javax.annotation-api - provided - @@ -2003,6 +2004,9 @@ flexible messaging model and an intuitive client API. src/assemble/LICENSE.bin.txt src/assemble/NOTICE.bin.txt + + **/META-INF/services/* + src/main/java/org/apache/bookkeeper/mledger/proto/MLDataFormats.java diff --git a/pulsar-broker-auth-oidc/pom.xml b/pulsar-broker-auth-oidc/pom.xml index 72351bf47d288..2d0931c3f2dfe 100644 --- a/pulsar-broker-auth-oidc/pom.xml +++ b/pulsar-broker-auth-oidc/pom.xml @@ -95,6 +95,10 @@ bcprov-jdk18on org.bouncycastle + + javax.annotation + javax.annotation-api + diff --git a/pulsar-broker-common/pom.xml b/pulsar-broker-common/pom.xml index 30b703cd78a92..858a234a32dcd 100644 --- a/pulsar-broker-common/pom.xml +++ b/pulsar-broker-common/pom.xml @@ -60,8 +60,8 @@ - javax.ws.rs - javax.ws.rs-api + jakarta.ws.rs + jakarta.ws.rs-api diff --git a/pulsar-broker/pom.xml b/pulsar-broker/pom.xml index 97ede1f76e969..a34f566447006 100644 --- a/pulsar-broker/pom.xml +++ b/pulsar-broker/pom.xml @@ -446,8 +446,8 @@ - javax.xml.bind - jaxb-api + jakarta.xml.bind + jakarta.xml.bind-api javax.activation diff --git a/pulsar-client-admin-shaded/pom.xml b/pulsar-client-admin-shaded/pom.xml index ab42f0e2aef59..74ced063fbfd4 100644 --- a/pulsar-client-admin-shaded/pom.xml +++ b/pulsar-client-admin-shaded/pom.xml @@ -1,4 +1,4 @@ - + - + 4.0.0 org.apache.pulsar pulsar 4.1.0-SNAPSHOT - pulsar-client-admin Pulsar Client Admin - ${project.groupId} @@ -73,7 +70,6 @@ - maven-antrun-plugin @@ -86,15 +82,13 @@ - + - org.apache.maven.plugins maven-shade-plugin @@ -107,48 +101,55 @@ true true - - org.apache.pulsar:pulsar-client-original - org.apache.pulsar:pulsar-client-admin-original - org.apache.commons:commons-lang3 - commons-codec:commons-codec - commons-collections:commons-collections - org.asynchttpclient:* - org.reactivestreams:reactive-streams - com.typesafe.netty:netty-reactive-streams - org.javassist:javassist - com.google.guava:guava + com.fasterxml.jackson.*:* + com.google.*:* + com.google.auth:* com.google.code.gson:gson + com.google.guava:guava com.google.re2j:re2j com.spotify:completable-futures - com.fasterxml.jackson.*:* - io.netty:* - io.netty.incubator:* - org.apache.pulsar:pulsar-common - org.apache.bookkeeper:* + com.squareup.*:* + com.sun.activation:javax.activation + com.typesafe.netty:netty-reactive-streams + com.yahoo.datasketches:* com.yahoo.datasketches:sketches-core - org.glassfish.jersey*:* - javax.ws.rs:* - javax.xml.bind:jaxb-api - jakarta.annotation:* - org.glassfish.hk2*:* + commons-*:* + commons-codec:commons-codec + commons-collections:commons-collections + io.airlift:* io.grpc:* + io.netty.incubator:* + io.netty:* + io.opencensus:* io.perfmark:* - com.yahoo.datasketches:* - com.squareup.*:* - com.google.*:* - commons-*:* + io.swagger:* + jakarta.activation:jakarta.activation-api + jakarta.annotation:jakarta.annotation-api + jakarta.ws.rs:jakarta.ws.rs-api + jakarta.xml.bind:jakarta.xml.bind-api + javax.ws.rs:* + javax.xml.bind:jaxb-api + net.jcip:jcip-annotations + org.apache.bookkeeper:* + org.apache.commons:commons-compress + org.apache.commons:commons-lang3 + org.apache.pulsar:pulsar-client-admin-original + + org.apache.pulsar:pulsar-client-messagecrypto-bc + org.apache.pulsar:pulsar-client-original + org.apache.pulsar:pulsar-common + org.asynchttpclient:* + org.checkerframework:* org.eclipse.jetty:* - com.google.auth:* + org.glassfish.hk2*:* + org.glassfish.jersey*:* + org.javassist:javassist org.jvnet.mimepull:* - io.opencensus:* org.objenesis:* + org.reactivestreams:reactive-streams org.yaml:snakeyaml - io.swagger:* - - org.apache.pulsar:pulsar-client-messagecrypto-bc com.fasterxml.jackson.core:jackson-annotations @@ -162,7 +163,7 @@ ** - + org/bouncycastle/** @@ -172,19 +173,54 @@ ** - + org/bouncycastle/** + + org.asynchttpclient:async-http-client + + ** + + + org/asynchttpclient/config/ahc-default.properties + + + + *:* + + **/module-info.class + findbugsExclude.xml + META-INF/*-LICENSE + META-INF/*-NOTICE + META-INF/*.DSA + META-INF/*.RSA + META-INF/*.SF + META-INF/DEPENDENCIES* + META-INF/io.netty.versions.properties + META-INF/LICENSE* + META-INF/license/** + META-INF/MANIFEST.MF + META-INF/maven/** + META-INF/native-image/** + META-INF/NOTICE* + META-INF/proguard/** + + - - org.asynchttpclient - org.apache.pulsar.shade.org.asynchttpclient + + + (META-INF/native/(lib)?)(netty.+\.(so|jnilib|dll))$ + $1org_apache_pulsar_shade_$3 + true - org.apache.commons - org.apache.pulsar.shade.org.apache.commons + com.fasterxml.jackson + org.apache.pulsar.shade.com.fasterxml.jackson + + com.fasterxml.jackson.annotation.* + com.google @@ -198,44 +234,61 @@ org.apache.pulsar.shade.com.spotify.futures - com.fasterxml.jackson - org.apache.pulsar.shade.com.fasterxml.jackson - - com.fasterxml.jackson.annotation.* - + com.squareup + org.apache.pulsar.shade.com.squareup - io.netty - org.apache.pulsar.shade.io.netty + com.sun.activation + org.apache.pulsar.shade.com.sun.activation - org.apache.pulsar.policies - org.apache.pulsar.shade.org.apache.pulsar.policies - - - org.apache.pulsar.checksum - org.apache.pulsar.shade.org.apache.pulsar.checksum + com.typesafe + org.apache.pulsar.shade.com.typesafe com.yahoo org.apache.pulsar.shade.com.yahoo - com.typesafe - org.apache.pulsar.shade.com.typesafe + io.airlift + org.apache.pulsar.shade.io.airlift - org.glassfish - org.apache.pulsar.shade.org.glassfish + io.grpc + org.apache.pulsar.shade.io.grpc - javax.ws - org.apache.pulsar.shade.javax.ws + io.netty + org.apache.pulsar.shade.io.netty + + + io.opencensus + org.apache.pulsar.shade.io.opencensus + + + io.swagger + org.apache.pulsar.shade.io.swagger + + + javassist + org.apache.pulsar.shade.javassist + + + javax.activation + org.apache.pulsar.shade.javax.activation javax.annotation org.apache.pulsar.shade.javax.annotation + + javax.inject + org.apache.pulsar.shade.javax.inject + + + javax.ws + org.apache.pulsar.shade.javax.ws + javax.xml.bind org.apache.pulsar.shade.javax.xml.bind @@ -245,71 +298,95 @@ org.apache.pulsar.shade.jersey - org.jvnet - org.apache.pulsar.shade.org.jvnet + META-INF/versions/(\d+)/com/fasterxml/jackson/core/ + META-INF/versions/$1/org/apache/pulsar/shade/com/fasterxml/jackson/core/ + + true + + + META-INF/versions/(\d+)/javax/xml/bind/ + META-INF/versions/$1/org/apache/pulsar/shade/javax/xml/bind/ + true + + + META-INF/versions/(\d+)/org/glassfish/ + META-INF/versions/$1/org/apache/pulsar/shade/org/glassfish/ + true + + + META-INF/versions/(\d+)/org/yaml/ + META-INF/versions/$1/org/apache/pulsar/shade/org/yaml/ + true + + + net.jcip + org.apache.pulsar.shade.net.jcip + + + okio + org.apache.pulsar.shade.okio org.aopalliance org.apache.pulsar.shade.org.aopalliance - javassist - org.apache.pulsar.shade.javassist + org.apache.bookkeeper + org.apache.pulsar.shade.org.apache.bookkeeper - javax.inject - org.apache.pulsar.shade.javax.inject + org.apache.commons + org.apache.pulsar.shade.org.apache.commons + + + org.apache.pulsar.checksum + org.apache.pulsar.shade.org.apache.pulsar.checksum + + + org.apache.pulsar.policies + org.apache.pulsar.shade.org.apache.pulsar.policies + + + org.asynchttpclient + org.apache.pulsar.shade.org.asynchttpclient + + + org.checkerframework + org.apache.pulsar.shade.org.checkerframework + + + org.eclipse.jetty + org.apache.pulsar.shade.org.eclipse.jetty + + + org.glassfish + org.apache.pulsar.shade.org.glassfish + + + org.jvnet + org.apache.pulsar.shade.org.jvnet + + + org.objenesis + org.apache.pulsar.shade.org.objenesis org.reactivestreams org.apache.pulsar.shade.org.reactivestreams - - io.grpc - org.apache.pulsar.shade.io.grpc - - - okio - org.apache.pulsar.shade.okio - - - com.squareup - org.apache.pulsar.shade.com.squareup - - - io.opencensus - org.apache.pulsar.shade.io.opencensus - - - org.eclipse.jetty - org.apache.pulsar.shade.org.eclipse.jetty - - - org.objenesis - org.apache.pulsar.shade.org.objenesis - - - org.yaml - org.apache.pulsar.shade.org.yaml - - - io.swagger - org.apache.pulsar.shade.io.swagger - - - org.apache.bookkeeper - org.apache.pulsar.shade.org.apache.bookkeeper - - - - (META-INF/native/(lib)?)(netty.+\.(so|jnilib|dll))$ - $1org_apache_pulsar_shade_$3 - true - + + org.yaml + org.apache.pulsar.shade.org.yaml + - - + + + + + true + + diff --git a/pulsar-client-admin-shaded/src/main/resources/META-INF/services/org.apache.pulsar.shade.javax.ws.rs.client.ClientBuilder b/pulsar-client-admin-shaded/src/main/resources/META-INF/services/org.apache.pulsar.shade.javax.ws.rs.client.ClientBuilder new file mode 100644 index 0000000000000..99a08cc8ca4d1 --- /dev/null +++ b/pulsar-client-admin-shaded/src/main/resources/META-INF/services/org.apache.pulsar.shade.javax.ws.rs.client.ClientBuilder @@ -0,0 +1 @@ +org.apache.pulsar.shade.org.glassfish.jersey.client.JerseyClientBuilder \ No newline at end of file diff --git a/pulsar-client-admin-shaded/src/main/resources/META-INF/services/org.apache.pulsar.shade.javax.ws.rs.ext.RuntimeDelegate b/pulsar-client-admin-shaded/src/main/resources/META-INF/services/org.apache.pulsar.shade.javax.ws.rs.ext.RuntimeDelegate new file mode 100644 index 0000000000000..0adc919b7f0c2 --- /dev/null +++ b/pulsar-client-admin-shaded/src/main/resources/META-INF/services/org.apache.pulsar.shade.javax.ws.rs.ext.RuntimeDelegate @@ -0,0 +1 @@ +org.apache.pulsar.shade.org.glassfish.jersey.internal.RuntimeDelegateImpl \ No newline at end of file diff --git a/pulsar-client-admin/pom.xml b/pulsar-client-admin/pom.xml index 36070618ed891..0a94e48e9b939 100644 --- a/pulsar-client-admin/pom.xml +++ b/pulsar-client-admin/pom.xml @@ -76,8 +76,8 @@ - javax.xml.bind - jaxb-api + jakarta.xml.bind + jakarta.xml.bind-api javax.activation diff --git a/pulsar-client-all/pom.xml b/pulsar-client-all/pom.xml index e26f6eeac57bf..74007745c70ee 100644 --- a/pulsar-client-all/pom.xml +++ b/pulsar-client-all/pom.xml @@ -1,3 +1,4 @@ + + xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> 4.0.0 - org.apache.pulsar pulsar 4.1.0-SNAPSHOT - pulsar-client-all Pulsar Client All - ${project.groupId} @@ -69,7 +67,6 @@ test - @@ -107,7 +104,6 @@ - maven-antrun-plugin @@ -120,15 +116,13 @@ - + - org.apache.maven.plugins @@ -145,67 +139,67 @@ true true false - - org.apache.pulsar:pulsar-client-original - org.apache.pulsar:pulsar-client-admin-original - org.apache.commons:commons-lang3 - commons-codec:commons-codec - commons-collections:commons-collections - org.asynchttpclient:* - io.netty:netty-codec-http - io.netty:netty-transport-native-epoll - org.reactivestreams:reactive-streams - com.typesafe.netty:netty-reactive-streams - org.javassist:javassist - com.google.guava:* - org.checkerframework:* + com.fasterxml.jackson.*:* + com.google.*:* + com.google.auth:* com.google.code.findbugs:* + com.google.code.gson:gson com.google.errorprone:* + com.google.guava:* com.google.j2objc:* - com.google.code.gson:gson com.google.re2j:re2j com.spotify:completable-futures - com.fasterxml.jackson.*:* - io.netty:netty - io.netty:netty-all - io.netty:netty-tcnative-boringssl-static - org.eclipse.jetty:* + com.squareup.*:* + com.sun.activation:javax.activation + + com.thoughtworks.paranamer:paranamer + com.typesafe.netty:netty-reactive-streams com.yahoo.datasketches:* + com.yahoo.datasketches:sketches-core commons-*:* - io.swagger:* + commons-codec:commons-codec + commons-collections:commons-collections io.airlift:* - - org.apache.pulsar:pulsar-common - org.apache.bookkeeper:* - com.yahoo.datasketches:sketches-core - org.glassfish.jersey*:* - javax.ws.rs:* - javax.xml.bind:jaxb-api - jakarta.annotation:* - org.glassfish.hk2*:* io.grpc:* - io.perfmark:* - com.yahoo.datasketches:* - io.netty:* io.netty.incubator:* - com.squareup.*:* - com.google.*:* - commons-*:* - org.eclipse.jetty:* - com.google.auth:* - org.jvnet.mimepull:* + io.netty:* + io.netty:netty + io.netty:netty-all + io.netty:netty-codec-http + io.netty:netty-tcnative-boringssl-static + io.netty:netty-transport-native-epoll io.opencensus:* - org.objenesis:* - org.yaml:snakeyaml + io.perfmark:* + io.swagger:* + jakarta.activation:jakarta.activation-api + jakarta.annotation:jakarta.annotation-api + jakarta.ws.rs:jakarta.ws.rs-api + jakarta.xml.bind:jakarta.xml.bind-api + javax.ws.rs:* + javax.xml.bind:jaxb-api + net.jcip:jcip-annotations org.apache.avro:* - - com.thoughtworks.paranamer:paranamer + org.apache.bookkeeper:* org.apache.commons:commons-compress - org.tukaani:xz + org.apache.commons:commons-lang3 + org.apache.pulsar:pulsar-client-admin-original org.apache.pulsar:pulsar-client-messagecrypto-bc + org.apache.pulsar:pulsar-client-original + org.apache.pulsar:pulsar-common + org.asynchttpclient:* + org.checkerframework:* + org.eclipse.jetty:* + org.glassfish.hk2*:* + org.glassfish.jersey*:* + org.javassist:javassist + org.jvnet.mimepull:* + org.objenesis:* + org.reactivestreams:reactive-streams + org.tukaani:xz + org.yaml:snakeyaml com.fasterxml.jackson.core:jackson-annotations @@ -219,23 +213,54 @@ ** - + org/bouncycastle/** + + org.asynchttpclient:async-http-client + + ** + + + org/asynchttpclient/config/ahc-default.properties + + + + *:* + + **/module-info.class + findbugsExclude.xml + META-INF/*-LICENSE + META-INF/*-NOTICE + META-INF/*.DSA + META-INF/*.RSA + META-INF/*.SF + META-INF/DEPENDENCIES* + META-INF/io.netty.versions.properties + META-INF/LICENSE* + META-INF/license/** + META-INF/MANIFEST.MF + META-INF/maven/** + META-INF/native-image/** + META-INF/NOTICE* + META-INF/proguard/** + + + - org.asynchttpclient - org.apache.pulsar.shade.org.asynchttpclient - - - org.apache.bookkeeper - org.apache.pulsar.shade.org.apache.bookkeeper + (META-INF/native/(lib)?)(netty.+\.(so|jnilib|dll))$ + $1org_apache_pulsar_shade_$3 + true - org.apache.commons - org.apache.pulsar.shade.org.apache.commons + com.fasterxml.jackson + org.apache.pulsar.shade.com.fasterxml.jackson + + com.fasterxml.jackson.annotation.* + com.google @@ -249,103 +274,109 @@ org.apache.pulsar.shade.com.spotify.futures - com.fasterxml.jackson - org.apache.pulsar.shade.com.fasterxml.jackson - - com.fasterxml.jackson.annotation.* - + com.squareup + org.apache.pulsar.shade.com.squareup - io.netty - org.apache.pulsar.shade.io.netty + com.sun.activation + org.apache.pulsar.shade.com.sun.activation - org.apache.pulsar.policies - org.apache.pulsar.shade.org.apache.pulsar.policies + com.thoughtworks.paranamer + org.apache.pulsar.shade.com.thoughtworks.paranamer - com.yahoo.datasketches - org.apache.pulsar.shade.com.yahoo.datasketches + com.typesafe + org.apache.pulsar.shade.com.typesafe com.yahoo org.apache.pulsar.shade.com.yahoo - org.eclipse.jetty - org.apache.pulsar.shade.org.eclipse - - - org.reactivestreams - org.apache.pulsar.shade.org.reactivestreams + com.yahoo.datasketches + org.apache.pulsar.shade.com.yahoo.datasketches - com.typesafe - org.apache.pulsar.shade.com.typesafe + io.airlift + org.apache.pulsar.shade.io.airlift - javax.ws - org.apache.pulsar.shade.javax.ws + io.grpc + org.apache.pulsar.shade.io.grpc - javax.annotation - org.apache.pulsar.shade.javax.annotation + io.netty + org.apache.pulsar.shade.io.netty - javax.xml.bind - org.apache.pulsar.shade.javax.xml.bind + io.opencensus + org.apache.pulsar.shade.io.opencensus - jersey - org.apache.pulsar.shade.jersey + io.swagger + org.apache.pulsar.shade.io.swagger - org.jvnet - org.apache.pulsar.shade.org.jvnet + javassist + org.apache.pulsar.shade.javassist - org.aopalliance - org.apache.pulsar.shade.org.aopalliance + javax.activation + org.apache.pulsar.shade.javax.activation - javassist - org.apache.pulsar.shade.javassist + javax.annotation + org.apache.pulsar.shade.javax.annotation javax.inject org.apache.pulsar.shade.javax.inject - org.glassfish - org.apache.pulsar.shade.org.glassfish + javax.ws + org.apache.pulsar.shade.javax.ws + + + javax.xml.bind + org.apache.pulsar.shade.javax.xml.bind + + + jersey + org.apache.pulsar.shade.jersey - io.grpc - org.apache.pulsar.shade.io.grpc + META-INF/versions/(\d+)/com/fasterxml/jackson/core/ + META-INF/versions/$1/org/apache/pulsar/shade/com/fasterxml/jackson/core/ + + true - okio - org.apache.pulsar.shade.okio + META-INF/versions/(\d+)/javax/xml/bind/ + META-INF/versions/$1/org/apache/pulsar/shade/javax/xml/bind/ + true - com.squareup - org.apache.pulsar.shade.com.squareup + META-INF/versions/(\d+)/org/glassfish/ + META-INF/versions/$1/org/apache/pulsar/shade/org/glassfish/ + true - io.opencensus - org.apache.pulsar.shade.io.opencensus + META-INF/versions/(\d+)/org/yaml/ + META-INF/versions/$1/org/apache/pulsar/shade/org/yaml/ + true - org.eclipse.jetty - org.apache.pulsar.shade.org.eclipse.jetty + net.jcip + org.apache.pulsar.shade.net.jcip - org.objenesis - org.apache.pulsar.shade.org.objenesis + okio + org.apache.pulsar.shade.okio - org.yaml - org.apache.pulsar.shade.org.yaml + org.aopalliance + org.apache.pulsar.shade.org.aopalliance org.apache.avro @@ -363,45 +394,68 @@ org.apache.avro.reflect.Union - - - org.codehaus.jackson - org.apache.pulsar.shade.org.codehaus.jackson - - com.thoughtworks.paranamer - org.apache.pulsar.shade.com.thoughtworks.paranamer + org.apache.bookkeeper + org.apache.pulsar.shade.org.apache.bookkeeper org.apache.commons org.apache.pulsar.shade.org.apache.commons - io.airlift - org.apache.pulsar.shade.io.airlift + org.apache.pulsar.policies + org.apache.pulsar.shade.org.apache.pulsar.policies + + + org.asynchttpclient + org.apache.pulsar.shade.org.asynchttpclient org.checkerframework org.apache.pulsar.shade.org.checkerframework + - javax.annotation - org.apache.pulsar.shade.javax.annotation + org.codehaus.jackson + org.apache.pulsar.shade.org.codehaus.jackson + + + org.eclipse.jetty + org.apache.pulsar.shade.org.eclipse + + + org.glassfish + org.apache.pulsar.shade.org.glassfish + + + org.jvnet + org.apache.pulsar.shade.org.jvnet + + + org.objenesis + org.apache.pulsar.shade.org.objenesis + + + org.reactivestreams + org.apache.pulsar.shade.org.reactivestreams org.tukaani org.apache.pulsar.shade.org.tukaani - - (META-INF/native/(lib)?)(netty.+\.(so|jnilib|dll))$ - $1org_apache_pulsar_shade_$3 - true + org.yaml + org.apache.pulsar.shade.org.yaml - - + + + + + true + + diff --git a/pulsar-client-all/src/main/resources/META-INF/services/org.apache.pulsar.shade.javax.ws.rs.client.ClientBuilder b/pulsar-client-all/src/main/resources/META-INF/services/org.apache.pulsar.shade.javax.ws.rs.client.ClientBuilder new file mode 100644 index 0000000000000..99a08cc8ca4d1 --- /dev/null +++ b/pulsar-client-all/src/main/resources/META-INF/services/org.apache.pulsar.shade.javax.ws.rs.client.ClientBuilder @@ -0,0 +1 @@ +org.apache.pulsar.shade.org.glassfish.jersey.client.JerseyClientBuilder \ No newline at end of file diff --git a/pulsar-client-all/src/main/resources/META-INF/services/org.apache.pulsar.shade.javax.ws.rs.ext.RuntimeDelegate b/pulsar-client-all/src/main/resources/META-INF/services/org.apache.pulsar.shade.javax.ws.rs.ext.RuntimeDelegate new file mode 100644 index 0000000000000..0adc919b7f0c2 --- /dev/null +++ b/pulsar-client-all/src/main/resources/META-INF/services/org.apache.pulsar.shade.javax.ws.rs.ext.RuntimeDelegate @@ -0,0 +1 @@ +org.apache.pulsar.shade.org.glassfish.jersey.internal.RuntimeDelegateImpl \ No newline at end of file diff --git a/pulsar-client-auth-athenz/pom.xml b/pulsar-client-auth-athenz/pom.xml index 99786b4d18dd8..0e994ce25c24e 100644 --- a/pulsar-client-auth-athenz/pom.xml +++ b/pulsar-client-auth-athenz/pom.xml @@ -46,6 +46,11 @@ athenz-zts-java-client-core + + jakarta.ws.rs + jakarta.ws.rs-api + + com.yahoo.athenz athenz-cert-refresher diff --git a/pulsar-client-auth-sasl/pom.xml b/pulsar-client-auth-sasl/pom.xml index 1a4720b3d589d..61d1157afda34 100644 --- a/pulsar-client-auth-sasl/pom.xml +++ b/pulsar-client-auth-sasl/pom.xml @@ -57,8 +57,8 @@ - javax.ws.rs - javax.ws.rs-api + jakarta.ws.rs + jakarta.ws.rs-api diff --git a/pulsar-client-shaded/pom.xml b/pulsar-client-shaded/pom.xml index cd87d901b21bb..1093b405731ea 100644 --- a/pulsar-client-shaded/pom.xml +++ b/pulsar-client-shaded/pom.xml @@ -1,3 +1,4 @@ + + xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> 4.0.0 - org.apache.pulsar pulsar 4.1.0-SNAPSHOT - pulsar-client Pulsar Client Java - ${project.groupId} @@ -48,11 +46,8 @@ ${project.version} - - - org.apache.maven.plugins maven-dependency-plugin @@ -88,7 +83,6 @@ - maven-antrun-plugin @@ -101,15 +95,13 @@ - + - org.apache.maven.plugins @@ -124,50 +116,54 @@ true true false - - org.apache.pulsar:pulsar-client-original - org.apache.bookkeeper:* - org.apache.commons:commons-lang3 - commons-codec:commons-codec - commons-collections:commons-collections - org.asynchttpclient:* - io.netty:netty-codec-http - io.netty:netty-transport-native-epoll - org.reactivestreams:reactive-streams - com.typesafe.netty:netty-reactive-streams - org.javassist:javassist - com.google.guava:* - org.checkerframework:* + com.fasterxml.jackson.*:* com.google.code.findbugs:* + com.google.code.gson:gson com.google.errorprone:* + com.google.guava:* com.google.j2objc:* - com.google.code.gson:gson com.google.re2j:re2j com.spotify:completable-futures - com.fasterxml.jackson.*:* - io.netty:* - io.netty.incubator:* - io.perfmark:* - org.eclipse.jetty:* + com.sun.activation:javax.activation + + com.thoughtworks.paranamer:paranamer + com.typesafe.netty:netty-reactive-streams com.yahoo.datasketches:* + com.yahoo.datasketches:sketches-core commons-*:* - io.swagger:* + commons-codec:commons-codec + commons-collections:commons-collections io.airlift:* - - org.apache.pulsar:pulsar-common - com.yahoo.datasketches:sketches-core - org.objenesis:* - org.yaml:snakeyaml - + io.netty.incubator:* + io.netty:* + io.netty:netty-codec-http + io.netty:netty-transport-native-epoll + io.perfmark:* + io.swagger:* + jakarta.activation:jakarta.activation-api + jakarta.annotation:jakarta.annotation-api + jakarta.ws.rs:jakarta.ws.rs-api + jakarta.xml.bind:jakarta.xml.bind-api + javax.ws.rs:* + net.jcip:jcip-annotations org.apache.avro:* - - com.thoughtworks.paranamer:paranamer + org.apache.bookkeeper:* org.apache.commons:commons-compress - org.tukaani:xz + org.apache.commons:commons-lang3 org.apache.pulsar:pulsar-client-messagecrypto-bc + org.apache.pulsar:pulsar-client-original + org.apache.pulsar:pulsar-common + org.asynchttpclient:* + org.checkerframework:* + org.eclipse.jetty:* + org.javassist:javassist + org.objenesis:* + org.reactivestreams:reactive-streams + org.tukaani:xz + org.yaml:snakeyaml com.fasterxml.jackson.core:jackson-annotations @@ -180,23 +176,45 @@ ** - + org/bouncycastle/** + + *:* + + **/module-info.class + findbugsExclude.xml + META-INF/*-LICENSE + META-INF/*-NOTICE + META-INF/*.DSA + META-INF/*.RSA + META-INF/*.SF + META-INF/DEPENDENCIES* + META-INF/io.netty.versions.properties + META-INF/LICENSE* + META-INF/license/** + META-INF/MANIFEST.MF + META-INF/maven/** + META-INF/native-image/** + META-INF/NOTICE* + META-INF/proguard/** + + + - org.asynchttpclient - org.apache.pulsar.shade.org.asynchttpclient - - - org.apache.commons - org.apache.pulsar.shade.org.apache.commons + (META-INF/native/(lib)?)(netty.+\.(so|jnilib|dll))$ + $1org_apache_pulsar_shade_$3 + true - io.airlift - org.apache.pulsar.shade.io.airlift + com.fasterxml.jackson + org.apache.pulsar.shade.com.fasterxml.jackson + + com.fasterxml.jackson.annotation.* + com.google @@ -210,63 +228,67 @@ org.apache.pulsar.shade.com.spotify.futures - com.fasterxml.jackson - org.apache.pulsar.shade.com.fasterxml.jackson - - com.fasterxml.jackson.annotation.* - + com.sun.activation + org.apache.pulsar.shade.com.sun.activation - io.netty - org.apache.pulsar.shade.io.netty + com.thoughtworks.paranamer + org.apache.pulsar.shade.com.thoughtworks.paranamer - org.checkerframework - org.apache.pulsar.shade.org.checkerframework + com.typesafe + org.apache.pulsar.shade.com.typesafe - javax.annotation - org.apache.pulsar.shade.javax.annotation + com.yahoo.datasketches + org.apache.pulsar.shade.com.yahoo.datasketches - io.swagger - org.apache.pulsar.shade.io.swagger + com.yahoo.memory + org.apache.pulsar.shade.com.yahoo.memory - org.apache.pulsar.policies - org.apache.pulsar.shade.org.apache.pulsar.policies + com.yahoo.sketches + org.apache.pulsar.shade.com.yahoo.sketches - com.yahoo.datasketches - org.apache.pulsar.shade.com.yahoo.datasketches + io.airlift + org.apache.pulsar.shade.io.airlift - com.yahoo.sketches - org.apache.pulsar.shade.com.yahoo.sketches + io.netty + org.apache.pulsar.shade.io.netty - org.eclipse.jetty - org.apache.pulsar.shade.org.eclipse + io.swagger + org.apache.pulsar.shade.io.swagger - org.reactivestreams - org.apache.pulsar.shade.org.reactivestreams + javax.activation + org.apache.pulsar.shade.javax.activation - com.typesafe - org.apache.pulsar.shade.com.typesafe + javax.annotation + org.apache.pulsar.shade.javax.annotation - com.yahoo.memory - org.apache.pulsar.shade.com.yahoo.memory + javax.ws + org.apache.pulsar.shade.javax.ws - org.objenesis - org.apache.pulsar.shade.org.objenesis + META-INF/versions/(\d+)/com/fasterxml/jackson/core/ + META-INF/versions/$1/org/apache/pulsar/shade/com/fasterxml/jackson/core/ + + true - org.yaml - org.apache.pulsar.shade.org.yaml + META-INF/versions/(\d+)/org/yaml/ + META-INF/versions/$1/org/apache/pulsar/shade/org/yaml/ + true + + + net.jcip + org.apache.pulsar.shade.net.jcip org.apache.avro @@ -284,43 +306,65 @@ org.apache.avro.reflect.Union - + + org.apache.bookkeeper + org.apache.pulsar.shade.org.apache.bookkeeper + + + org.apache.commons + org.apache.pulsar.shade.org.apache.commons + + + org.apache.pulsar.policies + org.apache.pulsar.shade.org.apache.pulsar.policies + + + org.asynchttpclient + org.apache.pulsar.shade.org.asynchttpclient + + + org.checkerframework + org.apache.pulsar.shade.org.checkerframework + + org.codehaus.jackson org.apache.pulsar.shade.org.codehaus.jackson - com.thoughtworks.paranamer - org.apache.pulsar.shade.com.thoughtworks.paranamer + org.eclipse.jetty + org.apache.pulsar.shade.org.eclipse - org.apache.commons - org.apache.pulsar.shade.org.apache.commons + org.objenesis + org.apache.pulsar.shade.org.objenesis - org.tukaani - org.apache.pulsar.shade.org.tukaani + org.reactivestreams + org.apache.pulsar.shade.org.reactivestreams - org.apache.bookkeeper - org.apache.pulsar.shade.org.apache.bookkeeper + org.tukaani + org.apache.pulsar.shade.org.tukaani - - (META-INF/native/(lib)?)(netty.+\.(so|jnilib|dll))$ - $1org_apache_pulsar_shade_$3 - true + org.yaml + org.apache.pulsar.shade.org.yaml - - + + + + + true + + - com.github.spotbugs spotbugs-maven-plugin diff --git a/pulsar-common/pom.xml b/pulsar-common/pom.xml index 241dd173ea9a4..c823b8408c9bd 100644 --- a/pulsar-common/pom.xml +++ b/pulsar-common/pom.xml @@ -184,8 +184,8 @@ - javax.ws.rs - javax.ws.rs-api + jakarta.ws.rs + jakarta.ws.rs-api diff --git a/pulsar-functions/proto/pom.xml b/pulsar-functions/proto/pom.xml index db87ff7ef2031..e9e9678d1b9b7 100644 --- a/pulsar-functions/proto/pom.xml +++ b/pulsar-functions/proto/pom.xml @@ -44,10 +44,10 @@ com.google.protobuf protobuf-java-util - + - javax.annotation - javax.annotation-api + jakarta.annotation + jakarta.annotation-api diff --git a/pulsar-functions/runtime/pom.xml b/pulsar-functions/runtime/pom.xml index b04d25a5af08a..b2df8f224bc26 100644 --- a/pulsar-functions/runtime/pom.xml +++ b/pulsar-functions/runtime/pom.xml @@ -77,6 +77,10 @@ bcprov-jdk18on org.bouncycastle + + javax.annotation + javax.annotation-api + @@ -106,7 +110,7 @@ - + org.apache.maven.plugins diff --git a/pulsar-functions/secrets/pom.xml b/pulsar-functions/secrets/pom.xml index c7ab69ec612db..ac7d89a0a0c27 100644 --- a/pulsar-functions/secrets/pom.xml +++ b/pulsar-functions/secrets/pom.xml @@ -48,6 +48,10 @@ bcprov-jdk18on org.bouncycastle + + javax.annotation + javax.annotation-api + @@ -76,7 +80,7 @@ - + com.github.spotbugs spotbugs-maven-plugin diff --git a/pulsar-proxy/pom.xml b/pulsar-proxy/pom.xml index 73ed347d24fee..2cea6e0893005 100644 --- a/pulsar-proxy/pom.xml +++ b/pulsar-proxy/pom.xml @@ -134,8 +134,8 @@ - javax.xml.bind - jaxb-api + jakarta.xml.bind + jakarta.xml.bind-api diff --git a/tiered-storage/jcloud/pom.xml b/tiered-storage/jcloud/pom.xml index 366ba9ae38cc9..8fa504227feb1 100644 --- a/tiered-storage/jcloud/pom.xml +++ b/tiered-storage/jcloud/pom.xml @@ -104,8 +104,8 @@ - javax.xml.bind - jaxb-api + jakarta.xml.bind + jakarta.xml.bind-api javax.activation From 280997e688722be7240e777d5ca3500362c689bf Mon Sep 17 00:00:00 2001 From: Qiang Zhao Date: Fri, 29 Nov 2024 22:30:16 +0800 Subject: [PATCH 12/22] [fix][broker] support missing tenant level fine-granted permissions (#23660) --- .../pulsar/broker/admin/impl/TenantsBase.java | 48 +++++- .../TenantEndpointsAuthorizationTest.java | 160 ++++++++++++++++++ .../common/policies/data/TenantOperation.java | 6 + 3 files changed, 209 insertions(+), 5 deletions(-) create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TenantEndpointsAuthorizationTest.java diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/TenantsBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/TenantsBase.java index 0d1f79a09dc14..ff32e41977aaa 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/TenantsBase.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/TenantsBase.java @@ -48,6 +48,7 @@ import org.apache.pulsar.common.naming.NamedEntity; import org.apache.pulsar.common.policies.data.TenantInfo; import org.apache.pulsar.common.policies.data.TenantInfoImpl; +import org.apache.pulsar.common.policies.data.TenantOperation; import org.apache.pulsar.common.util.FutureUtil; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -62,7 +63,7 @@ public class TenantsBase extends PulsarWebResource { @ApiResponse(code = 404, message = "Tenant doesn't exist")}) public void getTenants(@Suspended final AsyncResponse asyncResponse) { final String clientAppId = clientAppId(); - validateSuperUserAccessAsync() + validateBothSuperUserAndTenantOperation(null, TenantOperation.LIST_TENANTS) .thenCompose(__ -> tenantResources().listTenantsAsync()) .thenAccept(tenants -> { // deep copy the tenants to avoid concurrent sort exception @@ -84,7 +85,7 @@ public void getTenants(@Suspended final AsyncResponse asyncResponse) { public void getTenantAdmin(@Suspended final AsyncResponse asyncResponse, @ApiParam(value = "The tenant name") @PathParam("tenant") String tenant) { final String clientAppId = clientAppId(); - validateSuperUserAccessAsync() + validateBothSuperUserAndTenantOperation(tenant, TenantOperation.GET_TENANT) .thenCompose(__ -> tenantResources().getTenantAsync(tenant)) .thenApply(tenantInfo -> { if (!tenantInfo.isPresent()) { @@ -121,7 +122,7 @@ public void createTenant(@Suspended final AsyncResponse asyncResponse, asyncResponse.resume(new RestException(Status.PRECONDITION_FAILED, "Tenant name is not valid")); return; } - validateSuperUserAccessAsync() + validateBothSuperUserAndTenantOperation(tenant, TenantOperation.CREATE_TENANT) .thenCompose(__ -> validatePoliciesReadOnlyAccessAsync()) .thenCompose(__ -> validateClustersAsync(tenantInfo)) .thenCompose(__ -> validateAdminRoleAsync(tenantInfo)) @@ -169,7 +170,7 @@ public void updateTenant(@Suspended final AsyncResponse asyncResponse, @ApiParam(value = "The tenant name") @PathParam("tenant") String tenant, @ApiParam(value = "TenantInfo") TenantInfoImpl newTenantAdmin) { final String clientAppId = clientAppId(); - validateSuperUserAccessAsync() + validateBothSuperUserAndTenantOperation(tenant, TenantOperation.UPDATE_TENANT) .thenCompose(__ -> validatePoliciesReadOnlyAccessAsync()) .thenCompose(__ -> validateClustersAsync(newTenantAdmin)) .thenCompose(__ -> validateAdminRoleAsync(newTenantAdmin)) @@ -206,7 +207,7 @@ public void deleteTenant(@Suspended final AsyncResponse asyncResponse, @PathParam("tenant") @ApiParam(value = "The tenant name") String tenant, @QueryParam("force") @DefaultValue("false") boolean force) { final String clientAppId = clientAppId(); - validateSuperUserAccessAsync() + validateBothSuperUserAndTenantOperation(tenant, TenantOperation.DELETE_TENANT) .thenCompose(__ -> validatePoliciesReadOnlyAccessAsync()) .thenCompose(__ -> internalDeleteTenant(tenant, force)) .thenAccept(__ -> { @@ -304,4 +305,41 @@ private CompletableFuture validateAdminRoleAsync(TenantInfoImpl info) { } return CompletableFuture.completedFuture(null); } + + private CompletableFuture validateBothSuperUserAndTenantOperation(String tenant, + TenantOperation operation) { + final var superUserValidationFuture = validateSuperUserAccessAsync(); + final var tenantOperationValidationFuture = validateTenantOperationAsync(tenant, operation); + return CompletableFuture.allOf(superUserValidationFuture, tenantOperationValidationFuture) + .handle((__, err) -> { + if (!superUserValidationFuture.isCompletedExceptionally() + || !tenantOperationValidationFuture.isCompletedExceptionally()) { + return true; + } + if (log.isDebugEnabled()) { + Throwable superUserValidationException = null; + try { + superUserValidationFuture.join(); + } catch (Throwable ex) { + superUserValidationException = FutureUtil.unwrapCompletionException(ex); + } + Throwable brokerOperationValidationException = null; + try { + tenantOperationValidationFuture.join(); + } catch (Throwable ex) { + brokerOperationValidationException = FutureUtil.unwrapCompletionException(ex); + } + log.debug("validateBothTenantOperationAndSuperUser failed." + + " originalPrincipal={} clientAppId={} operation={} " + + "superuserValidationError={} tenantOperationValidationError={}", + originalPrincipal(), clientAppId(), operation.toString(), + superUserValidationException, brokerOperationValidationException); + } + throw new RestException(Status.UNAUTHORIZED, + String.format("Unauthorized to validateBothTenantOperationAndSuperUser for" + + " originalPrincipal [%s] and clientAppId [%s] " + + "about operation [%s] ", + originalPrincipal(), clientAppId(), operation.toString())); + }); + } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TenantEndpointsAuthorizationTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TenantEndpointsAuthorizationTest.java new file mode 100644 index 0000000000000..2cf3ea374c33c --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TenantEndpointsAuthorizationTest.java @@ -0,0 +1,160 @@ +/* + * 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.admin; + +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.ArgumentMatchers.isNull; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.verify; +import lombok.SneakyThrows; +import org.apache.commons.lang3.reflect.FieldUtils; +import org.apache.pulsar.broker.authorization.AuthorizationService; +import org.apache.pulsar.client.admin.PulsarAdmin; +import org.apache.pulsar.client.admin.PulsarAdminException; +import org.apache.pulsar.client.impl.auth.AuthenticationToken; +import org.apache.pulsar.common.policies.data.TenantInfo; +import org.apache.pulsar.common.policies.data.TenantOperation; +import org.apache.pulsar.security.MockedPulsarStandalone; +import org.mockito.Mockito; +import org.testng.Assert; +import org.testng.annotations.AfterClass; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; +import java.util.Set; +import java.util.UUID; + +@Test(groups = "broker-admin") +public class TenantEndpointsAuthorizationTest extends MockedPulsarStandalone { + + private AuthorizationService orignalAuthorizationService; + private AuthorizationService spyAuthorizationService; + + private PulsarAdmin superUserAdmin; + private PulsarAdmin nobodyAdmin; + + @SneakyThrows + @BeforeClass(alwaysRun = true) + public void setup() { + configureTokenAuthentication(); + configureDefaultAuthorization(); + start(); + this.superUserAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(SUPER_USER_TOKEN)) + .build(); + this.nobodyAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(NOBODY_TOKEN)) + .build(); + } + + @BeforeMethod(alwaysRun = true) + public void before() throws IllegalAccessException { + orignalAuthorizationService = getPulsarService().getBrokerService().getAuthorizationService(); + spyAuthorizationService = spy(orignalAuthorizationService); + FieldUtils.writeField(getPulsarService().getBrokerService(), "authorizationService", + spyAuthorizationService, true); + } + + @AfterMethod(alwaysRun = true) + public void after() throws IllegalAccessException { + if (orignalAuthorizationService != null) { + FieldUtils.writeField(getPulsarService().getBrokerService(), "authorizationService", orignalAuthorizationService, true); + } + } + + @SneakyThrows + @AfterClass(alwaysRun = true) + public void cleanup() { + if (superUserAdmin != null) { + superUserAdmin.close(); + superUserAdmin = null; + } + spyAuthorizationService = null; + orignalAuthorizationService = null; + super.close(); + } + + @Test + public void testListTenants() throws PulsarAdminException { + superUserAdmin.tenants().getTenants(); + // test allow broker operation + verify(spyAuthorizationService) + .allowTenantOperationAsync(isNull(), Mockito.eq(TenantOperation.LIST_TENANTS), any(), any()); + // fallback to superuser + verify(spyAuthorizationService).isSuperUser(any(), any()); + + // ---- test nobody + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, () -> nobodyAdmin.tenants().getTenants()); + } + + + @Test + public void testGetTenant() throws PulsarAdminException { + String tenantName = "public"; + superUserAdmin.tenants().getTenantInfo(tenantName); + // test allow broker operation + verify(spyAuthorizationService) + .allowTenantOperationAsync(eq(tenantName), Mockito.eq(TenantOperation.GET_TENANT), any(), any()); + // fallback to superuser + verify(spyAuthorizationService).isSuperUser(any(), any()); + + // ---- test nobody + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, () -> nobodyAdmin.tenants().getTenantInfo(tenantName)); + } + + @Test + public void testUpdateTenant() throws PulsarAdminException { + String tenantName = "public"; + superUserAdmin.tenants().updateTenant(tenantName, TenantInfo.builder() + .allowedClusters(Set.of(getPulsarService().getConfiguration().getClusterName())) + .adminRoles(Set.of("example")).build()); + // test allow broker operation + verify(spyAuthorizationService) + .allowTenantOperationAsync(eq(tenantName), Mockito.eq(TenantOperation.UPDATE_TENANT), any(), any()); + // fallback to superuser + verify(spyAuthorizationService).isSuperUser(any(), any()); + + // ---- test nobody + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, () -> nobodyAdmin.tenants() + .updateTenant(tenantName, TenantInfo.builder().adminRoles(Set.of("example")).build())); + } + + @Test + public void testDeleteTenant() throws PulsarAdminException { + String tenantName = UUID.randomUUID().toString(); + superUserAdmin.tenants().createTenant(tenantName, TenantInfo.builder() + .allowedClusters(Set.of(getPulsarService().getConfiguration().getClusterName())) + .adminRoles(Set.of("example")).build()); + + Mockito.clearInvocations(spyAuthorizationService); + superUserAdmin.tenants().deleteTenant(tenantName); + // test allow broker operation + verify(spyAuthorizationService) + .allowTenantOperationAsync(eq(tenantName), Mockito.eq(TenantOperation.DELETE_TENANT), any(), any()); + // fallback to superuser + verify(spyAuthorizationService).isSuperUser(any(), any()); + + // ---- test nobody + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, () -> nobodyAdmin.tenants().deleteTenant(tenantName)); + } +} diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/TenantOperation.java b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/TenantOperation.java index 1c52f69006403..e0518e510f9dc 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/TenantOperation.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/TenantOperation.java @@ -25,4 +25,10 @@ public enum TenantOperation { CREATE_NAMESPACE, DELETE_NAMESPACE, LIST_NAMESPACES, + + LIST_TENANTS, + GET_TENANT, + CREATE_TENANT, + UPDATE_TENANT, + DELETE_TENANT, } From bf1f67742243f1b40c258c64e1fd6a960611a780 Mon Sep 17 00:00:00 2001 From: hanmz Date: Fri, 29 Nov 2024 22:56:03 +0800 Subject: [PATCH 13/22] [fix][client] Fix race-condition causing doReconsumeLater to hang when creating retryLetterProducer has failed (#23560) --- .../pulsar/client/api/RetryTopicTest.java | 68 +++++++++++++++++++ .../pulsar/client/impl/ConsumerImpl.java | 2 + 2 files changed, 70 insertions(+) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/RetryTopicTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/RetryTopicTest.java index 9cb82fde04118..cd598585c8e87 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/RetryTopicTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/RetryTopicTest.java @@ -30,6 +30,7 @@ import java.util.Map; import java.util.Set; import java.util.UUID; +import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; import lombok.Cleanup; import lombok.Data; @@ -45,6 +46,7 @@ import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; +import org.testng.collections.Lists; @Test(groups = "broker-api") public class RetryTopicTest extends ProducerConsumerBase { @@ -713,4 +715,70 @@ public void testRetryProducerWillCloseByConsumer() throws Exception { admin.topics().delete(topicDLQ, false); } + + @Test(timeOut = 30000L) + public void testRetryTopicExceptionWithConcurrent() throws Exception { + final String topic = "persistent://my-property/my-ns/retry-topic"; + final int maxRedeliveryCount = 2; + final int sendMessages = 10; + // subscribe before publish + Consumer consumer = pulsarClient.newConsumer(Schema.BYTES) + .topic(topic) + .subscriptionName("my-subscription") + .subscriptionType(SubscriptionType.Shared) + .enableRetry(true) + .receiverQueueSize(100) + .deadLetterPolicy(DeadLetterPolicy.builder() + .maxRedeliverCount(maxRedeliveryCount) + .retryLetterTopic("persistent://my-property/my-ns/my-subscription-custom-Retry") + .build()) + .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest) + .subscribe(); + + Producer producer = pulsarClient.newProducer(Schema.BYTES) + .topic(topic) + .create(); + for (int i = 0; i < sendMessages; i++) { + producer.newMessage().key("1").value(String.format("Hello Pulsar [%d]", i).getBytes()).send(); + } + producer.close(); + + // mock a retry producer exception when reconsumelater is called + MultiTopicsConsumerImpl multiTopicsConsumer = (MultiTopicsConsumerImpl) consumer; + List> consumers = multiTopicsConsumer.getConsumers(); + for (ConsumerImpl c : consumers) { + Set deadLetterPolicyField = + ReflectionUtils.getAllFields(c.getClass(), ReflectionUtils.withName("deadLetterPolicy")); + + if (deadLetterPolicyField.size() != 0) { + Field field = deadLetterPolicyField.iterator().next(); + field.setAccessible(true); + DeadLetterPolicy deadLetterPolicy = (DeadLetterPolicy) field.get(c); + deadLetterPolicy.setRetryLetterTopic("#persistent://invalid-topic#"); + } + } + + List> messages = Lists.newArrayList(); + for (int i = 0; i < sendMessages; i++) { + messages.add(consumer.receive()); + } + + // mock call the reconsumeLater method concurrently + CountDownLatch latch = new CountDownLatch(messages.size()); + for (Message message : messages) { + new Thread(() -> { + try { + consumer.reconsumeLater(message, 1, TimeUnit.SECONDS); + } catch (Exception ignore) { + + } finally { + latch.countDown(); + } + }).start(); + } + + latch.await(); + consumer.close(); + } + } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java index 390a70095182f..d2753856264fc 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java @@ -752,6 +752,8 @@ protected CompletableFuture doReconsumeLater(Message message, AckType a } catch (Exception e) { result.completeExceptionally(e); } + } else { + result.completeExceptionally(new PulsarClientException("Retry letter producer is null.")); } MessageId finalMessageId = messageId; result.exceptionally(ex -> { From 7fc88d650202bca5d0462b41a89c5189f90a3859 Mon Sep 17 00:00:00 2001 From: Yike Xiao Date: Fri, 29 Nov 2024 23:03:55 +0800 Subject: [PATCH 14/22] [improve][client] Enhance error handling for non-exist subscription in consumer creation (#23254) --- .../apache/pulsar/client/api/MultiTopicsConsumerTest.java | 6 ++++-- .../org/apache/pulsar/client/api/PulsarClientException.java | 2 ++ 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/MultiTopicsConsumerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/MultiTopicsConsumerTest.java index 7a12acd47edf9..ea8eb6e8cc081 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/MultiTopicsConsumerTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/MultiTopicsConsumerTest.java @@ -413,8 +413,9 @@ public void testSubscriptionNotFound() throws PulsarAdminException, PulsarClient .isAckReceiptEnabled(true) .subscribe(); assertTrue(singleTopicConsumer instanceof ConsumerImpl); + } catch (PulsarClientException.SubscriptionNotFoundException ignore) { } catch (Throwable t) { - assertTrue(t.getCause().getCause() instanceof PulsarClientException.SubscriptionNotFoundException); + fail("Should throw PulsarClientException.SubscriptionNotFoundException instead"); } try { @@ -424,8 +425,9 @@ public void testSubscriptionNotFound() throws PulsarAdminException, PulsarClient .isAckReceiptEnabled(true) .subscribe(); assertTrue(multiTopicsConsumer instanceof MultiTopicsConsumerImpl); + } catch (PulsarClientException.SubscriptionNotFoundException ignore) { } catch (Throwable t) { - assertTrue(t.getCause().getCause() instanceof PulsarClientException.SubscriptionNotFoundException); + fail("Should throw PulsarClientException.SubscriptionNotFoundException instead"); } pulsar.getConfiguration().setAllowAutoSubscriptionCreation(true); diff --git a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/PulsarClientException.java b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/PulsarClientException.java index 9eb6c612a52a2..b2c9b2b697b42 100644 --- a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/PulsarClientException.java +++ b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/PulsarClientException.java @@ -1113,6 +1113,8 @@ public static PulsarClientException unwrap(Throwable t) { newException = new TransactionConflictException(msg); } else if (cause instanceof TopicDoesNotExistException) { newException = new TopicDoesNotExistException(msg); + } else if (cause instanceof SubscriptionNotFoundException) { + newException = new SubscriptionNotFoundException(msg); } else if (cause instanceof ProducerFencedException) { newException = new ProducerFencedException(msg); } else if (cause instanceof MemoryBufferIsFullError) { From 46037229947c8031207eaed70dd937e6990de544 Mon Sep 17 00:00:00 2001 From: Qiang Zhao Date: Sat, 30 Nov 2024 01:17:16 +0800 Subject: [PATCH 15/22] [feat][broker] Implement allowBrokerOperationAsync in PulsarAuthorizationProvider to avoid exception thrown (#23663) --- .../broker/authorization/PulsarAuthorizationProvider.java | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/PulsarAuthorizationProvider.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/PulsarAuthorizationProvider.java index 0af63724cc812..50783c4d1338b 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/PulsarAuthorizationProvider.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/PulsarAuthorizationProvider.java @@ -40,6 +40,7 @@ import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.policies.data.AuthAction; import org.apache.pulsar.common.policies.data.AuthPolicies; +import org.apache.pulsar.common.policies.data.BrokerOperation; import org.apache.pulsar.common.policies.data.NamespaceOperation; import org.apache.pulsar.common.policies.data.PolicyName; import org.apache.pulsar.common.policies.data.PolicyOperation; @@ -690,6 +691,13 @@ public CompletableFuture allowTopicOperationAsync(TopicName topicName, }); } + @Override + public CompletableFuture allowBrokerOperationAsync(String clusterName, String brokerId, + BrokerOperation brokerOperation, String role, + AuthenticationDataSource authData) { + return isSuperUser(role, authData, conf); + } + @Override public CompletableFuture allowTopicPolicyOperationAsync(TopicName topicName, String role, PolicyName policyName, From f27905560207eb2ade32f8086b4585dffb918b80 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Mon, 2 Dec 2024 17:21:32 +0200 Subject: [PATCH 16/22] [improve] Install coreutils in docker image to improve compatibility (#23667) --- docker/pulsar/Dockerfile | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/docker/pulsar/Dockerfile b/docker/pulsar/Dockerfile index f8c22dc14a821..c8fc3a4d13c8f 100644 --- a/docker/pulsar/Dockerfile +++ b/docker/pulsar/Dockerfile @@ -94,7 +94,8 @@ RUN apk add --no-cache \ procps \ curl \ bind-tools \ - openssl + openssl \ + coreutils # Upgrade all packages to get latest versions with security fixes RUN apk upgrade --no-cache From 59d4588c1924df6e6a5e9eb9c525f6db0beeddf4 Mon Sep 17 00:00:00 2001 From: Hang Chen Date: Wed, 4 Dec 2024 10:01:44 +0800 Subject: [PATCH 17/22] [fix] [broker] Add consumer name for subscription stats (#23671) --- .../src/main/java/org/apache/pulsar/broker/service/Consumer.java | 1 + 1 file changed, 1 insertion(+) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Consumer.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Consumer.java index bcd29d86490cf..0baba13cc0b07 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Consumer.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Consumer.java @@ -977,6 +977,7 @@ public ConsumerStatsImpl getStats() { stats.unackedMessages = unackedMessages; stats.blockedConsumerOnUnackedMsgs = blockedConsumerOnUnackedMsgs; stats.avgMessagesPerEntry = getAvgMessagesPerEntry(); + stats.consumerName = consumerName; if (readPositionWhenJoining != null) { stats.readPositionWhenJoining = readPositionWhenJoining.toString(); } From 1c1a5cc655511c22b3399005d8ff9102b3553627 Mon Sep 17 00:00:00 2001 From: Qiang Zhao Date: Wed, 4 Dec 2024 10:16:21 +0800 Subject: [PATCH 18/22] [fix][broker] support missing cluster level fine-granted permissions (#23675) --- .../authorization/AuthorizationProvider.java | 20 + .../authorization/AuthorizationService.java | 45 ++ .../PulsarAuthorizationProvider.java | 15 + .../broker/admin/impl/ClustersBase.java | 160 ++++++- .../ClusterEndpointsAuthorizationTest.java | 428 ++++++++++++++++++ .../policies/data/ClusterOperation.java | 34 ++ .../common/policies/data/PolicyName.java | 6 +- 7 files changed, 689 insertions(+), 19 deletions(-) create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/ClusterEndpointsAuthorizationTest.java create mode 100644 pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/ClusterOperation.java diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/AuthorizationProvider.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/AuthorizationProvider.java index 48386265940a3..cb61292f8e38e 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/AuthorizationProvider.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/AuthorizationProvider.java @@ -33,6 +33,7 @@ import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.policies.data.AuthAction; import org.apache.pulsar.common.policies.data.BrokerOperation; +import org.apache.pulsar.common.policies.data.ClusterOperation; import org.apache.pulsar.common.policies.data.NamespaceOperation; import org.apache.pulsar.common.policies.data.PolicyName; import org.apache.pulsar.common.policies.data.PolicyOperation; @@ -393,4 +394,23 @@ default CompletableFuture allowBrokerOperationAsync(String clusterName, return FutureUtil.failedFuture( new UnsupportedOperationException("allowBrokerOperationAsync is not supported yet.")); } + + + default CompletableFuture allowClusterOperationAsync(String clusterName, + ClusterOperation clusterOperation, + String role, + AuthenticationDataSource authData) { + return FutureUtil.failedFuture( + new UnsupportedOperationException("allowClusterOperationAsync is not supported yet.")); + } + + default CompletableFuture allowClusterPolicyOperationAsync(String clusterName, + String role, + PolicyName policy, + PolicyOperation operation, + AuthenticationDataSource authData) { + return FutureUtil.failedFuture( + new IllegalStateException("ClusterPolicyOperation [" + policy.name() + "/" + operation.name() + "] " + + "is not supported by the Authorization provider you are using.")); + } } diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/AuthorizationService.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/AuthorizationService.java index 1348a405b0dfa..40573d99d6012 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/AuthorizationService.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/AuthorizationService.java @@ -39,6 +39,7 @@ import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.policies.data.AuthAction; import org.apache.pulsar.common.policies.data.BrokerOperation; +import org.apache.pulsar.common.policies.data.ClusterOperation; import org.apache.pulsar.common.policies.data.NamespaceOperation; import org.apache.pulsar.common.policies.data.PolicyName; import org.apache.pulsar.common.policies.data.PolicyOperation; @@ -567,6 +568,50 @@ public CompletableFuture allowBrokerOperationAsync(String clusterName, } } + public CompletableFuture allowClusterOperationAsync(String clusterName, + ClusterOperation clusterOperation, + String originalRole, + String role, + AuthenticationDataSource authData) { + if (!isValidOriginalPrincipal(role, originalRole, authData)) { + return CompletableFuture.completedFuture(false); + } + + if (isProxyRole(role)) { + final var isRoleAuthorizedFuture = provider.allowClusterOperationAsync(clusterName, + clusterOperation, role, authData); + final var isOriginalAuthorizedFuture = provider.allowClusterOperationAsync(clusterName, + clusterOperation, originalRole, authData); + return isRoleAuthorizedFuture.thenCombine(isOriginalAuthorizedFuture, + (isRoleAuthorized, isOriginalAuthorized) -> isRoleAuthorized && isOriginalAuthorized); + } else { + return provider.allowClusterOperationAsync(clusterName, clusterOperation, role, authData); + } + } + + public CompletableFuture allowClusterPolicyOperationAsync(String clusterName, + PolicyName policy, + PolicyOperation operation, + String originalRole, + String role, + AuthenticationDataSource authData) { + if (!isValidOriginalPrincipal(role, originalRole, authData)) { + return CompletableFuture.completedFuture(false); + } + + if (isProxyRole(role)) { + final var isRoleAuthorizedFuture = provider.allowClusterPolicyOperationAsync(clusterName, role, + policy, operation, authData); + final var isOriginalAuthorizedFuture = provider.allowClusterPolicyOperationAsync(clusterName, originalRole, + policy, operation, authData); + return isRoleAuthorizedFuture.thenCombine(isOriginalAuthorizedFuture, + (isRoleAuthorized, isOriginalAuthorized) -> isRoleAuthorized && isOriginalAuthorized); + } else { + return provider.allowClusterPolicyOperationAsync(clusterName, role, policy, operation, authData); + } + } + + /** * @deprecated - will be removed after 2.12. Use async variant. */ diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/PulsarAuthorizationProvider.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/PulsarAuthorizationProvider.java index 50783c4d1338b..976e7b7ee12e7 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/PulsarAuthorizationProvider.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/PulsarAuthorizationProvider.java @@ -41,6 +41,7 @@ import org.apache.pulsar.common.policies.data.AuthAction; import org.apache.pulsar.common.policies.data.AuthPolicies; import org.apache.pulsar.common.policies.data.BrokerOperation; +import org.apache.pulsar.common.policies.data.ClusterOperation; import org.apache.pulsar.common.policies.data.NamespaceOperation; import org.apache.pulsar.common.policies.data.PolicyName; import org.apache.pulsar.common.policies.data.PolicyOperation; @@ -860,4 +861,18 @@ public CompletableFuture>> getPermissionsAsync(Names }); }); } + + @Override + public CompletableFuture allowClusterOperationAsync(String clusterName, ClusterOperation clusterOperation, + String role, AuthenticationDataSource authData) { + return isSuperUser(role, authData, conf); + } + + @Override + public CompletableFuture allowClusterPolicyOperationAsync(String clusterName, String role, + PolicyName policy, + PolicyOperation operation, + AuthenticationDataSource authData) { + return isSuperUser(role, authData, conf); + } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/ClustersBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/ClustersBase.java index b261033ca52c9..d24a3255b5556 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/ClustersBase.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/ClustersBase.java @@ -63,11 +63,14 @@ import org.apache.pulsar.common.policies.data.BrokerNamespaceIsolationDataImpl; import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.ClusterDataImpl; +import org.apache.pulsar.common.policies.data.ClusterOperation; import org.apache.pulsar.common.policies.data.ClusterPolicies.ClusterUrl; import org.apache.pulsar.common.policies.data.ClusterPoliciesImpl; import org.apache.pulsar.common.policies.data.FailureDomainImpl; import org.apache.pulsar.common.policies.data.NamespaceIsolationDataImpl; import org.apache.pulsar.common.policies.data.NamespaceIsolationPolicyUnloadScope; +import org.apache.pulsar.common.policies.data.PolicyName; +import org.apache.pulsar.common.policies.data.PolicyOperation; import org.apache.pulsar.common.policies.impl.NamespaceIsolationPolicies; import org.apache.pulsar.common.policies.impl.NamespaceIsolationPolicyImpl; import org.apache.pulsar.common.util.FutureUtil; @@ -117,7 +120,7 @@ public void getClusters(@Suspended AsyncResponse asyncResponse) { public void getCluster(@Suspended AsyncResponse asyncResponse, @ApiParam(value = "The cluster name", required = true) @PathParam("cluster") String cluster) { - validateSuperUserAccessAsync() + validateBothSuperuserAndClusterOperation(cluster, ClusterOperation.GET_CLUSTER) .thenCompose(__ -> clusterResources().getClusterAsync(cluster)) .thenAccept(clusterData -> { asyncResponse.resume(clusterData @@ -162,7 +165,7 @@ public void createCluster( ) ) ) ClusterDataImpl clusterData) { - validateSuperUserAccessAsync() + validateBothSuperuserAndClusterOperation(cluster, ClusterOperation.CREATE_CLUSTER) .thenCompose(__ -> validatePoliciesReadOnlyAccessAsync()) .thenCompose(__ -> { NamedEntity.checkName(cluster); @@ -227,7 +230,7 @@ public void updateCluster( ) ) ) ClusterDataImpl clusterData) { - validateSuperUserAccessAsync() + validateBothSuperuserAndClusterOperation(cluster, ClusterOperation.UPDATE_CLUSTER) .thenCompose(__ -> validatePoliciesReadOnlyAccessAsync()) .thenCompose(__ -> { try { @@ -271,7 +274,7 @@ public void getClusterMigration( required = true ) @PathParam("cluster") String cluster) { - validateSuperUserAccessAsync() + validateBothSuperuserAndClusterPolicyOperation(cluster, PolicyName.CLUSTER_MIGRATION, PolicyOperation.READ) .thenCompose(__ -> clusterResources().getClusterPoliciesResources().getClusterPoliciesAsync(cluster)) .thenAccept(policies -> { asyncResponse.resume( @@ -326,7 +329,7 @@ public void updateClusterMigration( asyncResponse.resume(new RestException(Status.BAD_REQUEST, "Cluster url must not be empty")); return; } - validateSuperUserAccessAsync() + validateBothSuperuserAndClusterPolicyOperation(cluster, PolicyName.CLUSTER_MIGRATION, PolicyOperation.WRITE) .thenCompose(__ -> validatePoliciesReadOnlyAccessAsync()) .thenCompose(__ -> clusterResources().getClusterPoliciesResources().setPoliciesWithCreateAsync(cluster, old -> { @@ -376,7 +379,7 @@ public void setPeerClusterNames(@Suspended AsyncResponse asyncResponse, "cluster-b" ]"""))) LinkedHashSet peerClusterNames) { - validateSuperUserAccessAsync() + validateBothSuperuserAndClusterOperation(cluster, ClusterOperation.UPDATE_PEER_CLUSTER) .thenCompose(__ -> validatePoliciesReadOnlyAccessAsync()) .thenCompose(__ -> innerSetPeerClusterNamesAsync(cluster, peerClusterNames)) .thenAccept(__ -> { @@ -437,7 +440,7 @@ private CompletableFuture innerSetPeerClusterNamesAsync(String cluster, public void getPeerCluster(@Suspended AsyncResponse asyncResponse, @ApiParam(value = "The cluster name", required = true) @PathParam("cluster") String cluster) { - validateSuperUserAccessAsync() + validateBothSuperuserAndClusterOperation(cluster, ClusterOperation.GET_PEER_CLUSTER) .thenCompose(__ -> clusterResources().getClusterAsync(cluster)) .thenAccept(clusterOpt -> { ClusterData clusterData = @@ -466,7 +469,7 @@ public void getPeerCluster(@Suspended AsyncResponse asyncResponse, public void deleteCluster(@Suspended AsyncResponse asyncResponse, @ApiParam(value = "The cluster name", required = true) @PathParam("cluster") String cluster) { - validateSuperUserAccessAsync() + validateBothSuperuserAndClusterOperation(cluster, ClusterOperation.DELETE_CLUSTER) .thenCompose(__ -> validatePoliciesReadOnlyAccessAsync()) .thenCompose(__ -> internalDeleteClusterAsync(cluster)) .thenAccept(__ -> { @@ -525,7 +528,7 @@ public void getNamespaceIsolationPolicies( @Suspended AsyncResponse asyncResponse, @ApiParam(value = "The cluster name", required = true) @PathParam("cluster") String cluster ) { - validateSuperUserAccessAsync() + validateBothSuperuserAndClusterPolicyOperation(cluster, PolicyName.NAMESPACE_ISOLATION, PolicyOperation.READ) .thenCompose(__ -> validateClusterExistAsync(cluster, Status.NOT_FOUND)) .thenCompose(__ -> internalGetNamespaceIsolationPolicies(cluster)) .thenAccept(asyncResponse::resume) @@ -583,7 +586,7 @@ public void getNamespaceIsolationPolicy( @ApiParam(value = "The name of the namespace isolation policy", required = true) @PathParam("policyName") String policyName ) { - validateSuperUserAccessAsync() + validateBothSuperuserAndClusterPolicyOperation(cluster, PolicyName.NAMESPACE_ISOLATION, PolicyOperation.READ) .thenCompose(__ -> validateClusterExistAsync(cluster, Status.PRECONDITION_FAILED)) .thenCompose(__ -> internalGetNamespaceIsolationPolicies(cluster)) .thenAccept(policies -> { @@ -619,7 +622,7 @@ public void getBrokersWithNamespaceIsolationPolicy( @Suspended AsyncResponse asyncResponse, @ApiParam(value = "The cluster name", required = true) @PathParam("cluster") String cluster) { - validateSuperUserAccessAsync() + validateBothSuperuserAndClusterPolicyOperation(cluster, PolicyName.NAMESPACE_ISOLATION, PolicyOperation.READ) .thenCompose(__ -> validateClusterExistAsync(cluster, Status.PRECONDITION_FAILED)) .thenCompose(__ -> pulsar().getLoadManager().get().getAvailableBrokersAsync()) .thenCompose(availableBrokers -> internalGetNamespaceIsolationPolicies(cluster) @@ -676,7 +679,7 @@ public void getBrokerWithNamespaceIsolationPolicy( @ApiParam(value = "The broker name (:)", required = true, example = "broker1:8080") @PathParam("broker") String broker) { - validateSuperUserAccessAsync() + validateBothSuperuserAndClusterPolicyOperation(cluster, PolicyName.NAMESPACE_ISOLATION, PolicyOperation.READ) .thenCompose(__ -> validateClusterExistAsync(cluster, PRECONDITION_FAILED)) .thenCompose(__ -> internalGetNamespaceIsolationPolicies(cluster)) .thenApply(policies -> internalGetBrokerNsIsolationData(broker, policies)) @@ -711,7 +714,7 @@ public void setNamespaceIsolationPolicy( @ApiParam(value = "The namespace isolation policy data", required = true) NamespaceIsolationDataImpl policyData ) { - validateSuperUserAccessAsync() + validateBothSuperuserAndClusterPolicyOperation(cluster, PolicyName.NAMESPACE_ISOLATION, PolicyOperation.WRITE) .thenCompose(__ -> validatePoliciesReadOnlyAccessAsync()) .thenCompose(__ -> validateClusterExistAsync(cluster, PRECONDITION_FAILED)) .thenCompose(__ -> { @@ -874,7 +877,7 @@ public void deleteNamespaceIsolationPolicy( @ApiParam(value = "The namespace isolation policy name", required = true) @PathParam("policyName") String policyName ) { - validateSuperUserAccessAsync() + validateBothSuperuserAndClusterPolicyOperation(cluster, PolicyName.NAMESPACE_ISOLATION, PolicyOperation.WRITE) .thenCompose(__ -> validateClusterExistAsync(cluster, PRECONDITION_FAILED)) .thenCompose(__ -> validatePoliciesReadOnlyAccessAsync()) .thenCompose(__ -> namespaceIsolationPolicies().getIsolationDataPoliciesAsync(cluster)) @@ -924,7 +927,7 @@ public void setFailureDomain( @PathParam("domainName") String domainName, @ApiParam(value = "The configuration data of a failure domain", required = true) FailureDomainImpl domain ) { - validateSuperUserAccessAsync() + validateBothSuperuserAndClusterOperation(cluster, ClusterOperation.UPDATE_FAILURE_DOMAIN) .thenCompose(__ -> validateClusterExistAsync(cluster, PRECONDITION_FAILED)) .thenCompose(__ -> validateBrokerExistsInOtherDomain(cluster, domainName, domain)) .thenCompose(__ -> clusterResources().getFailureDomainResources() @@ -967,7 +970,7 @@ public void getFailureDomains( @ApiParam(value = "The cluster name", required = true) @PathParam("cluster") String cluster ) { - validateSuperUserAccessAsync() + validateBothSuperuserAndClusterOperation(cluster, ClusterOperation.GET_FAILURE_DOMAIN) .thenCompose(__ -> clusterResources().getFailureDomainResources() .listFailureDomainsAsync(cluster) .thenCompose(domainNames -> { @@ -1023,7 +1026,7 @@ public void getDomain( @ApiParam(value = "The failure domain name", required = true) @PathParam("domainName") String domainName ) { - validateSuperUserAccessAsync() + validateBothSuperuserAndClusterOperation(cluster, ClusterOperation.GET_FAILURE_DOMAIN) .thenCompose(__ -> validateClusterExistAsync(cluster, PRECONDITION_FAILED)) .thenCompose(__ -> clusterResources().getFailureDomainResources() .getFailureDomainAsync(cluster, domainName)) @@ -1058,7 +1061,7 @@ public void deleteFailureDomain( @ApiParam(value = "The failure domain name", required = true) @PathParam("domainName") String domainName ) { - validateSuperUserAccessAsync() + validateBothSuperuserAndClusterOperation(cluster, ClusterOperation.DELETE_FAILURE_DOMAIN) .thenCompose(__ -> validateClusterExistAsync(cluster, PRECONDITION_FAILED)) .thenCompose(__ -> clusterResources() .getFailureDomainResources().deleteFailureDomainAsync(cluster, domainName)) @@ -1124,5 +1127,126 @@ private CompletableFuture validateBrokerExistsInOtherDomain(final String c }); } + + + private CompletableFuture validateBothSuperuserAndClusterOperation(String clusterName, + ClusterOperation operation) { + final var superUserAccessValidation = validateSuperUserAccessAsync(); + final var clusterOperationValidation = validateClusterOperationAsync(clusterName, operation); + return FutureUtil.waitForAll(List.of(superUserAccessValidation, clusterOperationValidation)) + .handle((result, err) -> { + if (!superUserAccessValidation.isCompletedExceptionally() + || !clusterOperationValidation.isCompletedExceptionally()) { + return null; + } + if (log.isDebugEnabled()) { + Throwable superUserValidationException = null; + try { + superUserAccessValidation.join(); + } catch (Throwable ex) { + superUserValidationException = FutureUtil.unwrapCompletionException(ex); + } + Throwable clusterOperationValidationException = null; + try { + clusterOperationValidation.join(); + } catch (Throwable ex) { + clusterOperationValidationException = FutureUtil.unwrapCompletionException(ex); + } + log.debug("validateBothSuperuserAndClusterOperation failed." + + " originalPrincipal={} clientAppId={} operation={} cluster={} " + + "superuserValidationError={} clusterOperationValidationError={}", + originalPrincipal(), clientAppId(), operation.toString(), clusterName, + superUserValidationException, clusterOperationValidationException); + } + throw new RestException(Status.UNAUTHORIZED, + String.format("Unauthorized to validateBothSuperuserAndClusterOperation for" + + " originalPrincipal [%s] and clientAppId [%s] " + + "about operation [%s] on cluster [%s]", + originalPrincipal(), clientAppId(), operation.toString(), clusterName)); + }); + } + + + private CompletableFuture validateBothSuperuserAndClusterPolicyOperation(String clusterName, PolicyName name, + PolicyOperation operation) { + final var superUserAccessValidation = validateSuperUserAccessAsync(); + final var clusterOperationValidation = validateClusterPolicyOperationAsync(clusterName, name, operation); + return FutureUtil.waitForAll(List.of(superUserAccessValidation, clusterOperationValidation)) + .handle((result, err) -> { + if (!superUserAccessValidation.isCompletedExceptionally() + || !clusterOperationValidation.isCompletedExceptionally()) { + return null; + } + if (log.isDebugEnabled()) { + Throwable superUserValidationException = null; + try { + superUserAccessValidation.join(); + } catch (Throwable ex) { + superUserValidationException = FutureUtil.unwrapCompletionException(ex); + } + Throwable clusterOperationValidationException = null; + try { + clusterOperationValidation.join(); + } catch (Throwable ex) { + clusterOperationValidationException = FutureUtil.unwrapCompletionException(ex); + } + log.debug("validateBothSuperuserAndClusterPolicyOperation failed." + + " originalPrincipal={} clientAppId={} operation={} cluster={} " + + "superuserValidationError={} clusterOperationValidationError={}", + originalPrincipal(), clientAppId(), operation.toString(), clusterName, + superUserValidationException, clusterOperationValidationException); + } + throw new RestException(Status.UNAUTHORIZED, + String.format("Unauthorized to validateBothSuperuserAndClusterPolicyOperation for" + + " originalPrincipal [%s] and clientAppId [%s] " + + "about operation [%s] on cluster [%s]", + originalPrincipal(), clientAppId(), operation.toString(), clusterName)); + }); + } + + + + + private CompletableFuture validateClusterOperationAsync(String cluster, ClusterOperation operation) { + final var pulsar = pulsar(); + if (pulsar.getBrokerService().isAuthenticationEnabled() + && pulsar.getBrokerService().isAuthorizationEnabled()) { + return pulsar.getBrokerService().getAuthorizationService() + .allowClusterOperationAsync(cluster, operation, originalPrincipal(), + clientAppId(), clientAuthData()) + .thenAccept(isAuthorized -> { + if (!isAuthorized) { + throw new RestException(Status.UNAUTHORIZED, + String.format("Unauthorized to validateClusterOperation for" + + " originalPrincipal [%s] and clientAppId [%s] " + + "about operation [%s] on cluster [%s]", + originalPrincipal(), clientAppId(), operation.toString(), cluster)); + } + }); + } + return CompletableFuture.completedFuture(null); + } + + private CompletableFuture validateClusterPolicyOperationAsync(String cluster, PolicyName policyName, + PolicyOperation operation) { + final var pulsar = pulsar(); + if (pulsar.getBrokerService().isAuthenticationEnabled() + && pulsar.getBrokerService().isAuthorizationEnabled()) { + return pulsar.getBrokerService().getAuthorizationService() + .allowClusterPolicyOperationAsync(cluster, policyName, operation, originalPrincipal(), + clientAppId(), clientAuthData()) + .thenAccept(isAuthorized -> { + if (!isAuthorized) { + throw new RestException(Status.UNAUTHORIZED, + String.format("Unauthorized to validateClusterPolicyOperation for" + + " originalPrincipal [%s] and clientAppId [%s] " + + "about operation [%s] on cluster [%s]", + originalPrincipal(), clientAppId(), operation.toString(), cluster)); + } + }); + } + return CompletableFuture.completedFuture(null); + } + private static final Logger log = LoggerFactory.getLogger(ClustersBase.class); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/ClusterEndpointsAuthorizationTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/ClusterEndpointsAuthorizationTest.java new file mode 100644 index 0000000000000..ccf5ccb9481ec --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/ClusterEndpointsAuthorizationTest.java @@ -0,0 +1,428 @@ +/* + * 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.admin; + +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.verify; +import lombok.SneakyThrows; +import org.apache.commons.lang3.reflect.FieldUtils; +import org.apache.pulsar.broker.authorization.AuthorizationService; +import org.apache.pulsar.client.admin.PulsarAdmin; +import org.apache.pulsar.client.admin.PulsarAdminException; +import org.apache.pulsar.client.impl.auth.AuthenticationToken; +import org.apache.pulsar.common.policies.data.ClusterData; +import org.apache.pulsar.common.policies.data.ClusterOperation; +import org.apache.pulsar.common.policies.data.ClusterPolicies; +import org.apache.pulsar.common.policies.data.FailureDomain; +import org.apache.pulsar.common.policies.data.NamespaceIsolationData; +import org.apache.pulsar.common.policies.data.PolicyName; +import org.apache.pulsar.common.policies.data.PolicyOperation; +import org.apache.pulsar.security.MockedPulsarStandalone; +import org.mockito.Mockito; +import org.testng.Assert; +import org.testng.annotations.AfterClass; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; +import java.util.LinkedHashSet; +import java.util.UUID; + + +@Test(groups = "broker-admin") +public class ClusterEndpointsAuthorizationTest extends MockedPulsarStandalone { + + private AuthorizationService orignalAuthorizationService; + private AuthorizationService spyAuthorizationService; + + private PulsarAdmin superUserAdmin; + private PulsarAdmin nobodyAdmin; + + @SneakyThrows + @BeforeClass(alwaysRun = true) + public void setup() { + configureTokenAuthentication(); + configureDefaultAuthorization(); + start(); + this.superUserAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(SUPER_USER_TOKEN)) + .build(); + this.nobodyAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(NOBODY_TOKEN)) + .build(); + } + + @BeforeMethod(alwaysRun = true) + public void before() throws IllegalAccessException { + orignalAuthorizationService = getPulsarService().getBrokerService().getAuthorizationService(); + spyAuthorizationService = spy(orignalAuthorizationService); + FieldUtils.writeField(getPulsarService().getBrokerService(), "authorizationService", + spyAuthorizationService, true); + } + + @AfterMethod(alwaysRun = true) + public void after() throws IllegalAccessException { + if (orignalAuthorizationService != null) { + FieldUtils.writeField(getPulsarService().getBrokerService(), "authorizationService", orignalAuthorizationService, true); + } + } + + @SneakyThrows + @AfterClass(alwaysRun = true) + public void cleanup() { + if (superUserAdmin != null) { + superUserAdmin.close(); + superUserAdmin = null; + } + spyAuthorizationService = null; + orignalAuthorizationService = null; + super.close(); + } + + + @Test + public void testGetCluster() throws PulsarAdminException { + final String clusterName = getPulsarService().getConfiguration().getClusterName(); + superUserAdmin.clusters().getCluster(clusterName); + // test allow cluster operation + verify(spyAuthorizationService) + .allowClusterOperationAsync(eq(clusterName), eq(ClusterOperation.GET_CLUSTER), any(), any(), any()); + // fallback to superuser + verify(spyAuthorizationService).isSuperUser(any(), any()); + + // ---- test nobody + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> nobodyAdmin.clusters().getCluster(clusterName)); + } + + + @Test + public void testCreateCluster() throws PulsarAdminException { + final String clusterName = UUID.randomUUID().toString(); + superUserAdmin.clusters().createCluster(clusterName, ClusterData.builder().build()); + // test allow cluster operation + verify(spyAuthorizationService) + .allowClusterOperationAsync(eq(clusterName), eq(ClusterOperation.CREATE_CLUSTER), any(), any(), any()); + // fallback to superuser + verify(spyAuthorizationService).isSuperUser(any(), any()); + + // ---- test nobody + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> nobodyAdmin.clusters().createCluster(clusterName, ClusterData.builder().build())); + } + + @Test + public void testUpdateCluster() { + final String clusterName = UUID.randomUUID().toString(); + try { + superUserAdmin.clusters().updateCluster(clusterName, ClusterData.builder().serviceUrl("aaa").build()); + } catch (Throwable ignore) { + + } + // test allow cluster operation + verify(spyAuthorizationService) + .allowClusterOperationAsync(eq(clusterName), eq(ClusterOperation.UPDATE_CLUSTER), any(), any(), any()); + // fallback to superuser + verify(spyAuthorizationService).isSuperUser(any(), any()); + // ---- test nobody + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> nobodyAdmin.clusters().updateCluster(clusterName, ClusterData.builder().build())); + } + + + @Test + public void testGetClusterMigration() { + final String clusterName = UUID.randomUUID().toString(); + try { + superUserAdmin.clusters().getClusterMigration(clusterName); + } catch (Throwable ignore) { + + } + // test allow cluster operation + verify(spyAuthorizationService) + .allowClusterPolicyOperationAsync(eq(clusterName), eq(PolicyName.CLUSTER_MIGRATION), + eq(PolicyOperation.READ), any(), any(), any()); + // fallback to superuser + verify(spyAuthorizationService).isSuperUser(any(), any()); + // ---- test nobody + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> nobodyAdmin.clusters().getClusterMigration(clusterName)); + } + + + @Test + public void testUpdateClusterMigration() throws PulsarAdminException { + final String clusterName = UUID.randomUUID().toString(); + superUserAdmin.clusters().createCluster(clusterName, ClusterData.builder().build()); + Mockito.clearInvocations(spyAuthorizationService); + + superUserAdmin.clusters().updateClusterMigration(clusterName, false, new ClusterPolicies.ClusterUrl()); + // test allow cluster operation + verify(spyAuthorizationService) + .allowClusterPolicyOperationAsync(eq(clusterName), eq(PolicyName.CLUSTER_MIGRATION), + eq(PolicyOperation.WRITE), any(), any(), any()); + // fallback to superuser + verify(spyAuthorizationService).isSuperUser(any(), any()); + // ---- test nobody + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> nobodyAdmin.clusters() + .updateClusterMigration(clusterName, false, new ClusterPolicies.ClusterUrl())); + } + + @Test + public void testSetPeerClusterNames() throws PulsarAdminException { + final LinkedHashSet linkedHashSet = new LinkedHashSet<>(); + linkedHashSet.add("a"); + final String clusterName = getPulsarService().getConfiguration().getClusterName(); + try { + superUserAdmin.clusters().updatePeerClusterNames(clusterName, linkedHashSet); + } catch (Throwable ignore) { + + } + // test allow cluster operation + verify(spyAuthorizationService) + .allowClusterOperationAsync(eq(clusterName), + eq(ClusterOperation.UPDATE_PEER_CLUSTER), any(), any(), any()); + // fallback to superuser + verify(spyAuthorizationService).isSuperUser(any(), any()); + // ---- test nobody + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> nobodyAdmin.clusters().updatePeerClusterNames(clusterName, linkedHashSet)); + } + + @Test + public void testGetPeerCluster() throws PulsarAdminException { + final String clusterName = getPulsarService().getConfiguration().getClusterName(); + superUserAdmin.clusters().getPeerClusterNames(clusterName); + + // test allow cluster operation + verify(spyAuthorizationService) + .allowClusterOperationAsync(eq(clusterName), + eq(ClusterOperation.GET_PEER_CLUSTER), any(), any(), any()); + // fallback to superuser + verify(spyAuthorizationService).isSuperUser(any(), any()); + // ---- test nobody + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> nobodyAdmin.clusters().getPeerClusterNames(clusterName)); + } + + @Test + public void testDeleteCluster() throws PulsarAdminException { + final String clusterName = UUID.randomUUID().toString(); + superUserAdmin.clusters().createCluster(clusterName, ClusterData.builder().build()); + Mockito.clearInvocations(spyAuthorizationService); + + superUserAdmin.clusters().deleteCluster(clusterName); + + // test allow cluster operation + verify(spyAuthorizationService) + .allowClusterOperationAsync(eq(clusterName), + eq(ClusterOperation.DELETE_CLUSTER), any(), any(), any()); + // fallback to superuser + verify(spyAuthorizationService).isSuperUser(any(), any()); + // ---- test nobody + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> nobodyAdmin.clusters().deleteCluster(clusterName)); + } + + + @Test + public void testGetNamespaceIsolationPolicies() throws PulsarAdminException { + final String clusterName = getPulsarService().getConfiguration().getClusterName(); + superUserAdmin.clusters().getNamespaceIsolationPolicies(clusterName); + + // test allow cluster operation + verify(spyAuthorizationService) + .allowClusterPolicyOperationAsync(eq(clusterName), + eq(PolicyName.NAMESPACE_ISOLATION), eq(PolicyOperation.READ), any(), any(), any()); + // fallback to superuser + verify(spyAuthorizationService).isSuperUser(any(), any()); + // ---- test nobody + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> nobodyAdmin.clusters().getNamespaceIsolationPolicies(clusterName)); + } + + + @Test + public void testGetNamespaceIsolationPolicy() throws PulsarAdminException { + final String clusterName = getPulsarService().getConfiguration().getClusterName(); + superUserAdmin.clusters().getNamespaceIsolationPolicy(clusterName, ""); + + // test allow cluster operation + verify(spyAuthorizationService) + .allowClusterPolicyOperationAsync(eq(clusterName), + eq(PolicyName.NAMESPACE_ISOLATION), eq(PolicyOperation.READ), any(), any(), any()); + // fallback to superuser + verify(spyAuthorizationService).isSuperUser(any(), any()); + // ---- test nobody + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> nobodyAdmin.clusters().getNamespaceIsolationPolicy(clusterName, "")); + } + + + @Test + public void testGetBrokersWithNamespaceIsolationPolicy() throws PulsarAdminException { + final String clusterName = getPulsarService().getConfiguration().getClusterName(); + superUserAdmin.clusters().getBrokersWithNamespaceIsolationPolicy(clusterName); + + // test allow cluster operation + verify(spyAuthorizationService) + .allowClusterPolicyOperationAsync(eq(clusterName), + eq(PolicyName.NAMESPACE_ISOLATION), eq(PolicyOperation.READ), any(), any(), any()); + // fallback to superuser + verify(spyAuthorizationService).isSuperUser(any(), any()); + // ---- test nobody + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> nobodyAdmin.clusters().getBrokersWithNamespaceIsolationPolicy(clusterName)); + } + + + @Test + public void testGetBrokerWithNamespaceIsolationPolicy() throws PulsarAdminException { + final String clusterName = getPulsarService().getConfiguration().getClusterName(); + superUserAdmin.clusters().getBrokerWithNamespaceIsolationPolicy(clusterName, getPulsarService().getBrokerId()); + + // test allow cluster operation + verify(spyAuthorizationService) + .allowClusterPolicyOperationAsync(eq(clusterName), + eq(PolicyName.NAMESPACE_ISOLATION), eq(PolicyOperation.READ), any(), any(), any()); + // fallback to superuser + verify(spyAuthorizationService).isSuperUser(any(), any()); + // ---- test nobody + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> nobodyAdmin.clusters().getBrokerWithNamespaceIsolationPolicy(clusterName, "")); + } + + + @Test + public void testSetNamespaceIsolationPolicy() { + final String clusterName = getPulsarService().getConfiguration().getClusterName(); + + try { + superUserAdmin.clusters().updateNamespaceIsolationPolicy(clusterName, "test", + NamespaceIsolationData.builder().build()); + } catch (Throwable ignore) { + + } + + // test allow cluster operation + verify(spyAuthorizationService) + .allowClusterPolicyOperationAsync(eq(clusterName), + eq(PolicyName.NAMESPACE_ISOLATION), eq(PolicyOperation.WRITE), any(), any(), any()); + // fallback to superuser + verify(spyAuthorizationService).isSuperUser(any(), any()); + // ---- test nobody + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> nobodyAdmin.clusters().updateNamespaceIsolationPolicy(clusterName, "test", + NamespaceIsolationData.builder().build())); + } + + @Test + public void testDeleteNamespaceIsolationPolicy() throws PulsarAdminException { + final String clusterName = getPulsarService().getConfiguration().getClusterName(); + superUserAdmin.clusters().deleteNamespaceIsolationPolicy(clusterName, "test"); + + // test allow cluster operation + verify(spyAuthorizationService) + .allowClusterPolicyOperationAsync(eq(clusterName), + eq(PolicyName.NAMESPACE_ISOLATION), eq(PolicyOperation.WRITE), any(), any(), any()); + // fallback to superuser + verify(spyAuthorizationService).isSuperUser(any(), any()); + // ---- test nobody + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> nobodyAdmin.clusters().deleteNamespaceIsolationPolicy(clusterName, "test")); + } + + + @Test + public void testSetFailureDomain() throws PulsarAdminException { + final String clusterName = getPulsarService().getConfiguration().getClusterName(); + superUserAdmin.clusters().updateFailureDomain(clusterName, "test", FailureDomain.builder().build()); + + // test allow cluster operation + verify(spyAuthorizationService) + .allowClusterOperationAsync(eq(clusterName), eq(ClusterOperation.UPDATE_FAILURE_DOMAIN), any(), any(), any()); + // fallback to superuser + verify(spyAuthorizationService).isSuperUser(any(), any()); + // ---- test nobody + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> nobodyAdmin.clusters().updateFailureDomain(clusterName, + "test", FailureDomain.builder().build())); + } + + @Test + public void testGetFailureDomains() throws PulsarAdminException { + final String clusterName = getPulsarService().getConfiguration().getClusterName(); + superUserAdmin.clusters().getFailureDomains(clusterName); + + // test allow cluster operation + verify(spyAuthorizationService) + .allowClusterOperationAsync(eq(clusterName), eq(ClusterOperation.GET_FAILURE_DOMAIN), any(), any(), any()); + // fallback to superuser + verify(spyAuthorizationService).isSuperUser(any(), any()); + // ---- test nobody + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> nobodyAdmin.clusters().getFailureDomains(clusterName)); + } + + + @Test + public void testGetDomain() throws PulsarAdminException { + final String clusterName = getPulsarService().getConfiguration().getClusterName(); + try { + superUserAdmin.clusters().getFailureDomain(clusterName, "test"); + } catch (Throwable ignore) { + + } + + // test allow cluster operation + verify(spyAuthorizationService) + .allowClusterOperationAsync(eq(clusterName), eq(ClusterOperation.GET_FAILURE_DOMAIN), any(), any(), any()); + // fallback to superuser + verify(spyAuthorizationService).isSuperUser(any(), any()); + // ---- test nobody + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> nobodyAdmin.clusters().getFailureDomain(clusterName, "test")); + } + + @Test + public void testDeleteFailureDomain() throws PulsarAdminException { + final String clusterName = getPulsarService().getConfiguration().getClusterName(); + try { + superUserAdmin.clusters().deleteFailureDomain(clusterName, "test"); + } catch (Throwable ignore) { + + } + + // test allow cluster operation + verify(spyAuthorizationService) + .allowClusterOperationAsync(eq(clusterName), eq(ClusterOperation.DELETE_FAILURE_DOMAIN), any(), any(), any()); + // fallback to superuser + verify(spyAuthorizationService).isSuperUser(any(), any()); + // ---- test nobody + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> nobodyAdmin.clusters().deleteFailureDomain(clusterName, "test")); + } + + +} diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/ClusterOperation.java b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/ClusterOperation.java new file mode 100644 index 0000000000000..bbdc64f729ef4 --- /dev/null +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/ClusterOperation.java @@ -0,0 +1,34 @@ +/* + * 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.common.policies.data; + +public enum ClusterOperation { + LIST_CLUSTERS, + GET_CLUSTER, + CREATE_CLUSTER, + UPDATE_CLUSTER, + DELETE_CLUSTER, + + // detailed update + GET_PEER_CLUSTER, + UPDATE_PEER_CLUSTER, + GET_FAILURE_DOMAIN, + UPDATE_FAILURE_DOMAIN, + DELETE_FAILURE_DOMAIN +} diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/PolicyName.java b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/PolicyName.java index 86ab545215e99..d77f92eb03292 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/PolicyName.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/PolicyName.java @@ -53,5 +53,9 @@ public enum PolicyName { ENTRY_FILTERS, SHADOW_TOPIC, DISPATCHER_PAUSE_ON_ACK_STATE_PERSISTENT, - ALLOW_CLUSTERS + ALLOW_CLUSTERS, + + // cluster policies + CLUSTER_MIGRATION, + NAMESPACE_ISOLATION, } From 7f7e12bf6e1a74119b87be2c85a509a935b20e57 Mon Sep 17 00:00:00 2001 From: jiangpengcheng Date: Wed, 4 Dec 2024 18:13:50 +0800 Subject: [PATCH 19/22] [fix][broker] Invoke custom BrokerInterceptor's `onFilter` method if it's defined (#23676) --- .../BrokerInterceptorWithClassLoader.java | 13 +++++++++++++ .../org/apache/pulsar/broker/web/WebService.java | 14 +++++++++++++- .../plugins/LoggingBrokerInterceptor.java | 6 +++++- .../plugins/TestBrokerInterceptors.java | 1 + 4 files changed, 32 insertions(+), 2 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/intercept/BrokerInterceptorWithClassLoader.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/intercept/BrokerInterceptorWithClassLoader.java index 3997e214f4316..849f7aa39f0ef 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/intercept/BrokerInterceptorWithClassLoader.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/intercept/BrokerInterceptorWithClassLoader.java @@ -22,6 +22,7 @@ import io.netty.buffer.ByteBuf; import java.io.IOException; import java.util.Map; +import javax.servlet.FilterChain; import javax.servlet.ServletException; import javax.servlet.ServletRequest; import javax.servlet.ServletResponse; @@ -272,6 +273,18 @@ public void initialize(PulsarService pulsarService) throws Exception { } } + @Override + public void onFilter(ServletRequest request, ServletResponse response, FilterChain chain) + throws ServletException, IOException { + final ClassLoader previousContext = Thread.currentThread().getContextClassLoader(); + try { + Thread.currentThread().setContextClassLoader(narClassLoader); + this.interceptor.onFilter(request, response, chain); + } finally { + Thread.currentThread().setContextClassLoader(previousContext); + } + } + @Override public void close() { final ClassLoader previousContext = Thread.currentThread().getContextClassLoader(); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/web/WebService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/web/WebService.java index 5f5e260890a02..7eb1f2fae09b6 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/web/WebService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/web/WebService.java @@ -41,6 +41,8 @@ import org.apache.pulsar.broker.PulsarServerException; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.ServiceConfiguration; +import org.apache.pulsar.broker.intercept.BrokerInterceptor; +import org.apache.pulsar.broker.intercept.BrokerInterceptors; import org.apache.pulsar.common.util.PulsarSslConfiguration; import org.apache.pulsar.common.util.PulsarSslFactory; import org.apache.pulsar.jetty.tls.JettySslContextFactory; @@ -258,7 +260,17 @@ private static class FilterInitializer { // Enable PreInterceptFilter only when interceptors are enabled filterHolders.add( new FilterHolder(new PreInterceptFilter(pulsarService.getBrokerInterceptor(), handler))); - filterHolders.add(new FilterHolder(new ProcessHandlerFilter(pulsarService.getBrokerInterceptor()))); + // The `ProcessHandlerFilter` is used to overwrite `doFilter` method, which cannot be called multiple + // times inside one `Filter`, so we cannot use one `ProcessHandlerFilter` with a `BrokerInterceptors` to + // hold all interceptors, instead we need to create a `ProcessHandlerFilter` for each `interceptor`. + if (pulsarService.getBrokerInterceptor() instanceof BrokerInterceptors) { + for (BrokerInterceptor interceptor: ((BrokerInterceptors) pulsarService.getBrokerInterceptor()) + .getInterceptors().values()) { + filterHolders.add(new FilterHolder(new ProcessHandlerFilter(interceptor))); + } + } else { + filterHolders.add(new FilterHolder(new ProcessHandlerFilter(pulsarService.getBrokerInterceptor()))); + } } if (config.isAuthenticationEnabled()) { diff --git a/tests/docker-images/java-test-plugins/src/main/java/org/apache/pulsar/tests/integration/plugins/LoggingBrokerInterceptor.java b/tests/docker-images/java-test-plugins/src/main/java/org/apache/pulsar/tests/integration/plugins/LoggingBrokerInterceptor.java index 992c6dd69a6b2..7e46ba18492d2 100644 --- a/tests/docker-images/java-test-plugins/src/main/java/org/apache/pulsar/tests/integration/plugins/LoggingBrokerInterceptor.java +++ b/tests/docker-images/java-test-plugins/src/main/java/org/apache/pulsar/tests/integration/plugins/LoggingBrokerInterceptor.java @@ -19,8 +19,10 @@ package org.apache.pulsar.tests.integration.plugins; import io.netty.buffer.ByteBuf; +import java.io.IOException; import java.util.Map; import javax.servlet.FilterChain; +import javax.servlet.ServletException; import javax.servlet.ServletRequest; import javax.servlet.ServletResponse; import org.apache.bookkeeper.mledger.Entry; @@ -122,7 +124,9 @@ public void txnEnded(String txnID, long txnAction) { } @Override - public void onFilter(ServletRequest request, ServletResponse response, FilterChain chain) { + public void onFilter(ServletRequest request, ServletResponse response, FilterChain chain) + throws ServletException, IOException { log.info("onFilter"); + chain.doFilter(request, response); } } diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/plugins/TestBrokerInterceptors.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/plugins/TestBrokerInterceptors.java index 98000c6f40636..b39339969e52c 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/plugins/TestBrokerInterceptors.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/plugins/TestBrokerInterceptors.java @@ -96,6 +96,7 @@ public void test(Supplier serviceUrlSupplier) throws Exception { "consumerCreated", "messageProduced", "beforeSendMessage: OK", + "onFilter", }) { assertTrue(log.contains("LoggingBrokerInterceptor - " + line), "Log did not contain line '" + line + "'"); } From 04cec0fca7930f0f800fef4119d7bd0de6097da6 Mon Sep 17 00:00:00 2001 From: Wenzhi Feng <52550727+thetumbled@users.noreply.github.com> Date: Thu, 5 Dec 2024 16:55:38 +0800 Subject: [PATCH 20/22] [improve][pip] PIP-393: Improve performance of Negative Acknowledgement (#23601) --- pip/pip-393.md | 226 +++++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 226 insertions(+) create mode 100644 pip/pip-393.md diff --git a/pip/pip-393.md b/pip/pip-393.md new file mode 100644 index 0000000000000..646c2beb5fe40 --- /dev/null +++ b/pip/pip-393.md @@ -0,0 +1,226 @@ + +# PIP-393: Improve performance of Negative Acknowledgement + +# Background knowledge + +Negative Acknowledgement is a feature in Pulsar that allows consumers to trigger the redelivery +of a message after some time when they fail to process it. When user calls `negativeAcknowledge` method, +`NegativeAcksTracker` in `ConsumerImpl` will add an entry into the map `NegativeAcksTracker.nackedMessages`, +mapping the message ID to the redelivery time. When the redelivery time comes, `NegativeAcksTracker` will +send a redelivery request to the broker to redeliver the message. + +# Motivation + +There are many issues with the current implementation of Negative Acknowledgement in Pulsar: +- the memory occupation is high. +- the code execution efficiency is low. +- the redelivery time is not accurate. +- multiple negative ack for messages in the same entry(batch) will interfere with each other. +All of these problem is severe and need to be solved. + +## Memory occupation is high +After the improvement of https://github.com/apache/pulsar/pull/23582, we have reduce half more memory occupation +of `NegativeAcksTracker` by replacing `HashMap` with `ConcurrentLongLongPairHashMap`. With 1 million entry, the memory +occupation decrease from 178MB to 64MB. With 10 million entry, the memory occupation decrease from 1132MB to 512MB. +The average memory occupation of each entry decrease from 1132MB/10000000=118byte to 512MB/10000000=53byte. + +But it is not enough. Assuming that we negative ack message 10k/s, assigning 1h redelivery delay for each message, +the memory occupation of `NegativeAcksTracker` will be `3600*10000*53/1024/1024/1024=1.77GB`, if the delay is 5h, +the required memory is `3600*10000*53/1024/1024/1024*5=8.88GB`, which increase too fast. + +## Code execution efficiency is low +Currently, each time the timer task is triggered, it will iterate all the entries in `NegativeAcksTracker.nackedMessages`, +which is unnecessary. We can sort entries by timestamp and only iterate the entries that need to be redelivered. + +## Redelivery time is not accurate +Currently, the redelivery check time is controlled by the `timerIntervalNanos`, which is 1/3 of the `negativeAckRedeliveryDelay`. +That means, if the `negativeAckRedeliveryDelay` is 1h, check task will be started every 20min, the deviation of the redelivery +time is 20min, which is unacceptable. + +## Multiple negative ack for messages in the same entry(batch) will interfere with each other +Currently, `NegativeAcksTracker#nackedMessages` map `(ledgerId, entryId)` to `timestamp`, which means multiple nacks from messages +in the same batch share single one timestamp. +If we let msg1 redelivered 10s later, then let msg2 redelivered 20s later, these two messages are delivered 20s later together. +msg1 will not be redelivered 10s later as the timestamp recorded in `NegativeAcksTracker#nackedMessages` is overrode by the second +nack call. + + +# Goals + +Refactor the `NegativeAcksTracker` to solve the above problems. + +To avoid interation of all entries in `NegativeAcksTracker.nackedMessages`, we use a sorted map to store the entries. +To reduce memory occupation, we use util class provided by fastutil(https://fastutil.di.unimi.it/docs/), and design +a new algorithm to store the entries, reduce the memory occupation to even 1% less than the current implementation. +(the actual effect rely on the configuration and the throughput). + +# Detailed Design + +## Design & Implementation Details + +### New Data Structure +Use following data structure to store the entries: +```java +Long2ObjectSortedMap> nackedMessages = new Long2ObjectAVLTreeMap<>(); +``` +mapping `timestamp -> ledgerId -> entryId`. +We need to sort timestamp in ascending order, so we use a sorted map to map timestamp to `ledgerId -> entryId` map. +As there will many entries in the map, we use `Long2ObjectAVLTreeMap` instead of `Long2ObjectRBTreeMap`. +As for the inner map, we use `Long2ObjectMap` to map `ledgerId` to `entryId` because we don't need to keep the order of `ledgerId`. +`Long2ObjectOpenHashMap` will be satisfied. +All entry id for the same ledger id will be stored in a bit set, as we only care about the existence of the entry id. + + +### TimeStamp Bucket +Timestamp in ms is used as the key of the map. As most of the use cases don't require that the precision of the delay time is 1ms, +we can make the timestamp bucketed, that is, we can trim the lower bit of the timestamp to map the timestamp to a bucket. +For example, if we trim the lower 1 bit of the timestamp, the timestamp 0b1000 and 0b1001 will be mapped to the same bucket 0b1000. +Then all messages in the same bucket will be redelivered at the same time. +If user can accept 1024ms deviation of the redelivery time, we can trim the lower 10 bits of the timestamp, which can group a lot +entries into the same bucket and reduce the memory occupation. + +following code snippet will be helpful to understand the design: +```java + static long trimLowerBit(long timestamp, int bits) { + return timestamp & (-1L << bits); + } +``` + +```java +Long2ObjectSortedMap> map = new Long2ObjectAVLTreeMap<>(); +Long2ObjectMap ledgerMap = new Long2ObjectOpenHashMap<>(); +LongSet entrySet = new LongOpenHashSet(); +entrySet.add(entryId); +ledgerMap.put(ledgerId, entrySet); +map.put(timestamp, ledgerMap); +``` + +### Configuration + +Add a new configuration `negativeAckPrecisionBitCnt` to control the precision of the redelivery time. +``` +@ApiModelProperty( + name = "negativeAckPrecisionBitCnt", + value = "The redelivery time precision bit count. The lower bits of the redelivery time will be\n" + + "trimmed to reduce the memory occupation. The default value is 8, which means the redelivery time\n" + + "will be bucketed by 256ms. In worst cases, the redelivery time will be 512ms earlier(no later)\n" + + "than the expected time. If the value is 0, the redelivery time will be accurate to ms.". + ) + private long negativeAckPrecisionBitCnt = 8; +``` +The higher the value, the more entries will be grouped into the same bucket, the less memory occupation, the less accurate the redelivery time. +Default value is 8, which means the redelivery time will be bucketed by 256ms. In worst cases, the redelivery time will be 512ms earlier(no later) +than the expected time. + + +## Space complexity analysis +### Space complexity of `ConcurrentLongLongPairHashMap` +Before analyzing the new data structure, we need to know how much space it take before this pip. + +We need to store 4 long field for `(ledgerId, entryId, partitionIndex, timestamp)` for each entry, which takes `4*8=32byte`. +As `ConcurrentLongLongPairHashMap` use open hash addressing and linear probe to handle hash conflict, there are some +redundant spaces to avoid high conflict rate. There are two configurations that control how much redundant space to reserver: +`fill factor` and `idle factor`. When the space utility rate soar high to `fill factor`, the size of backing array will +be double, when the space utility rate reduce to `idle factor`, the size of backing array will reduce by half. + +The default value of `fill factor` is 0.66, `idle factor` is 0.15, which means the min space occupation of +`ConcurrentLongLongPairHashMap` is `32/0.66N byte = 48N byte`, the max space occupation is `32/0.15N byte=213N byte`, +where N is the number of entries. + +In the experiment showed in the PR, there are 1 million entries in the map, taking up `32*1000000/1024/1024byte=30MB`, +the space utility rate is 30/64=0.46, in the range of `[0.15, 0.66]`. + + +### Space complexity of the new data structure +The space used by new data structure is related to several factors: `message rate`, `the time deviation user accepted`, +`the max entries written in one ledger`. +- Pulsar conf `managedLedgerMaxEntriesPerLedger=50000` determine the max entries can be written into one ledger, +we use the default value to analyze. +- `the time deviation user accepted`: when user accept 1024ms delivery time deviation, we can trim the lower 10 bit +of the timestamp in ms, which can bucket 1024 timestamp. + +Following we will analyze the space used by one bucket, and calculate the average space used by one entry. + +Assuming that the message rate is `x msg/ms`, and we trim `y bit` of the timestamp, one bucket will contains `2**x` ms, and +`M=2**x*y` msgs. +- For one single bucket, we only need to store one timestamp, which takes `8byte`. +- Then, we need to store the ledgerId, when M is greater than 5w(`managedLedgerMaxEntriesPerLedger`), the ledger will switch. +There are `L=ceil(M/50000)` ledgers, which take `8*L` byte. +- Further, we analyze how much space the entry id takes. As there are `L=ceil(M/50000)` ledgers, there will be `L` bitmap to store, +which take `L*size(bitmap)`. The total space consumed by new data structure is `8byte + 8L byte + L*size(bitmap)`. + +As the `size(bitmap)` is far more greater than `8byte`, we can ignore the first two items. Then we get the formular of space +consumed **one bucket**: `D=L*size(bitmap)=ceil(M/50000)*size(bitmap)`. + +Entry id is stored in a `Roaring64Bitmap`, for simplicity we can replace it with `RoaringBitmap`, as the max entry id is 49999, +which is smaller than `4294967296 (2 * Integer.MAX_VALUE)`(the max value can be stored in `RoaringBitmap`). The space consume +by `RoaringBitmap` depends on how many elements it contains, when the size of bitmap < 4096, the space is `4N byte`, +when the size of bitmap > 4096, the consumed space is a fixed value `8KB`. + +Then we get the final result: +- when M>50000, `D = ceil(M/50000)*size(bitmap) ~= M/50000 * 8KB = M/50000 * 8 * 1024 byte = 0.163M byte`, +each entry takes `0.163byte` by average. +- when 409650000, `0.163N byte`. + - when 4096 +* Mailing List discussion thread: https://lists.apache.org/thread/yojl7ylk7cyjxktq3cn8849hvmyv0fg8 +* Mailing List voting thread: https://lists.apache.org/thread/hyc1r2s9chowdhck53lq07tznopt50dy From d791ecd961127cfc3d781cc33ee0e9ae2d7015c4 Mon Sep 17 00:00:00 2001 From: thetumbled <843221020@qq.com> Date: Thu, 5 Dec 2024 18:07:31 +0800 Subject: [PATCH 21/22] fix conflict. --- .../pulsar/client/impl/NegativeAcksTest.java | 50 ++++++- pulsar-client-all/pom.xml | 1 + pulsar-client-shaded/pom.xml | 1 + pulsar-client/pom.xml | 10 ++ .../pulsar/client/impl/ConsumerImpl.java | 4 +- .../client/impl/NegativeAcksTracker.java | 140 ++++++++++++------ .../impl/conf/ConsumerConfigurationData.java | 10 ++ 7 files changed, 165 insertions(+), 51 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/NegativeAcksTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/NegativeAcksTest.java index b372ecabc5de4..f8bc30f09667c 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/NegativeAcksTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/NegativeAcksTest.java @@ -19,6 +19,7 @@ package org.apache.pulsar.client.impl; import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertNotNull; import static org.testng.Assert.assertNull; import static org.testng.Assert.assertTrue; import java.util.HashSet; @@ -311,7 +312,7 @@ public void testNegativeAcksDeleteFromUnackedTracker() throws Exception { // negative topic message id consumer.negativeAcknowledge(topicMessageId); NegativeAcksTracker negativeAcksTracker = consumer.getNegativeAcksTracker(); - assertEquals(negativeAcksTracker.getNackedMessagesCount().orElse((long) -1).longValue(), 1L); + assertEquals(negativeAcksTracker.getNackedMessagesCount(), 1L); assertEquals(unAckedMessageTracker.size(), 0); negativeAcksTracker.close(); // negative batch message id @@ -319,11 +320,56 @@ public void testNegativeAcksDeleteFromUnackedTracker() throws Exception { consumer.negativeAcknowledge(batchMessageId); consumer.negativeAcknowledge(batchMessageId2); consumer.negativeAcknowledge(batchMessageId3); - assertEquals(negativeAcksTracker.getNackedMessagesCount().orElse((long) -1).longValue(), 1L); + assertEquals(negativeAcksTracker.getNackedMessagesCount(), 1L); assertEquals(unAckedMessageTracker.size(), 0); negativeAcksTracker.close(); } + /** + * If we nack multiple messages in the same batch with different redelivery delays, the messages should be redelivered + * with the correct delay. However, all messages are redelivered at the same time. + * @throws Exception + */ + @Test + public void testNegativeAcksWithBatch() throws Exception { + cleanup(); + conf.setAcknowledgmentAtBatchIndexLevelEnabled(true); + setup(); + String topic = BrokerTestUtil.newUniqueName("testNegativeAcksWithBatch"); + + @Cleanup + Consumer consumer = pulsarClient.newConsumer(Schema.STRING) + .topic(topic) + .subscriptionName("sub1") + .acknowledgmentGroupTime(0, TimeUnit.SECONDS) + .subscriptionType(SubscriptionType.Shared) + .enableBatchIndexAcknowledgment(true) + .negativeAckRedeliveryDelay(3, TimeUnit.SECONDS) + .subscribe(); + + @Cleanup + Producer producer = pulsarClient.newProducer(Schema.STRING) + .topic(topic) + .enableBatching(true) + .batchingMaxPublishDelay(1, TimeUnit.HOURS) + .batchingMaxMessages(2) + .create(); + // send two messages in the same batch + producer.sendAsync("test-0"); + producer.sendAsync("test-1"); + producer.flush(); + + // negative ack the first message + consumer.negativeAcknowledge(consumer.receive()); + // wait for 2s, negative ack the second message + Thread.sleep(2000); + consumer.negativeAcknowledge(consumer.receive()); + + // now 2s has passed, the first message should be redelivered 1s later. + Message msg1 = consumer.receive(2, TimeUnit.SECONDS); + assertNotNull(msg1); + } + @Test public void testNegativeAcksWithBatchAckEnabled() throws Exception { cleanup(); diff --git a/pulsar-client-all/pom.xml b/pulsar-client-all/pom.xml index 74007745c70ee..f226ac57588fd 100644 --- a/pulsar-client-all/pom.xml +++ b/pulsar-client-all/pom.xml @@ -200,6 +200,7 @@ org.reactivestreams:reactive-streams org.tukaani:xz org.yaml:snakeyaml + it.unimi.dsi:fastutil com.fasterxml.jackson.core:jackson-annotations diff --git a/pulsar-client-shaded/pom.xml b/pulsar-client-shaded/pom.xml index 1093b405731ea..47c6ac835054f 100644 --- a/pulsar-client-shaded/pom.xml +++ b/pulsar-client-shaded/pom.xml @@ -164,6 +164,7 @@ org.reactivestreams:reactive-streams org.tukaani:xz org.yaml:snakeyaml + it.unimi.dsi:fastutil com.fasterxml.jackson.core:jackson-annotations diff --git a/pulsar-client/pom.xml b/pulsar-client/pom.xml index 49bb3c6490ae9..e1a70ed074833 100644 --- a/pulsar-client/pom.xml +++ b/pulsar-client/pom.xml @@ -207,6 +207,16 @@ test + + org.roaringbitmap + RoaringBitmap + + + + it.unimi.dsi + fastutil + + diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java index d2753856264fc..a7eb89bda157f 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java @@ -2745,7 +2745,7 @@ private int removeExpiredMessagesFromQueue(Set messageIds) { int messagesFromQueue = 0; Message peek = incomingMessages.peek(); if (peek != null) { - MessageIdAdv messageId = MessageIdAdvUtils.discardBatch(peek.getMessageId()); + MessageId messageId = NegativeAcksTracker.discardBatchAndPartitionIndex(peek.getMessageId()); if (!messageIds.contains(messageId)) { // first message is not expired, then no message is expired in queue. return 0; @@ -2756,7 +2756,7 @@ private int removeExpiredMessagesFromQueue(Set messageIds) { while (message != null) { decreaseIncomingMessageSize(message); messagesFromQueue++; - MessageIdAdv id = MessageIdAdvUtils.discardBatch(message.getMessageId()); + MessageId id = NegativeAcksTracker.discardBatchAndPartitionIndex(message.getMessageId()); if (!messageIds.contains(id)) { messageIds.add(id); break; diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/NegativeAcksTracker.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/NegativeAcksTracker.java index 5256ebf04f43c..8252f2d5ab8d5 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/NegativeAcksTracker.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/NegativeAcksTracker.java @@ -24,48 +24,50 @@ import io.netty.util.Timer; import java.io.Closeable; import java.util.HashSet; -import java.util.Optional; import java.util.Set; import java.util.concurrent.TimeUnit; + +import it.unimi.dsi.fastutil.longs.Long2ObjectMap; +import it.unimi.dsi.fastutil.longs.Long2ObjectOpenHashMap; +import it.unimi.dsi.fastutil.longs.Long2ObjectRBTreeMap; +import it.unimi.dsi.fastutil.longs.Long2ObjectSortedMap; +import it.unimi.dsi.fastutil.longs.LongBidirectionalIterator; import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.MessageIdAdv; import org.apache.pulsar.client.api.RedeliveryBackoff; import org.apache.pulsar.client.impl.conf.ConsumerConfigurationData; -import org.apache.pulsar.common.util.collections.ConcurrentLongLongPairHashMap; +import org.roaringbitmap.longlong.Roaring64Bitmap; import org.slf4j.Logger; import org.slf4j.LoggerFactory; class NegativeAcksTracker implements Closeable { private static final Logger log = LoggerFactory.getLogger(NegativeAcksTracker.class); - private ConcurrentLongLongPairHashMap nackedMessages = null; + // timestamp -> ledgerId -> entryId, no need to batch index, if different messages have + // different timestamp, there will be multiple entries in the map + // RB Tree -> LongOpenHashMap -> Roaring64Bitmap + private Long2ObjectSortedMap> nackedMessages = null; private final ConsumerBase consumer; private final Timer timer; - private final long nackDelayNanos; - private final long timerIntervalNanos; + private final long nackDelayMs; private final RedeliveryBackoff negativeAckRedeliveryBackoff; + private final int negativeAckPrecisionBitCnt; private Timeout timeout; // Set a min delay to allow for grouping nacks within a single batch - private static final long MIN_NACK_DELAY_NANOS = TimeUnit.MILLISECONDS.toNanos(100); - private static final long NON_PARTITIONED_TOPIC_PARTITION_INDEX = Long.MAX_VALUE; + private static final long MIN_NACK_DELAY_MS = 100; + private static final int DUMMY_PARTITION_INDEX = -2; public NegativeAcksTracker(ConsumerBase consumer, ConsumerConfigurationData conf) { this.consumer = consumer; this.timer = consumer.getClient().timer(); - this.nackDelayNanos = Math.max(TimeUnit.MICROSECONDS.toNanos(conf.getNegativeAckRedeliveryDelayMicros()), - MIN_NACK_DELAY_NANOS); + this.nackDelayMs = Math.max(TimeUnit.MICROSECONDS.toMillis(conf.getNegativeAckRedeliveryDelayMicros()), + MIN_NACK_DELAY_MS); this.negativeAckRedeliveryBackoff = conf.getNegativeAckRedeliveryBackoff(); - if (negativeAckRedeliveryBackoff != null) { - this.timerIntervalNanos = Math.max( - TimeUnit.MILLISECONDS.toNanos(negativeAckRedeliveryBackoff.next(0)), - MIN_NACK_DELAY_NANOS) / 3; - } else { - this.timerIntervalNanos = nackDelayNanos / 3; - } + this.negativeAckPrecisionBitCnt = conf.getNegativeAckPrecisionBitCnt(); } private void triggerRedelivery(Timeout t) { @@ -76,21 +78,48 @@ private void triggerRedelivery(Timeout t) { return; } - long now = System.nanoTime(); - nackedMessages.forEach((ledgerId, entryId, partitionIndex, timestamp) -> { - if (timestamp < now) { - MessageId msgId = new MessageIdImpl(ledgerId, entryId, - // need to covert non-partitioned topic partition index to -1 - (int) (partitionIndex == NON_PARTITIONED_TOPIC_PARTITION_INDEX ? -1 : partitionIndex)); - addChunkedMessageIdsAndRemoveFromSequenceMap(msgId, messagesToRedeliver, this.consumer); - messagesToRedeliver.add(msgId); + long currentTimestamp = System.currentTimeMillis(); + for (long timestamp : nackedMessages.keySet()) { + if (timestamp > currentTimestamp) { + // We are done with all the messages that need to be redelivered + break; + } + + Long2ObjectMap ledgerMap = nackedMessages.get(timestamp); + for (Long2ObjectMap.Entry ledgerEntry : ledgerMap.long2ObjectEntrySet()) { + long ledgerId = ledgerEntry.getLongKey(); + Roaring64Bitmap entrySet = ledgerEntry.getValue(); + entrySet.forEach(entryId -> { + MessageId msgId = new MessageIdImpl(ledgerId, entryId, DUMMY_PARTITION_INDEX); + addChunkedMessageIdsAndRemoveFromSequenceMap(msgId, messagesToRedeliver, this.consumer); + messagesToRedeliver.add(msgId); + }); + } + } + + // remove entries from the nackedMessages map + LongBidirectionalIterator iterator = nackedMessages.keySet().iterator(); + while (iterator.hasNext()) { + long timestamp = iterator.nextLong(); + if (timestamp <= currentTimestamp) { + iterator.remove(); + } else { + break; + } + } + + // Schedule the next redelivery if there are still messages to redeliver + if (!nackedMessages.isEmpty()) { + long nextTriggerTimestamp = nackedMessages.firstLongKey(); + long delayMs = Math.max(nextTriggerTimestamp - currentTimestamp, 0); + if (delayMs > 0) { + this.timeout = timer.newTimeout(this::triggerRedelivery, delayMs, TimeUnit.MILLISECONDS); + } else { + this.timeout = timer.newTimeout(this::triggerRedelivery, 0, TimeUnit.MILLISECONDS); } - }); - for (MessageId messageId : messagesToRedeliver) { - nackedMessages.remove(((MessageIdImpl) messageId).getLedgerId(), - ((MessageIdImpl) messageId).getEntryId()); + } else { + this.timeout = null; } - this.timeout = timer.newTimeout(this::triggerRedelivery, timerIntervalNanos, TimeUnit.NANOSECONDS); } // release the lock of NegativeAcksTracker before calling consumer.redeliverUnacknowledgedMessages, @@ -110,39 +139,56 @@ public synchronized void add(Message message) { add(message.getMessageId(), message.getRedeliveryCount()); } + static long trimLowerBit(long timestamp, int bits) { + return timestamp & (-1L << bits); + } + private synchronized void add(MessageId messageId, int redeliveryCount) { if (nackedMessages == null) { - nackedMessages = ConcurrentLongLongPairHashMap.newBuilder() - .autoShrink(true) - .concurrencyLevel(1) - .build(); + nackedMessages = new Long2ObjectRBTreeMap<>(); } - long backoffNs; + long backoffMs; if (negativeAckRedeliveryBackoff != null) { - backoffNs = TimeUnit.MILLISECONDS.toNanos(negativeAckRedeliveryBackoff.next(redeliveryCount)); + backoffMs = TimeUnit.MILLISECONDS.toMillis(negativeAckRedeliveryBackoff.next(redeliveryCount)); } else { - backoffNs = nackDelayNanos; + backoffMs = nackDelayMs; } - MessageIdAdv messageIdAdv = MessageIdAdvUtils.discardBatch(messageId); - // ConcurrentLongLongPairHashMap requires the key and value >=0. - // partitionIndex is -1 if the message is from a non-partitioned topic, but we don't use - // partitionIndex actually, so we can set it to Long.MAX_VALUE in the case of non-partitioned topic to - // avoid exception from ConcurrentLongLongPairHashMap. - nackedMessages.put(messageIdAdv.getLedgerId(), messageIdAdv.getEntryId(), - messageIdAdv.getPartitionIndex() >= 0 ? messageIdAdv.getPartitionIndex() : - NON_PARTITIONED_TOPIC_PARTITION_INDEX, System.nanoTime() + backoffNs); + MessageIdAdv messageIdAdv = (MessageIdAdv) messageId; + long timestamp = trimLowerBit(System.currentTimeMillis() + backoffMs, negativeAckPrecisionBitCnt); + nackedMessages.computeIfAbsent(timestamp, k -> new Long2ObjectOpenHashMap<>()) + .computeIfAbsent(messageIdAdv.getLedgerId(), k -> new Roaring64Bitmap()) + .add(messageIdAdv.getEntryId()); if (this.timeout == null) { // Schedule a task and group all the redeliveries for same period. Leave a small buffer to allow for // nack immediately following the current one will be batched into the same redeliver request. - this.timeout = timer.newTimeout(this::triggerRedelivery, timerIntervalNanos, TimeUnit.NANOSECONDS); + this.timeout = timer.newTimeout(this::triggerRedelivery, backoffMs, TimeUnit.MILLISECONDS); } } + /** + * Discard the batch index and partition index from the message id. + * + * @param messageId + * @return + */ + public static MessageIdAdv discardBatchAndPartitionIndex(MessageId messageId) { + if (messageId instanceof ChunkMessageIdImpl) { + return (MessageIdAdv) messageId; + } + MessageIdAdv msgId = (MessageIdAdv) messageId; + return new MessageIdImpl(msgId.getLedgerId(), msgId.getEntryId(), DUMMY_PARTITION_INDEX); + } + @VisibleForTesting - Optional getNackedMessagesCount() { - return Optional.ofNullable(nackedMessages).map(ConcurrentLongLongPairHashMap::size); + synchronized long getNackedMessagesCount() { + if (nackedMessages == null) { + return 0; + } + return nackedMessages.values().stream().mapToLong( + ledgerMap -> ledgerMap.values().stream().mapToLong( + Roaring64Bitmap::getLongCardinality).sum()).sum(); } @Override diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/conf/ConsumerConfigurationData.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/conf/ConsumerConfigurationData.java index f9ff5913f62da..f430371d37c75 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/conf/ConsumerConfigurationData.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/conf/ConsumerConfigurationData.java @@ -155,6 +155,16 @@ public class ConsumerConfigurationData implements Serializable, Cloneable { ) private long negativeAckRedeliveryDelayMicros = TimeUnit.MINUTES.toMicros(1); + @ApiModelProperty( + name = "negativeAckPrecisionBitCnt", + value = "The redelivery time precision bit count. The lower bits of the redelivery time will be" + + "trimmed to reduce the memory occupation.\nThe default value is 8, which means the" + + "redelivery time will be bucketed by 256ms, the redelivery time could be earlier(no later)" + + "than the expected time, but no more than 256ms. \nIf set to k, the redelivery time will be" + + "bucketed by 2^k ms.\nIf the value is 0, the redelivery time will be accurate to ms." + ) + private int negativeAckPrecisionBitCnt = 8; + @ApiModelProperty( name = "maxTotalReceiverQueueSizeAcrossPartitions", value = "The max total receiver queue size across partitions.\n" From 782c586b8a7c3a1ac31c2f915f711ace85ff95d2 Mon Sep 17 00:00:00 2001 From: thetumbled <843221020@qq.com> Date: Thu, 5 Dec 2024 18:17:41 +0800 Subject: [PATCH 22/22] fix check. --- .../pulsar/client/impl/NegativeAcksTracker.java | 13 ++++++------- 1 file changed, 6 insertions(+), 7 deletions(-) diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/NegativeAcksTracker.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/NegativeAcksTracker.java index 8252f2d5ab8d5..273880569c307 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/NegativeAcksTracker.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/NegativeAcksTracker.java @@ -22,16 +22,15 @@ import com.google.common.annotations.VisibleForTesting; import io.netty.util.Timeout; import io.netty.util.Timer; -import java.io.Closeable; -import java.util.HashSet; -import java.util.Set; -import java.util.concurrent.TimeUnit; - +import it.unimi.dsi.fastutil.longs.Long2ObjectAVLTreeMap; import it.unimi.dsi.fastutil.longs.Long2ObjectMap; import it.unimi.dsi.fastutil.longs.Long2ObjectOpenHashMap; -import it.unimi.dsi.fastutil.longs.Long2ObjectRBTreeMap; import it.unimi.dsi.fastutil.longs.Long2ObjectSortedMap; import it.unimi.dsi.fastutil.longs.LongBidirectionalIterator; +import java.io.Closeable; +import java.util.HashSet; +import java.util.Set; +import java.util.concurrent.TimeUnit; import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.MessageIdAdv; @@ -145,7 +144,7 @@ static long trimLowerBit(long timestamp, int bits) { private synchronized void add(MessageId messageId, int redeliveryCount) { if (nackedMessages == null) { - nackedMessages = new Long2ObjectRBTreeMap<>(); + nackedMessages = new Long2ObjectAVLTreeMap<>(); } long backoffMs;