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/09/26 03:31:03 UTC

[GitHub] [pulsar] 315157973 commented on a change in pull request #12136: Support for setting geo-replication clusters on topic level

315157973 commented on a change in pull request #12136:
URL: https://github.com/apache/pulsar/pull/12136#discussion_r716132723



##########
File path: pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java
##########
@@ -1334,33 +1335,37 @@ public void closeFailed(ManagedLedgerException exception, Object ctx) {
             log.debug("[{}] Checking replication status", name);
         }
 
-        CompletableFuture<Policies> policiesFuture = brokerService.pulsar().getPulsarResources()
-                .getNamespaceResources()
-                .getPoliciesAsync(TopicName.get(topic).getNamespaceObject())
-                .thenCompose(optPolicies -> {
-                            if (!optPolicies.isPresent()) {
-                                return FutureUtil.failedFuture(
-                                        new ServerMetadataException(
-                                                new MetadataStoreException.NotFoundException()));
-                            }
+        CompletableFuture<List<String>> replicationClustersFuture = brokerService.pulsar()
+                .getTopicPoliciesService()
+                .getTopicPoliciesAsyncWithRetry(name, null, brokerService.pulsar().getExecutor())
+                .thenCompose(topicPolicies -> {
+                    if (!topicPolicies.isPresent()
+                            || CollectionUtils.isEmpty(topicPolicies.get().getReplicationClusters())) {

Review comment:
       We need to define the state:
   null means no setting
   The list is Empty, which means that the value is not set, that is, the copy of this Topic under Namespace is disabled
   
   If empty also means that there is no setting, subsequent Topic-level Policies cannot be disabled
   

##########
File path: pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdTopics.java
##########
@@ -226,6 +226,10 @@ public CmdTopics(Supplier<PulsarAdmin> admin) {
 
         jcommander.addCommand("set-replicated-subscription-status", new SetReplicatedSubscriptionStatus());
 
+        jcommander.addCommand("get-replication-clusters", new GetReplicationClusters());

Review comment:
       Please also add unit test in `PulsarAdminToolTest`

##########
File path: pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java
##########
@@ -2708,6 +2709,60 @@ protected PersistentOfflineTopicStats internalGetBacklog(boolean authoritative)
             });
     }
 
+    protected CompletableFuture<Void> internalSetReplicationClusters(List<String> clusterIds) {
+        validateTopicPolicyOperation(topicName, PolicyName.REPLICATION, PolicyOperation.WRITE);
+        validatePoliciesReadOnlyAccess();
+
+        Set<String> replicationClusters = Sets.newHashSet(clusterIds);
+        if (replicationClusters.contains("global")) {
+            throw new RestException(Status.PRECONDITION_FAILED,
+                    "Cannot specify global in the list of replication clusters");
+        }
+        Set<String> clusters = clusters();
+        for (String clusterId : replicationClusters) {
+            if (!clusters.contains(clusterId)) {
+                throw new RestException(Status.FORBIDDEN, "Invalid cluster id: " + clusterId);
+            }
+            validatePeerClusterConflict(clusterId, replicationClusters);
+            validateClusterForTenant(namespaceName.getTenant(), clusterId);
+        }
+
+        return getTopicPoliciesAsyncWithRetry(topicName).thenCompose(op -> {
+                    TopicPolicies topicPolicies = op.orElseGet(TopicPolicies::new);
+                    topicPolicies.setReplicationClusters(Lists.newArrayList(replicationClusters));
+                    return pulsar().getTopicPoliciesService().updateTopicPoliciesAsync(topicName, topicPolicies)
+                            .thenRun(() -> {
+                                log.info("[{}] Successfully set replication clusters for namespace={}, "
+                                                + "topic={}, clusters={}",
+                                        clientAppId(),
+                                        namespaceName,
+                                        topicName.getLocalName(),
+                                        topicPolicies.getReplicationClusters());
+                            });
+                }
+        );
+    }
+
+    protected CompletableFuture<Void> internalRemoveReplicationClusters() {
+        validateTopicPolicyOperation(topicName, PolicyName.REPLICATION, PolicyOperation.WRITE);
+        validatePoliciesReadOnlyAccess();
+
+        return getTopicPoliciesAsyncWithRetry(topicName).thenCompose(op -> {
+                    TopicPolicies topicPolicies = op.orElseGet(TopicPolicies::new);
+                    topicPolicies.setReplicationClusters(Collections.emptyList());

Review comment:
       emptyList is a static variable. I think we can use list.clear or `new ArrayList()`

##########
File path: pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicPoliciesTest.java
##########
@@ -2641,4 +2642,23 @@ public void testDoNotCreateSystemTopicForHeartbeatNamespace() {
         });
     }
 
+    @Test(timeOut = 30000)

Review comment:
       Please add the following unit tests:
   1)The namespace and topic level policies are set at the same time, whether the priority is correct
   
   2)Disabling topic-level policies




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