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 2022/04/01 01:18:22 UTC

[GitHub] [pulsar] mattisonchao opened a new pull request #14920: [fix][broker] Reject auto create partitioned topic when topic name contains ``-partition-``

mattisonchao opened a new pull request #14920:
URL: https://github.com/apache/pulsar/pull/14920


   ### Motivation
   
   When the user set config as follow:
   
   ```json
   {
       allowAutoTopicCreation: true,
       allowAutoTopicCreationType: "partitioned",
       defaultNumPartitions: 1
   }
   ```
   they can create partitioned topic success with topic name ``persistent://prop/autoNs/failedcreate-partition-abcde``. 
   
   run get-partitioned-topic-metadata
   
   ```json
   {
     "partitions" : 1
   }
   ```
   run ``admin.topics().deletePartitionedTopic(topicName);``
   
   > Partitioned Topic Name should not contain '-partition-'
   
   run ``admin.topics().delete(topicName)``
   
   this operation can success delete the topic, but metadata still exists.
   
   ### Modifications
   
   *Describe the modifications you've done.*
   
   ### Verifying this change
   
   - [x] Make sure that the change passes the CI checks.
   
   Add TCP/HTTP client test for this change.
   
   ### Documentation
   
   - [x] `no-need-doc` 
     
   
   


-- 
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



[GitHub] [pulsar] mattisonchao closed pull request #14920: [fix][broker] Reject auto create partitioned topic when topic name contains ``-partition-``

Posted by GitBox <gi...@apache.org>.
mattisonchao closed pull request #14920:
URL: https://github.com/apache/pulsar/pull/14920


   


-- 
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



[GitHub] [pulsar] AnonHxy commented on a change in pull request #14920: [fix][broker] Reject auto create partitioned topic when topic name contains ``-partition-``

Posted by GitBox <gi...@apache.org>.
AnonHxy commented on a change in pull request #14920:
URL: https://github.com/apache/pulsar/pull/14920#discussion_r837076970



##########
File path: pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java
##########
@@ -2606,40 +2606,30 @@ private void createPendingLoadTopic() {
             return FutureUtil.failedFuture(new NamingException("namespace service is not ready"));
         }
         return pulsar.getNamespaceService().checkTopicExists(topicName)
-                .thenCompose(topicExists -> {
-                    return fetchPartitionedTopicMetadataAsync(topicName)
-                            .thenCompose(metadata -> {
-                                CompletableFuture<PartitionedTopicMetadata> future = new CompletableFuture<>();
-
-                                // There are a couple of potentially blocking calls, which we cannot make from the
-                                // MetadataStore callback thread.
-                                pulsar.getExecutor().execute(() -> {
-                                    // If topic is already exist, creating partitioned topic is not allowed.
-
-                                    if (metadata.partitions == 0
-                                            && !topicExists
-                                            && !topicName.isPartitioned()
-                                            && pulsar.getBrokerService().isAllowAutoTopicCreation(topicName)
-                                            && pulsar.getBrokerService().isDefaultTopicTypePartitioned(topicName)) {
-
-                                        pulsar.getBrokerService().createDefaultPartitionedTopicAsync(topicName)
-                                                .thenAccept(md -> future.complete(md))
-                                                .exceptionally(ex -> {
-                                                    future.completeExceptionally(ex);
-                                                    return null;
-                                                });
-                                    } else {
-                                        future.complete(metadata);
-                                    }
-                                });
-
-                                return future;
-                            });
-                });
+                .thenCompose(topicExists -> fetchPartitionedTopicMetadataAsync(topicName)
+                        .thenComposeAsync(metadata -> {
+                            // There are a couple of potentially blocking calls, which we cannot make from the
+                            // MetadataStore callback thread.
+                            // If topic is already exist, creating partitioned topic is not allowed.
+                            if (metadata.partitions == 0
+                                    && !topicExists
+                                    && !topicName.isPartitioned()
+                                    && pulsar.getBrokerService().isAllowAutoTopicCreation(topicName)
+                                    && pulsar.getBrokerService().isDefaultTopicTypePartitioned(topicName)) {
+                                return pulsar.getBrokerService().createDefaultPartitionedTopicAsync(topicName);
+                            } else {
+                                return CompletableFuture.completedFuture(null);

Review comment:
       Return `CompletableFuture.completedFuture(null)` will cause NPE by the methods who invoke it if  `metadata.partitions !=0`.  Maybe should return `CompletableFuture.completedFuture(metadata)` I think




-- 
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



[GitHub] [pulsar] mattisonchao commented on a change in pull request #14920: [fix][broker] Reject auto create partitioned topic when topic name contains ``-partition-``

Posted by GitBox <gi...@apache.org>.
mattisonchao commented on a change in pull request #14920:
URL: https://github.com/apache/pulsar/pull/14920#discussion_r839150299



##########
File path: pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentTopicTest.java
##########
@@ -271,4 +274,38 @@ public void testPersistentPartitionedTopicUnload() throws Exception {
             producer.close();
         }
     }
+
+    @Test
+    public void testAutoCreatePartitionedTopicThatNameIncludePartition() throws Exception {
+        final String topicName = "persistent://prop/autoNs/failedcreate-partition-abcde";
+        final String ns = "prop/autoNs";
+        admin.namespaces().createNamespace(ns);
+        pulsar.getConfiguration().setAllowAutoTopicCreationType("partitioned");
+        try {
+            @Cleanup
+            Producer<byte[]> producer = pulsarClient.newProducer().topic(topicName)
+                    .create();
+            Assert.fail("unexpected operation");
+        } catch (PulsarClientException ex) {
+            Assert.assertTrue(ex.getMessage()
+                    .contains("Invalid topic name"));
+        }
+        Assert.assertEquals(admin.topics().getList(ns).size(), 0);
+        URI tcpLookupUrl = new URI(pulsar.getBrokerServiceUrl());
+        PulsarClient client = PulsarClient.builder()
+                .serviceUrl(tcpLookupUrl.toString())
+                .build();
+        try {
+            @Cleanup
+            Producer<byte[]> producer = client.newProducer()
+                    .topic(topicName)
+                    .create();
+            Assert.fail("unexpected operation");
+        } catch (PulsarClientException ex) {
+            Assert.assertTrue(ex.getMessage()
+                    .contains("Invalid topic name"));
+        }
+        Assert.assertEquals(admin.topics().getList(ns).size(), 0);
+        pulsar.getConfiguration().setAllowAutoTopicCreationType("non-partitioned");
+    }

Review comment:
       line 309 just to reset the configuration.
   
   I'm not sure if clients should be denied creating a non-partitioned topic with the ``-partitioned-`` name, but I don't think it's a big deal, since a non-partitioned topic name containing ``-partitioned-`` doesn't seem to affect the logic.
   I'm still wondering about that.
   Thanks and let me know what you think.




-- 
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



[GitHub] [pulsar] Jason918 commented on pull request #14920: [fix][broker] Reject auto create partitioned topic when topic name contains ``-partition-``

Posted by GitBox <gi...@apache.org>.
Jason918 commented on pull request #14920:
URL: https://github.com/apache/pulsar/pull/14920#issuecomment-1085483757


   > I think it's better to narrow the limitation to ends with `-partition-\d+`
   
   @mattisonchao What do you think about this? 


-- 
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



[GitHub] [pulsar] RobertIndie commented on a change in pull request #14920: [fix][broker] Reject auto create partitioned topic when topic name contains ``-partition-``

Posted by GitBox <gi...@apache.org>.
RobertIndie commented on a change in pull request #14920:
URL: https://github.com/apache/pulsar/pull/14920#discussion_r839167512



##########
File path: pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentTopicTest.java
##########
@@ -271,4 +274,38 @@ public void testPersistentPartitionedTopicUnload() throws Exception {
             producer.close();
         }
     }
+
+    @Test
+    public void testAutoCreatePartitionedTopicThatNameIncludePartition() throws Exception {
+        final String topicName = "persistent://prop/autoNs/failedcreate-partition-abcde";
+        final String ns = "prop/autoNs";
+        admin.namespaces().createNamespace(ns);
+        pulsar.getConfiguration().setAllowAutoTopicCreationType("partitioned");
+        try {
+            @Cleanup
+            Producer<byte[]> producer = pulsarClient.newProducer().topic(topicName)
+                    .create();
+            Assert.fail("unexpected operation");
+        } catch (PulsarClientException ex) {
+            Assert.assertTrue(ex.getMessage()
+                    .contains("Invalid topic name"));
+        }
+        Assert.assertEquals(admin.topics().getList(ns).size(), 0);
+        URI tcpLookupUrl = new URI(pulsar.getBrokerServiceUrl());
+        PulsarClient client = PulsarClient.builder()
+                .serviceUrl(tcpLookupUrl.toString())
+                .build();
+        try {
+            @Cleanup
+            Producer<byte[]> producer = client.newProducer()
+                    .topic(topicName)
+                    .create();
+            Assert.fail("unexpected operation");
+        } catch (PulsarClientException ex) {
+            Assert.assertTrue(ex.getMessage()
+                    .contains("Invalid topic name"));
+        }
+        Assert.assertEquals(admin.topics().getList(ns).size(), 0);
+        pulsar.getConfiguration().setAllowAutoTopicCreationType("non-partitioned");
+    }

Review comment:
       Thanks for your explanation. Agree with you.




-- 
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



[GitHub] [pulsar] mattisonchao commented on a change in pull request #14920: [fix][broker] Reject auto create partitioned topic when topic name contains ``-partition-``

Posted by GitBox <gi...@apache.org>.
mattisonchao commented on a change in pull request #14920:
URL: https://github.com/apache/pulsar/pull/14920#discussion_r837080478



##########
File path: pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java
##########
@@ -2606,40 +2606,30 @@ private void createPendingLoadTopic() {
             return FutureUtil.failedFuture(new NamingException("namespace service is not ready"));
         }
         return pulsar.getNamespaceService().checkTopicExists(topicName)
-                .thenCompose(topicExists -> {
-                    return fetchPartitionedTopicMetadataAsync(topicName)
-                            .thenCompose(metadata -> {
-                                CompletableFuture<PartitionedTopicMetadata> future = new CompletableFuture<>();
-
-                                // There are a couple of potentially blocking calls, which we cannot make from the
-                                // MetadataStore callback thread.
-                                pulsar.getExecutor().execute(() -> {
-                                    // If topic is already exist, creating partitioned topic is not allowed.
-
-                                    if (metadata.partitions == 0
-                                            && !topicExists
-                                            && !topicName.isPartitioned()
-                                            && pulsar.getBrokerService().isAllowAutoTopicCreation(topicName)
-                                            && pulsar.getBrokerService().isDefaultTopicTypePartitioned(topicName)) {
-
-                                        pulsar.getBrokerService().createDefaultPartitionedTopicAsync(topicName)
-                                                .thenAccept(md -> future.complete(md))
-                                                .exceptionally(ex -> {
-                                                    future.completeExceptionally(ex);
-                                                    return null;
-                                                });
-                                    } else {
-                                        future.complete(metadata);
-                                    }
-                                });
-
-                                return future;
-                            });
-                });
+                .thenCompose(topicExists -> fetchPartitionedTopicMetadataAsync(topicName)
+                        .thenComposeAsync(metadata -> {
+                            // There are a couple of potentially blocking calls, which we cannot make from the
+                            // MetadataStore callback thread.
+                            // If topic is already exist, creating partitioned topic is not allowed.
+                            if (metadata.partitions == 0
+                                    && !topicExists
+                                    && !topicName.isPartitioned()
+                                    && pulsar.getBrokerService().isAllowAutoTopicCreation(topicName)
+                                    && pulsar.getBrokerService().isDefaultTopicTypePartitioned(topicName)) {
+                                return pulsar.getBrokerService().createDefaultPartitionedTopicAsync(topicName);
+                            } else {
+                                return CompletableFuture.completedFuture(null);

Review comment:
       Good catch! thanks.




-- 
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



[GitHub] [pulsar] mattisonchao commented on a change in pull request #14920: [fix][broker] Reject auto create partitioned topic when topic name contains ``-partition-``

Posted by GitBox <gi...@apache.org>.
mattisonchao commented on a change in pull request #14920:
URL: https://github.com/apache/pulsar/pull/14920#discussion_r837079883



##########
File path: pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java
##########
@@ -2606,40 +2606,30 @@ private void createPendingLoadTopic() {
             return FutureUtil.failedFuture(new NamingException("namespace service is not ready"));
         }
         return pulsar.getNamespaceService().checkTopicExists(topicName)
-                .thenCompose(topicExists -> {
-                    return fetchPartitionedTopicMetadataAsync(topicName)
-                            .thenCompose(metadata -> {
-                                CompletableFuture<PartitionedTopicMetadata> future = new CompletableFuture<>();
-
-                                // There are a couple of potentially blocking calls, which we cannot make from the
-                                // MetadataStore callback thread.
-                                pulsar.getExecutor().execute(() -> {
-                                    // If topic is already exist, creating partitioned topic is not allowed.
-
-                                    if (metadata.partitions == 0
-                                            && !topicExists
-                                            && !topicName.isPartitioned()
-                                            && pulsar.getBrokerService().isAllowAutoTopicCreation(topicName)
-                                            && pulsar.getBrokerService().isDefaultTopicTypePartitioned(topicName)) {
-
-                                        pulsar.getBrokerService().createDefaultPartitionedTopicAsync(topicName)
-                                                .thenAccept(md -> future.complete(md))
-                                                .exceptionally(ex -> {
-                                                    future.completeExceptionally(ex);
-                                                    return null;
-                                                });
-                                    } else {
-                                        future.complete(metadata);
-                                    }
-                                });
-
-                                return future;
-                            });
-                });
+                .thenCompose(topicExists -> fetchPartitionedTopicMetadataAsync(topicName)
+                        .thenComposeAsync(metadata -> {
+                            // There are a couple of potentially blocking calls, which we cannot make from the
+                            // MetadataStore callback thread.
+                            // If topic is already exist, creating partitioned topic is not allowed.
+                            if (metadata.partitions == 0
+                                    && !topicExists
+                                    && !topicName.isPartitioned()
+                                    && pulsar.getBrokerService().isAllowAutoTopicCreation(topicName)
+                                    && pulsar.getBrokerService().isDefaultTopicTypePartitioned(topicName)) {
+                                return pulsar.getBrokerService().createDefaultPartitionedTopicAsync(topicName);
+                            } else {
+                                return CompletableFuture.completedFuture(null);
+                            }
+                        }, pulsar.getExecutor())
+                );

Review comment:
       Yes, only for removing redundant future.




-- 
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



[GitHub] [pulsar] mattisonchao commented on a change in pull request #14920: [fix][broker] Reject auto create partitioned topic when topic name contains ``-partition-``

Posted by GitBox <gi...@apache.org>.
mattisonchao commented on a change in pull request #14920:
URL: https://github.com/apache/pulsar/pull/14920#discussion_r839148531



##########
File path: pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java
##########
@@ -2606,40 +2606,30 @@ private void createPendingLoadTopic() {
             return FutureUtil.failedFuture(new NamingException("namespace service is not ready"));
         }
         return pulsar.getNamespaceService().checkTopicExists(topicName)
-                .thenCompose(topicExists -> {
-                    return fetchPartitionedTopicMetadataAsync(topicName)
-                            .thenCompose(metadata -> {
-                                CompletableFuture<PartitionedTopicMetadata> future = new CompletableFuture<>();
-
-                                // There are a couple of potentially blocking calls, which we cannot make from the
-                                // MetadataStore callback thread.
-                                pulsar.getExecutor().execute(() -> {
-                                    // If topic is already exist, creating partitioned topic is not allowed.
-
-                                    if (metadata.partitions == 0
-                                            && !topicExists
-                                            && !topicName.isPartitioned()
-                                            && pulsar.getBrokerService().isAllowAutoTopicCreation(topicName)
-                                            && pulsar.getBrokerService().isDefaultTopicTypePartitioned(topicName)) {
-
-                                        pulsar.getBrokerService().createDefaultPartitionedTopicAsync(topicName)
-                                                .thenAccept(md -> future.complete(md))
-                                                .exceptionally(ex -> {
-                                                    future.completeExceptionally(ex);
-                                                    return null;
-                                                });
-                                    } else {
-                                        future.complete(metadata);
-                                    }
-                                });
-
-                                return future;
-                            });
-                });
+                .thenCompose(topicExists -> fetchPartitionedTopicMetadataAsync(topicName)
+                        .thenComposeAsync(metadata -> {
+                            // There are a couple of potentially blocking calls, which we cannot make from the
+                            // MetadataStore callback thread.
+                            // If topic is already exist, creating partitioned topic is not allowed.
+                            if (metadata.partitions == 0
+                                    && !topicExists
+                                    && !topicName.isPartitioned()
+                                    && pulsar.getBrokerService().isAllowAutoTopicCreation(topicName)
+                                    && pulsar.getBrokerService().isDefaultTopicTypePartitioned(topicName)) {
+                                return pulsar.getBrokerService().createDefaultPartitionedTopicAsync(topicName);
+                            } else {
+                                return CompletableFuture.completedFuture(metadata);
+                            }
+                        }, pulsar.getExecutor())
+                );
     }
 
     @SuppressWarnings("deprecation")
     private CompletableFuture<PartitionedTopicMetadata> createDefaultPartitionedTopicAsync(TopicName topicName) {
+        if (topicName.toString().contains(TopicName.PARTITIONED_TOPIC_SUFFIX)) {
+            return FutureUtil.failedFuture(new PulsarServerException.
+                    InvalidTopicNameException("Invalid topic name: " + topicName));
+        }

Review comment:
       fixed

##########
File path: pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java
##########
@@ -2606,40 +2606,30 @@ private void createPendingLoadTopic() {
             return FutureUtil.failedFuture(new NamingException("namespace service is not ready"));
         }
         return pulsar.getNamespaceService().checkTopicExists(topicName)
-                .thenCompose(topicExists -> {
-                    return fetchPartitionedTopicMetadataAsync(topicName)
-                            .thenCompose(metadata -> {
-                                CompletableFuture<PartitionedTopicMetadata> future = new CompletableFuture<>();
-
-                                // There are a couple of potentially blocking calls, which we cannot make from the
-                                // MetadataStore callback thread.
-                                pulsar.getExecutor().execute(() -> {
-                                    // If topic is already exist, creating partitioned topic is not allowed.
-
-                                    if (metadata.partitions == 0
-                                            && !topicExists
-                                            && !topicName.isPartitioned()
-                                            && pulsar.getBrokerService().isAllowAutoTopicCreation(topicName)
-                                            && pulsar.getBrokerService().isDefaultTopicTypePartitioned(topicName)) {
-
-                                        pulsar.getBrokerService().createDefaultPartitionedTopicAsync(topicName)
-                                                .thenAccept(md -> future.complete(md))
-                                                .exceptionally(ex -> {
-                                                    future.completeExceptionally(ex);
-                                                    return null;
-                                                });
-                                    } else {
-                                        future.complete(metadata);
-                                    }
-                                });
-
-                                return future;
-                            });
-                });
+                .thenCompose(topicExists -> fetchPartitionedTopicMetadataAsync(topicName)

Review comment:
       fixed




-- 
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



[GitHub] [pulsar] Technoboy- commented on a change in pull request #14920: [fix][broker] Reject auto create partitioned topic when topic name contains ``-partition-``

Posted by GitBox <gi...@apache.org>.
Technoboy- commented on a change in pull request #14920:
URL: https://github.com/apache/pulsar/pull/14920#discussion_r837064631



##########
File path: pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java
##########
@@ -2606,40 +2606,30 @@ private void createPendingLoadTopic() {
             return FutureUtil.failedFuture(new NamingException("namespace service is not ready"));
         }
         return pulsar.getNamespaceService().checkTopicExists(topicName)
-                .thenCompose(topicExists -> {
-                    return fetchPartitionedTopicMetadataAsync(topicName)
-                            .thenCompose(metadata -> {
-                                CompletableFuture<PartitionedTopicMetadata> future = new CompletableFuture<>();
-
-                                // There are a couple of potentially blocking calls, which we cannot make from the
-                                // MetadataStore callback thread.
-                                pulsar.getExecutor().execute(() -> {
-                                    // If topic is already exist, creating partitioned topic is not allowed.
-
-                                    if (metadata.partitions == 0
-                                            && !topicExists
-                                            && !topicName.isPartitioned()
-                                            && pulsar.getBrokerService().isAllowAutoTopicCreation(topicName)
-                                            && pulsar.getBrokerService().isDefaultTopicTypePartitioned(topicName)) {
-
-                                        pulsar.getBrokerService().createDefaultPartitionedTopicAsync(topicName)
-                                                .thenAccept(md -> future.complete(md))
-                                                .exceptionally(ex -> {
-                                                    future.completeExceptionally(ex);
-                                                    return null;
-                                                });
-                                    } else {
-                                        future.complete(metadata);
-                                    }
-                                });
-
-                                return future;
-                            });
-                });
+                .thenCompose(topicExists -> fetchPartitionedTopicMetadataAsync(topicName)
+                        .thenComposeAsync(metadata -> {
+                            // There are a couple of potentially blocking calls, which we cannot make from the
+                            // MetadataStore callback thread.
+                            // If topic is already exist, creating partitioned topic is not allowed.
+                            if (metadata.partitions == 0
+                                    && !topicExists
+                                    && !topicName.isPartitioned()
+                                    && pulsar.getBrokerService().isAllowAutoTopicCreation(topicName)
+                                    && pulsar.getBrokerService().isDefaultTopicTypePartitioned(topicName)) {
+                                return pulsar.getBrokerService().createDefaultPartitionedTopicAsync(topicName);
+                            } else {
+                                return CompletableFuture.completedFuture(null);
+                            }
+                        }, pulsar.getExecutor())
+                );

Review comment:
       Why change line 2609~2624? Only for removing reducant `future` ?




-- 
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



[GitHub] [pulsar] Technoboy- commented on a change in pull request #14920: [fix][broker] Reject auto create partitioned topic when topic name contains ``-partition-``

Posted by GitBox <gi...@apache.org>.
Technoboy- commented on a change in pull request #14920:
URL: https://github.com/apache/pulsar/pull/14920#discussion_r837064233



##########
File path: pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentTopicTest.java
##########
@@ -31,26 +31,24 @@
 import static org.testng.Assert.assertNull;
 
 import java.lang.reflect.Field;
+import java.net.URI;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.UUID;
 import java.util.concurrent.TimeUnit;
 
 import com.google.common.collect.Sets;
+import lombok.Cleanup;
 import org.apache.bookkeeper.client.LedgerHandle;
 import org.apache.bookkeeper.mledger.ManagedLedger;
 import org.apache.pulsar.broker.service.BrokerTestBase;
-import org.apache.pulsar.client.api.Consumer;
-import org.apache.pulsar.client.api.Message;
-import org.apache.pulsar.client.api.MessageRoutingMode;
-import org.apache.pulsar.client.api.Producer;
-import org.apache.pulsar.client.api.Schema;
-import org.apache.pulsar.client.api.SubscriptionType;
+import org.apache.pulsar.client.api.*;

Review comment:
       Should not use `*`.




-- 
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



[GitHub] [pulsar] Jason918 commented on a change in pull request #14920: [fix][broker] Reject auto create partitioned topic when topic name contains ``-partition-``

Posted by GitBox <gi...@apache.org>.
Jason918 commented on a change in pull request #14920:
URL: https://github.com/apache/pulsar/pull/14920#discussion_r838491109



##########
File path: pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java
##########
@@ -2606,40 +2606,30 @@ private void createPendingLoadTopic() {
             return FutureUtil.failedFuture(new NamingException("namespace service is not ready"));
         }
         return pulsar.getNamespaceService().checkTopicExists(topicName)
-                .thenCompose(topicExists -> {
-                    return fetchPartitionedTopicMetadataAsync(topicName)
-                            .thenCompose(metadata -> {
-                                CompletableFuture<PartitionedTopicMetadata> future = new CompletableFuture<>();
-
-                                // There are a couple of potentially blocking calls, which we cannot make from the
-                                // MetadataStore callback thread.
-                                pulsar.getExecutor().execute(() -> {
-                                    // If topic is already exist, creating partitioned topic is not allowed.
-
-                                    if (metadata.partitions == 0
-                                            && !topicExists
-                                            && !topicName.isPartitioned()
-                                            && pulsar.getBrokerService().isAllowAutoTopicCreation(topicName)
-                                            && pulsar.getBrokerService().isDefaultTopicTypePartitioned(topicName)) {
-
-                                        pulsar.getBrokerService().createDefaultPartitionedTopicAsync(topicName)
-                                                .thenAccept(md -> future.complete(md))
-                                                .exceptionally(ex -> {
-                                                    future.completeExceptionally(ex);
-                                                    return null;
-                                                });
-                                    } else {
-                                        future.complete(metadata);
-                                    }
-                                });
-
-                                return future;
-                            });
-                });
+                .thenCompose(topicExists -> fetchPartitionedTopicMetadataAsync(topicName)
+                        .thenComposeAsync(metadata -> {
+                            // There are a couple of potentially blocking calls, which we cannot make from the
+                            // MetadataStore callback thread.
+                            // If topic is already exist, creating partitioned topic is not allowed.
+                            if (metadata.partitions == 0
+                                    && !topicExists
+                                    && !topicName.isPartitioned()
+                                    && pulsar.getBrokerService().isAllowAutoTopicCreation(topicName)
+                                    && pulsar.getBrokerService().isDefaultTopicTypePartitioned(topicName)) {
+                                return pulsar.getBrokerService().createDefaultPartitionedTopicAsync(topicName);
+                            } else {
+                                return CompletableFuture.completedFuture(metadata);
+                            }
+                        }, pulsar.getExecutor())
+                );
     }
 
     @SuppressWarnings("deprecation")
     private CompletableFuture<PartitionedTopicMetadata> createDefaultPartitionedTopicAsync(TopicName topicName) {
+        if (topicName.toString().contains(TopicName.PARTITIONED_TOPIC_SUFFIX)) {
+            return FutureUtil.failedFuture(new PulsarServerException.
+                    InvalidTopicNameException("Invalid topic name: " + topicName));
+        }

Review comment:
       +1, We should give exact reason.




-- 
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



[GitHub] [pulsar] AnonHxy commented on pull request #14920: [fix][broker] Reject auto create partitioned topic when topic name contains ``-partition-``

Posted by GitBox <gi...@apache.org>.
AnonHxy commented on pull request #14920:
URL: https://github.com/apache/pulsar/pull/14920#issuecomment-1081441887


   LGTM


-- 
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



[GitHub] [pulsar] RobertIndie commented on a change in pull request #14920: [fix][broker] Reject auto create partitioned topic when topic name contains ``-partition-``

Posted by GitBox <gi...@apache.org>.
RobertIndie commented on a change in pull request #14920:
URL: https://github.com/apache/pulsar/pull/14920#discussion_r838189889



##########
File path: pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentTopicTest.java
##########
@@ -271,4 +274,38 @@ public void testPersistentPartitionedTopicUnload() throws Exception {
             producer.close();
         }
     }
+
+    @Test
+    public void testAutoCreatePartitionedTopicThatNameIncludePartition() throws Exception {
+        final String topicName = "persistent://prop/autoNs/failedcreate-partition-abcde";
+        final String ns = "prop/autoNs";
+        admin.namespaces().createNamespace(ns);
+        pulsar.getConfiguration().setAllowAutoTopicCreationType("partitioned");
+        try {
+            @Cleanup
+            Producer<byte[]> producer = pulsarClient.newProducer().topic(topicName)
+                    .create();
+            Assert.fail("unexpected operation");
+        } catch (PulsarClientException ex) {
+            Assert.assertTrue(ex.getMessage()
+                    .contains("Invalid topic name"));
+        }
+        Assert.assertEquals(admin.topics().getList(ns).size(), 0);
+        URI tcpLookupUrl = new URI(pulsar.getBrokerServiceUrl());
+        PulsarClient client = PulsarClient.builder()
+                .serviceUrl(tcpLookupUrl.toString())
+                .build();
+        try {
+            @Cleanup
+            Producer<byte[]> producer = client.newProducer()
+                    .topic(topicName)
+                    .create();
+            Assert.fail("unexpected operation");
+        } catch (PulsarClientException ex) {
+            Assert.assertTrue(ex.getMessage()
+                    .contains("Invalid topic name"));
+        }
+        Assert.assertEquals(admin.topics().getList(ns).size(), 0);
+        pulsar.getConfiguration().setAllowAutoTopicCreationType("non-partitioned");
+    }

Review comment:
       Seems that we also need to add the `non-partitioned` type test case.




-- 
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



[GitHub] [pulsar] Jason918 commented on a change in pull request #14920: [fix][broker] Reject auto create partitioned topic when topic name contains ``-partition-``

Posted by GitBox <gi...@apache.org>.
Jason918 commented on a change in pull request #14920:
URL: https://github.com/apache/pulsar/pull/14920#discussion_r840272225



##########
File path: pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java
##########
@@ -2640,6 +2640,11 @@ private void createPendingLoadTopic() {
 
     @SuppressWarnings("deprecation")
     private CompletableFuture<PartitionedTopicMetadata> createDefaultPartitionedTopicAsync(TopicName topicName) {
+        if (topicName.toString().contains(TopicName.PARTITIONED_TOPIC_SUFFIX)) {

Review comment:
       Can we create a namespace contains "-partition-" ?




-- 
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



[GitHub] [pulsar] mattisonchao commented on a change in pull request #14920: [fix][broker] Reject auto create partitioned topic when topic name contains ``-partition-``

Posted by GitBox <gi...@apache.org>.
mattisonchao commented on a change in pull request #14920:
URL: https://github.com/apache/pulsar/pull/14920#discussion_r837080024



##########
File path: pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentTopicTest.java
##########
@@ -31,26 +31,24 @@
 import static org.testng.Assert.assertNull;
 
 import java.lang.reflect.Field;
+import java.net.URI;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.UUID;
 import java.util.concurrent.TimeUnit;
 
 import com.google.common.collect.Sets;
+import lombok.Cleanup;
 import org.apache.bookkeeper.client.LedgerHandle;
 import org.apache.bookkeeper.mledger.ManagedLedger;
 import org.apache.pulsar.broker.service.BrokerTestBase;
-import org.apache.pulsar.client.api.Consumer;
-import org.apache.pulsar.client.api.Message;
-import org.apache.pulsar.client.api.MessageRoutingMode;
-import org.apache.pulsar.client.api.Producer;
-import org.apache.pulsar.client.api.Schema;
-import org.apache.pulsar.client.api.SubscriptionType;
+import org.apache.pulsar.client.api.*;

Review comment:
       Sure, thanks~




-- 
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



[GitHub] [pulsar] mattisonchao commented on pull request #14920: [fix][broker] Reject auto create partitioned topic when topic name contains ``-partition-``

Posted by GitBox <gi...@apache.org>.
mattisonchao commented on pull request #14920:
URL: https://github.com/apache/pulsar/pull/14920#issuecomment-1081433941


   @Technoboy-  @AnonHxy  PTAL :)


-- 
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



[GitHub] [pulsar] Technoboy- commented on a change in pull request #14920: [fix][broker] Reject auto create partitioned topic when topic name contains ``-partition-``

Posted by GitBox <gi...@apache.org>.
Technoboy- commented on a change in pull request #14920:
URL: https://github.com/apache/pulsar/pull/14920#discussion_r837283660



##########
File path: pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java
##########
@@ -2606,40 +2606,30 @@ private void createPendingLoadTopic() {
             return FutureUtil.failedFuture(new NamingException("namespace service is not ready"));
         }
         return pulsar.getNamespaceService().checkTopicExists(topicName)
-                .thenCompose(topicExists -> {
-                    return fetchPartitionedTopicMetadataAsync(topicName)
-                            .thenCompose(metadata -> {
-                                CompletableFuture<PartitionedTopicMetadata> future = new CompletableFuture<>();
-
-                                // There are a couple of potentially blocking calls, which we cannot make from the
-                                // MetadataStore callback thread.
-                                pulsar.getExecutor().execute(() -> {
-                                    // If topic is already exist, creating partitioned topic is not allowed.
-
-                                    if (metadata.partitions == 0
-                                            && !topicExists
-                                            && !topicName.isPartitioned()
-                                            && pulsar.getBrokerService().isAllowAutoTopicCreation(topicName)
-                                            && pulsar.getBrokerService().isDefaultTopicTypePartitioned(topicName)) {
-
-                                        pulsar.getBrokerService().createDefaultPartitionedTopicAsync(topicName)
-                                                .thenAccept(md -> future.complete(md))
-                                                .exceptionally(ex -> {
-                                                    future.completeExceptionally(ex);
-                                                    return null;
-                                                });
-                                    } else {
-                                        future.complete(metadata);
-                                    }
-                                });
-
-                                return future;
-                            });
-                });
+                .thenCompose(topicExists -> fetchPartitionedTopicMetadataAsync(topicName)
+                        .thenComposeAsync(metadata -> {
+                            // There are a couple of potentially blocking calls, which we cannot make from the
+                            // MetadataStore callback thread.
+                            // If topic is already exist, creating partitioned topic is not allowed.
+                            if (metadata.partitions == 0
+                                    && !topicExists
+                                    && !topicName.isPartitioned()
+                                    && pulsar.getBrokerService().isAllowAutoTopicCreation(topicName)
+                                    && pulsar.getBrokerService().isDefaultTopicTypePartitioned(topicName)) {
+                                return pulsar.getBrokerService().createDefaultPartitionedTopicAsync(topicName);
+                            } else {
+                                return CompletableFuture.completedFuture(metadata);
+                            }
+                        }, pulsar.getExecutor())
+                );
     }
 
     @SuppressWarnings("deprecation")
     private CompletableFuture<PartitionedTopicMetadata> createDefaultPartitionedTopicAsync(TopicName topicName) {
+        if (topicName.toString().contains(TopicName.PARTITIONED_TOPIC_SUFFIX)) {
+            return FutureUtil.failedFuture(new PulsarServerException.
+                    InvalidTopicNameException("Invalid topic name: " + topicName));
+        }

Review comment:
       The prompt message is not clear , like : `"Invalid topic name: " + topicName + ",  should not contain -partition-" `
   




-- 
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



[GitHub] [pulsar] Jason918 commented on a change in pull request #14920: [fix][broker] Reject auto create partitioned topic when topic name contains ``-partition-``

Posted by GitBox <gi...@apache.org>.
Jason918 commented on a change in pull request #14920:
URL: https://github.com/apache/pulsar/pull/14920#discussion_r838496385



##########
File path: pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java
##########
@@ -2606,40 +2606,30 @@ private void createPendingLoadTopic() {
             return FutureUtil.failedFuture(new NamingException("namespace service is not ready"));
         }
         return pulsar.getNamespaceService().checkTopicExists(topicName)
-                .thenCompose(topicExists -> {
-                    return fetchPartitionedTopicMetadataAsync(topicName)
-                            .thenCompose(metadata -> {
-                                CompletableFuture<PartitionedTopicMetadata> future = new CompletableFuture<>();
-
-                                // There are a couple of potentially blocking calls, which we cannot make from the
-                                // MetadataStore callback thread.
-                                pulsar.getExecutor().execute(() -> {
-                                    // If topic is already exist, creating partitioned topic is not allowed.
-
-                                    if (metadata.partitions == 0
-                                            && !topicExists
-                                            && !topicName.isPartitioned()
-                                            && pulsar.getBrokerService().isAllowAutoTopicCreation(topicName)
-                                            && pulsar.getBrokerService().isDefaultTopicTypePartitioned(topicName)) {
-
-                                        pulsar.getBrokerService().createDefaultPartitionedTopicAsync(topicName)
-                                                .thenAccept(md -> future.complete(md))
-                                                .exceptionally(ex -> {
-                                                    future.completeExceptionally(ex);
-                                                    return null;
-                                                });
-                                    } else {
-                                        future.complete(metadata);
-                                    }
-                                });
-
-                                return future;
-                            });
-                });
+                .thenCompose(topicExists -> fetchPartitionedTopicMetadataAsync(topicName)

Review comment:
       Seems not related to this PR?




-- 
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



[GitHub] [pulsar] mattisonchao commented on pull request #14920: [fix][broker] Reject auto create partitioned topic when topic name contains ``-partition-``

Posted by GitBox <gi...@apache.org>.
mattisonchao commented on pull request #14920:
URL: https://github.com/apache/pulsar/pull/14920#issuecomment-1085628368


   > > I think it's better to narrow the limitation to ends with `-partition-\d+`
   > 
   > @mattisonchao What do you think about this?
   
   @Jason918 
   I'm sorry for omitting this message, I think it's a great idea.


-- 
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