|
18 | 18 | */ |
19 | 19 | package org.apache.pulsar.broker.admin; |
20 | 20 |
|
| 21 | +import static org.awaitility.Awaitility.await; |
21 | 22 | import static org.mockito.ArgumentMatchers.any; |
22 | 23 | import static org.mockito.ArgumentMatchers.anyBoolean; |
23 | 24 | import static org.mockito.ArgumentMatchers.anyString; |
|
66 | 67 | import org.apache.pulsar.broker.web.RestException; |
67 | 68 | import org.apache.pulsar.client.api.Consumer; |
68 | 69 | import org.apache.pulsar.client.api.Message; |
| 70 | +import org.apache.pulsar.client.api.MessageId; |
69 | 71 | import org.apache.pulsar.client.api.Producer; |
70 | 72 | import org.apache.pulsar.client.api.Schema; |
71 | 73 | import org.apache.pulsar.client.api.SubscriptionInitialPosition; |
|
82 | 84 | import org.apache.pulsar.client.impl.schema.generic.GenericJsonSchema; |
83 | 85 | import org.apache.pulsar.client.impl.schema.generic.GenericSchemaImpl; |
84 | 86 | import org.apache.pulsar.common.naming.TopicDomain; |
| 87 | +import org.apache.pulsar.common.policies.data.BacklogQuota; |
85 | 88 | import org.apache.pulsar.common.policies.data.ClusterDataImpl; |
86 | 89 | import org.apache.pulsar.common.policies.data.TenantInfoImpl; |
87 | 90 | import org.apache.pulsar.common.schema.KeyValue; |
@@ -931,4 +934,84 @@ public void testProduceWithAutoConsumeSchema() throws Exception { |
931 | 934 | } |
932 | 935 | } |
933 | 936 |
|
| 937 | + @Test |
| 938 | + public void testProduceWithBacklogQuotaSizeExceeded() throws Exception { |
| 939 | + String namespaceName = testTenant + "/" + testNamespace; |
| 940 | + String topicName = "persistent://" + namespaceName + "/" + testTopicName; |
| 941 | + admin.topics().createNonPartitionedTopic(topicName); |
| 942 | + BacklogQuota backlogQuota = BacklogQuota.builder() |
| 943 | + .limitSize(0) |
| 944 | + .retentionPolicy(BacklogQuota.RetentionPolicy.producer_exception) |
| 945 | + .build(); |
| 946 | + admin.namespaces().setBacklogQuota(namespaceName, backlogQuota, |
| 947 | + BacklogQuota.BacklogQuotaType.destination_storage); |
| 948 | + |
| 949 | + AsyncResponse asyncResponse = mock(AsyncResponse.class); |
| 950 | + ProducerMessages producerMessages = new ProducerMessages(); |
| 951 | + String message = "[{\"payload\":\"rest-produce\"}]"; |
| 952 | + producerMessages.setMessages(createMessages(message)); |
| 953 | + topics.produceOnPersistentTopic(asyncResponse, testTenant, testNamespace, testTopicName, |
| 954 | + false, producerMessages); |
| 955 | + ArgumentCaptor<Response> responseCaptor = ArgumentCaptor.forClass(Response.class); |
| 956 | + verify(asyncResponse, timeout(5000).times(1)).resume(responseCaptor.capture()); |
| 957 | + Assert.assertEquals(responseCaptor.getValue().getStatus(), Response.Status.OK.getStatusCode()); |
| 958 | + Object responseEntity = responseCaptor.getValue().getEntity(); |
| 959 | + Assert.assertTrue(responseEntity instanceof ProducerAcks); |
| 960 | + ProducerAcks response = (ProducerAcks) responseEntity; |
| 961 | + Assert.assertEquals(response.getMessagePublishResults().size(), 1); |
| 962 | + for (int index = 0; index < response.getMessagePublishResults().size(); index++) { |
| 963 | + Assert.assertEquals(response.getMessagePublishResults().get(index).getErrorCode(), 2); |
| 964 | + Assert.assertTrue(response.getMessagePublishResults().get(index).getErrorMsg() |
| 965 | + .contains("Cannot create producer on topic with backlog quota exceeded")); |
| 966 | + } |
| 967 | + } |
| 968 | + |
| 969 | + @Test |
| 970 | + public void testProduceWithBacklogQuotaTimeExceeded() throws Exception { |
| 971 | + pulsar.getConfiguration().setPreciseTimeBasedBacklogQuotaCheck(true); |
| 972 | + String namespaceName = testTenant + "/" + testNamespace; |
| 973 | + String topicName = "persistent://" + namespaceName + "/" + testTopicName; |
| 974 | + admin.topics().createNonPartitionedTopic(topicName); |
| 975 | + BacklogQuota backlogQuota = BacklogQuota.builder() |
| 976 | + .limitTime(1) |
| 977 | + .retentionPolicy(BacklogQuota.RetentionPolicy.producer_exception) |
| 978 | + .build(); |
| 979 | + admin.namespaces().setBacklogQuota(namespaceName, backlogQuota, |
| 980 | + BacklogQuota.BacklogQuotaType.message_age); |
| 981 | + admin.topics().createSubscription(topicName, "time-quota-sub", MessageId.earliest); |
| 982 | + |
| 983 | + @Cleanup |
| 984 | + Producer<String> producer = pulsarClient.newProducer(Schema.STRING) |
| 985 | + .topic(topicName) |
| 986 | + .create(); |
| 987 | + producer.send("backlog-message"); |
| 988 | + Topic topic = pulsar.getBrokerService().getTopic(topicName, false) |
| 989 | + .get() |
| 990 | + .orElseThrow(() -> new IllegalStateException("Topic not loaded: " + topicName)); |
| 991 | + PersistentTopic persistentTopic = (PersistentTopic) topic; |
| 992 | + await().atMost(5, TimeUnit.SECONDS).untilAsserted(() -> |
| 993 | + Assert.assertTrue(persistentTopic.checkTimeBacklogExceeded(true).get())); |
| 994 | + |
| 995 | + AsyncResponse asyncResponse = mock(AsyncResponse.class); |
| 996 | + ProducerMessages producerMessages = new ProducerMessages(); |
| 997 | + String message = "[" |
| 998 | + + "{\"key\":\"my-key\",\"payload\":\"RestProducer:1\",\"eventTime\":1603045262772,\"sequenceId\":1}," |
| 999 | + + "{\"key\":\"my-key\",\"payload\":\"RestProducer:2\",\"eventTime\":1603045262772,\"sequenceId\":2}]"; |
| 1000 | + producerMessages.setMessages(createMessages(message)); |
| 1001 | + topics.produceOnPersistentTopic(asyncResponse, testTenant, testNamespace, testTopicName, |
| 1002 | + false, producerMessages); |
| 1003 | + ArgumentCaptor<Response> responseCaptor = ArgumentCaptor.forClass(Response.class); |
| 1004 | + verify(asyncResponse, timeout(5000).times(1)).resume(responseCaptor.capture()); |
| 1005 | + Assert.assertEquals(responseCaptor.getValue().getStatus(), Response.Status.OK.getStatusCode()); |
| 1006 | + Object responseEntity = responseCaptor.getValue().getEntity(); |
| 1007 | + Assert.assertTrue(responseEntity instanceof ProducerAcks); |
| 1008 | + ProducerAcks response = (ProducerAcks) responseEntity; |
| 1009 | + Assert.assertEquals(response.getMessagePublishResults().size(), 2); |
| 1010 | + for (int index = 0; index < response.getMessagePublishResults().size(); index++) { |
| 1011 | + Assert.assertEquals(response.getMessagePublishResults().get(index).getErrorCode(), 2); |
| 1012 | + Assert.assertTrue(response.getMessagePublishResults().get(index).getErrorMsg() |
| 1013 | + .contains("Cannot create producer on topic with backlog quota exceeded")); |
| 1014 | + } |
| 1015 | + } |
| 1016 | + |
934 | 1017 | } |
0 commit comments