You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pulsar.apache.org by GitBox <gi...@apache.org> on 2021/11/23 21:49:55 UTC

[GitHub] [pulsar] eolivelli commented on a change in pull request #12874: [broker] Optimize blocking backlogQuotaCheck to non-blocking in ServerCnx#handleProducer

eolivelli commented on a change in pull request #12874:
URL: https://github.com/apache/pulsar/pull/12874#discussion_r755527273



##########
File path: pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java
##########
@@ -1189,65 +1190,70 @@ protected void handleProducer(final CommandProducer cmdProducer) {
             service.getOrCreateTopic(topicName.toString()).thenAccept((Topic topic) -> {
                 // Before creating producer, check if backlog quota exceeded
                 // on topic for size based limit and time based limit
-                for (BacklogQuota.BacklogQuotaType backlogQuotaType : BacklogQuota.BacklogQuotaType.values()) {
-                    if (topic.isBacklogQuotaExceeded(producerName, backlogQuotaType)) {
-                        IllegalStateException illegalStateException = new IllegalStateException(
-                                "Cannot create producer on topic with backlog quota exceeded");
-                        BacklogQuota.RetentionPolicy retentionPolicy = topic
-                                .getBacklogQuota(backlogQuotaType).getPolicy();
-                        if (retentionPolicy == BacklogQuota.RetentionPolicy.producer_request_hold) {
-                            commandSender.sendErrorResponse(requestId,
-                                    ServerError.ProducerBlockedQuotaExceededError,
-                                    illegalStateException.getMessage());
-                        } else if (retentionPolicy == BacklogQuota.RetentionPolicy.producer_exception) {
-                            commandSender.sendErrorResponse(requestId,
-                                    ServerError.ProducerBlockedQuotaExceededException,
-                                    illegalStateException.getMessage());
-                        }
-                        producerFuture.completeExceptionally(illegalStateException);
-                        producers.remove(producerId, producerFuture);
-                        return;
+                CompletableFuture<Void> backlogQuotaCheckFuture = CompletableFuture.allOf(
+                        topic.checkBacklogQuotaExceeded(producerName, BacklogQuotaType.destination_storage),
+                        topic.checkBacklogQuotaExceeded(producerName, BacklogQuotaType.message_age));
+                backlogQuotaCheckFuture.exceptionally(throwable -> {
+                    //throwable should be CompletionException holding TopicBacklogQuotaExceededException
+                    BrokerServiceException.TopicBacklogQuotaExceededException exception =
+                            (BrokerServiceException.TopicBacklogQuotaExceededException) throwable.getCause();
+                    IllegalStateException illegalStateException = new IllegalStateException(exception);
+                    BacklogQuota.RetentionPolicy retentionPolicy = exception.getRetentionPolicy();
+                    if (retentionPolicy == BacklogQuota.RetentionPolicy.producer_request_hold) {
+                        commandSender.sendErrorResponse(requestId,
+                                ServerError.ProducerBlockedQuotaExceededError,
+                                illegalStateException.getMessage());
+                    } else if (retentionPolicy == BacklogQuota.RetentionPolicy.producer_exception) {
+                        commandSender.sendErrorResponse(requestId,
+                                ServerError.ProducerBlockedQuotaExceededException,
+                                illegalStateException.getMessage());
                     }
-                }
-                // Check whether the producer will publish encrypted messages or not
-                if ((topic.isEncryptionRequired() || encryptionRequireOnProducer) && !isEncrypted) {
-                    String msg = String.format("Encryption is required in %s", topicName);
-                    log.warn("[{}] {}", remoteAddress, msg);
-                    commandSender.sendErrorResponse(requestId, ServerError.MetadataError, msg);
+                    producerFuture.completeExceptionally(illegalStateException);
                     producers.remove(producerId, producerFuture);
-                    return;
-                }
+                    return null;
+                });
 
-                disableTcpNoDelayIfNeeded(topicName.toString(), producerName);
+                backlogQuotaCheckFuture.thenRun(() -> {

Review comment:
       Which thread will execute this code?
   
   The current thread (in case we reach here and the futures are already completed) or some other thread on completion of the one of the two futures above.
   
   I am not sure we have control over what's happening here

##########
File path: pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java
##########
@@ -2518,17 +2528,26 @@ public boolean isSizeBacklogExceeded() {
      * @return determine if backlog quota enforcement needs to be done for topic based on time limit
      */
     public boolean isTimeBacklogExceeded() {
+        try {
+            return checkTimeBacklogExceeded().get();
+        } catch (Throwable e) {

Review comment:
       This is a code smell

##########
File path: pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java
##########
@@ -1189,65 +1190,70 @@ protected void handleProducer(final CommandProducer cmdProducer) {
             service.getOrCreateTopic(topicName.toString()).thenAccept((Topic topic) -> {
                 // Before creating producer, check if backlog quota exceeded
                 // on topic for size based limit and time based limit
-                for (BacklogQuota.BacklogQuotaType backlogQuotaType : BacklogQuota.BacklogQuotaType.values()) {
-                    if (topic.isBacklogQuotaExceeded(producerName, backlogQuotaType)) {
-                        IllegalStateException illegalStateException = new IllegalStateException(
-                                "Cannot create producer on topic with backlog quota exceeded");
-                        BacklogQuota.RetentionPolicy retentionPolicy = topic
-                                .getBacklogQuota(backlogQuotaType).getPolicy();
-                        if (retentionPolicy == BacklogQuota.RetentionPolicy.producer_request_hold) {
-                            commandSender.sendErrorResponse(requestId,
-                                    ServerError.ProducerBlockedQuotaExceededError,
-                                    illegalStateException.getMessage());
-                        } else if (retentionPolicy == BacklogQuota.RetentionPolicy.producer_exception) {
-                            commandSender.sendErrorResponse(requestId,
-                                    ServerError.ProducerBlockedQuotaExceededException,
-                                    illegalStateException.getMessage());
-                        }
-                        producerFuture.completeExceptionally(illegalStateException);
-                        producers.remove(producerId, producerFuture);
-                        return;
+                CompletableFuture<Void> backlogQuotaCheckFuture = CompletableFuture.allOf(
+                        topic.checkBacklogQuotaExceeded(producerName, BacklogQuotaType.destination_storage),
+                        topic.checkBacklogQuotaExceeded(producerName, BacklogQuotaType.message_age));
+                backlogQuotaCheckFuture.exceptionally(throwable -> {
+                    //throwable should be CompletionException holding TopicBacklogQuotaExceededException
+                    BrokerServiceException.TopicBacklogQuotaExceededException exception =

Review comment:
       What happens if there is something else?
   A ClassCastException or a NPE.
   This will break the system 
   
   I believe we should handle the case in which this expectation is not met

##########
File path: pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java
##########
@@ -1189,65 +1190,70 @@ protected void handleProducer(final CommandProducer cmdProducer) {
             service.getOrCreateTopic(topicName.toString()).thenAccept((Topic topic) -> {
                 // Before creating producer, check if backlog quota exceeded
                 // on topic for size based limit and time based limit
-                for (BacklogQuota.BacklogQuotaType backlogQuotaType : BacklogQuota.BacklogQuotaType.values()) {
-                    if (topic.isBacklogQuotaExceeded(producerName, backlogQuotaType)) {
-                        IllegalStateException illegalStateException = new IllegalStateException(
-                                "Cannot create producer on topic with backlog quota exceeded");
-                        BacklogQuota.RetentionPolicy retentionPolicy = topic
-                                .getBacklogQuota(backlogQuotaType).getPolicy();
-                        if (retentionPolicy == BacklogQuota.RetentionPolicy.producer_request_hold) {
-                            commandSender.sendErrorResponse(requestId,
-                                    ServerError.ProducerBlockedQuotaExceededError,
-                                    illegalStateException.getMessage());
-                        } else if (retentionPolicy == BacklogQuota.RetentionPolicy.producer_exception) {
-                            commandSender.sendErrorResponse(requestId,
-                                    ServerError.ProducerBlockedQuotaExceededException,
-                                    illegalStateException.getMessage());
-                        }
-                        producerFuture.completeExceptionally(illegalStateException);
-                        producers.remove(producerId, producerFuture);
-                        return;
+                CompletableFuture<Void> backlogQuotaCheckFuture = CompletableFuture.allOf(
+                        topic.checkBacklogQuotaExceeded(producerName, BacklogQuotaType.destination_storage),
+                        topic.checkBacklogQuotaExceeded(producerName, BacklogQuotaType.message_age));
+                backlogQuotaCheckFuture.exceptionally(throwable -> {

Review comment:
       Which thread will execute this code?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org