You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pulsar.apache.org by ji...@apache.org on 2022/09/26 02:55:06 UTC
[pulsar] 02/02: [fix][broker] Fix namespace backlog quota check with retention. (#17706)
This is an automated email from the ASF dual-hosted git repository.
jianghaiting pushed a commit to branch branch-2.10
in repository https://gitbox.apache.org/repos/asf/pulsar.git
commit 44a07542e9472363588b1c4146f6371bf4997c32
Author: JiangHaiting <ji...@apache.org>
AuthorDate: Mon Sep 19 10:29:51 2022 +0800
[fix][broker] Fix namespace backlog quota check with retention. (#17706)
(cherry picked from commit c6967cd37d11c01c9cc873233e157c7d1af70c6a)
---
.../apache/pulsar/broker/admin/AdminResource.java | 10 +++++--
.../apache/pulsar/broker/admin/AdminApiTest.java | 32 ++++++++++++++++++++++
2 files changed, 39 insertions(+), 3 deletions(-)
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/AdminResource.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/AdminResource.java
index 488cf73c9ba..eb5b62d0de5 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/AdminResource.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/AdminResource.java
@@ -355,17 +355,21 @@ public abstract class AdminResource extends PulsarWebResource {
}
protected boolean checkBacklogQuota(BacklogQuota quota, RetentionPolicies retention) {
- if (retention == null || retention.getRetentionSizeInMB() <= 0 || retention.getRetentionTimeInMinutes() <= 0) {
+ if (retention == null
+ || (retention.getRetentionSizeInMB() <= 0 && retention.getRetentionTimeInMinutes() <= 0)) {
return true;
}
if (quota == null) {
quota = pulsar().getBrokerService().getBacklogQuotaManager().getDefaultQuota();
}
- if (quota.getLimitSize() >= (retention.getRetentionSizeInMB() * 1024 * 1024)) {
+
+ if (retention.getRetentionSizeInMB() > 0
+ && quota.getLimitSize() >= (retention.getRetentionSizeInMB() * 1024 * 1024)) {
return false;
}
// time based quota is in second
- if (quota.getLimitTime() >= (retention.getRetentionTimeInMinutes() * 60)) {
+ if (retention.getRetentionTimeInMinutes() > 0
+ && quota.getLimitTime() >= retention.getRetentionTimeInMinutes() * 60) {
return false;
}
return true;
diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiTest.java
index 98400036f03..1c6e469935d 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiTest.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiTest.java
@@ -3135,4 +3135,36 @@ public class AdminApiTest extends MockedPulsarServiceBaseTest {
assertEquals(peekedMessages.get(i).getData(), receivedMessages.get(i).getData());
}
}
+
+ @Test
+ public void testRetentionAndBacklogQuotaCheck() throws PulsarAdminException {
+ String namespace = "prop-xyz/ns1";
+ //test size check.
+ admin.namespaces().setRetention(namespace, new RetentionPolicies(-1, 10));
+ admin.namespaces().setBacklogQuota(namespace, BacklogQuota.builder().limitSize(9 * 1024 * 1024).build());
+ Assert.expectThrows(PulsarAdminException.PreconditionFailedException.class, () -> {
+ admin.namespaces().setBacklogQuota(namespace, BacklogQuota.builder().limitSize(100 * 1024 * 1024).build());
+ });
+
+ //test time check
+ admin.namespaces().setRetention(namespace, new RetentionPolicies(10, -1));
+ admin.namespaces().setBacklogQuota(namespace, BacklogQuota.builder().limitTime(9 * 60).build());
+ Assert.expectThrows(PulsarAdminException.PreconditionFailedException.class, () -> {
+ admin.namespaces().setBacklogQuota(namespace, BacklogQuota.builder().limitTime(11 * 60).build());
+ });
+
+ // test both size and time.
+ admin.namespaces().setRetention(namespace, new RetentionPolicies(10, 10));
+ admin.namespaces().setBacklogQuota(namespace, BacklogQuota.builder().limitSize(9 * 1024 * 1024).build());
+ admin.namespaces().setBacklogQuota(namespace, BacklogQuota.builder().limitTime(9 * 60).build());
+ admin.namespaces().setBacklogQuota(namespace, BacklogQuota.builder().limitSize(9 * 1024 * 1024).
+ limitTime(9 * 60).build());
+ Assert.expectThrows(PulsarAdminException.PreconditionFailedException.class, () -> {
+ admin.namespaces().setBacklogQuota(namespace, BacklogQuota.builder().limitSize(100 * 1024 * 1024).build());
+ });
+ Assert.expectThrows(PulsarAdminException.PreconditionFailedException.class, () -> {
+ admin.namespaces().setBacklogQuota(namespace, BacklogQuota.builder().limitTime(100 * 60).build());
+ });
+
+ }
}