codelipenghui commented on a change in pull request #4963: Add more config for auto-topic-creation URL: https://github.com/apache/pulsar/pull/4963#discussion_r321186093
########## File path: pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/AdminResource.java ########## @@ -566,6 +574,78 @@ public PartitionedTopicMetadata deserialize(String key, byte[] content) throws E return metadataFuture; } + protected static PartitionedTopicMetadata fetchPartitionedTopicMetadataCheckAllowAutoCreation( + PulsarService pulsar, String path, TopicName topicName) { + try { + return fetchPartitionedTopicMetadataCheckAllowAutoCreationAsync(pulsar, path, topicName) + .get(); + } catch (Exception e) { + if (e.getCause() instanceof RestException) { + throw (RestException) e; + } + throw new RestException(e); + } + } + + protected static CompletableFuture<PartitionedTopicMetadata> fetchPartitionedTopicMetadataCheckAllowAutoCreationAsync( + PulsarService pulsar, String path, TopicName topicName) { + CompletableFuture<PartitionedTopicMetadata> metadataFuture = new CompletableFuture<>(); + try { + boolean allowAutoTopicCreation = pulsar.getConfiguration().isAllowAutoTopicCreation(); + String topicType = pulsar.getConfiguration().getAllowAutoTopicCreationType(); + boolean topicExist; + try { + topicExist = pulsar.getNamespaceService() + .getListOfTopics(topicName.getNamespaceObject(), PulsarApi.CommandGetTopicsOfNamespace.Mode.ALL) + .contains(topicName.toString()); + } catch (Exception e) { + log.warn("Unexpected error while getting list of topics. topic={}. Error: {}", + topicName, e.getMessage(), e); + throw new RestException(e); + } + fetchPartitionedTopicMetadataAsync(pulsar, path).whenComplete((metadata, ex) -> { + if (ex != null) { + metadataFuture.completeExceptionally(ex); + // If topic is already exist, creating partitioned topic is not allowed. + } else if (metadata.partitions == 0 && !topicExist && allowAutoTopicCreation && + TopicType.PARTITIONED.toString().equals(topicType)) { + createDefaultPartitionedTopicAsync(pulsar, path).whenComplete((defaultMetadata, e) -> { + if (e == null) { + metadataFuture.complete(defaultMetadata); + } else { + metadataFuture.completeExceptionally(e); + } + }); + } else { + metadataFuture.complete(metadata); + } + }); + } catch (Exception e) { + metadataFuture.completeExceptionally(e); + } + return metadataFuture; + } + + protected static CompletableFuture<PartitionedTopicMetadata> createDefaultPartitionedTopicAsync( + PulsarService pulsar, String path) { + int defaultNumPartitions = pulsar.getConfiguration().getDefaultNumPartitions(); + checkArgument(defaultNumPartitions > 0, "Default number of partitions should be more than 0"); + PartitionedTopicMetadata configMetadata = new PartitionedTopicMetadata(defaultNumPartitions); + CompletableFuture<PartitionedTopicMetadata> partitionedTopicFuture = new CompletableFuture<>(); + try { + byte[] content = jsonMapper().writeValueAsBytes(configMetadata); + ZkUtils.createFullPathOptimistic(pulsar.getGlobalZkCache().getZooKeeper(), path, content, + ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); + // we wait for the data to be synced in all quorums and the observers + Thread.sleep(PARTITIONED_TOPIC_WAIT_SYNC_TIME_MS); Review comment: don't understand too much why it is necessary, @sijie @jiazhai Please have take a look, is it worried that the client got the partition meta and the zookeeper cache has not been updated at broker? ---------------------------------------------------------------- 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. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services