@@ -2304,28 +2304,56 @@ public void getReplicationClusters(@Suspended final AsyncResponse asyncResponse,
2304
2304
@ PathParam ("tenant" ) String tenant ,
2305
2305
@ PathParam ("namespace" ) String namespace ,
2306
2306
@ PathParam ("topic" ) @ Encoded String encodedTopic ,
2307
+ @ QueryParam ("isGlobal" ) @ DefaultValue ("false" ) boolean isGlobal ,
2307
2308
@ QueryParam ("applied" ) @ DefaultValue ("false" ) boolean applied ,
2308
2309
@ ApiParam (value = "Whether leader broker redirected this call to this broker. "
2309
2310
+ "For internal use." )
2310
2311
@ QueryParam ("authoritative" ) @ DefaultValue ("false" ) boolean authoritative ) {
2311
2312
validateTopicName (tenant , namespace , encodedTopic );
2312
2313
validateTopicPolicyOperationAsync (topicName , PolicyName .REPLICATION , PolicyOperation .READ )
2313
2314
.thenCompose (__ -> preValidation (authoritative ))
2314
- .thenCompose (__ -> getTopicPoliciesAsyncWithRetry (topicName ))
2315
- .thenAccept (op -> {
2316
- asyncResponse .resume (op .map (TopicPolicies ::getReplicationClustersSet ).orElseGet (() -> {
2317
- if (applied ) {
2318
- return getNamespacePolicies (namespaceName ).replication_clusters ;
2315
+ .thenCompose (__ -> {
2316
+ if (applied ) {
2317
+ return getAppliedReplicatedClusters ();
2318
+ }
2319
+ return getTopicPoliciesAsyncWithRetry (topicName , isGlobal ).thenApply (policy -> {
2320
+ if (policy != null && policy .isPresent ()
2321
+ && CollectionUtils .isNotEmpty (policy .get ().getReplicationClustersSet ())) {
2322
+ return policy .get ().getReplicationClustersSet ();
2319
2323
}
2320
2324
return null ;
2321
- })) ;
2325
+ });
2322
2326
})
2327
+ .thenAccept (asyncResponse ::resume )
2323
2328
.exceptionally (ex -> {
2324
2329
handleTopicPolicyException ("getReplicationClusters" , ex , asyncResponse );
2325
2330
return null ;
2326
2331
});
2327
2332
}
2328
2333
2334
+ private CompletableFuture <Set <String >> getAppliedReplicatedClusters () {
2335
+ return getTopicPoliciesAsyncWithRetry (topicName , false )
2336
+ .thenCompose (localPolicy -> {
2337
+ if (localPolicy != null && localPolicy .isPresent ()
2338
+ && CollectionUtils .isNotEmpty (localPolicy .get ().getReplicationClustersSet ())) {
2339
+ return CompletableFuture .completedFuture (localPolicy .get ().getReplicationClustersSet ());
2340
+ }
2341
+ return getTopicPoliciesAsyncWithRetry (topicName , true )
2342
+ .thenCompose (globalPolicy -> {
2343
+ if (globalPolicy != null && globalPolicy .isPresent ()
2344
+ && CollectionUtils .isNotEmpty (globalPolicy .get ().getReplicationClustersSet ())) {
2345
+ return CompletableFuture .completedFuture (globalPolicy .get ().getReplicationClustersSet ());
2346
+ }
2347
+ return getNamespacePoliciesAsync (namespaceName ).thenApply (v -> {
2348
+ if (v != null ) {
2349
+ return v .replication_clusters ;
2350
+ }
2351
+ return null ;
2352
+ });
2353
+ });
2354
+ });
2355
+ }
2356
+
2329
2357
@ POST
2330
2358
@ Path ("/{tenant}/{namespace}/{topic}/replication" )
2331
2359
@ ApiOperation (value = "Set the replication clusters for a topic." )
@@ -2341,32 +2369,14 @@ public void setReplicationClusters(
2341
2369
@ Suspended final AsyncResponse asyncResponse ,
2342
2370
@ PathParam ("tenant" ) String tenant , @ PathParam ("namespace" ) String namespace ,
2343
2371
@ PathParam ("topic" ) @ Encoded String encodedTopic ,
2372
+ @ QueryParam ("isGlobal" ) @ DefaultValue ("false" ) boolean isGlobal ,
2344
2373
@ ApiParam (value = "Whether leader broker redirected this call to this broker. For internal use." )
2345
2374
@ QueryParam ("authoritative" ) @ DefaultValue ("false" ) boolean authoritative ,
2346
2375
@ ApiParam (value = "List of replication clusters" , required = true ) List <String > clusterIds ) {
2347
2376
validateTopicName (tenant , namespace , encodedTopic );
2348
2377
validateTopicPolicyOperationAsync (topicName , PolicyName .REPLICATION , PolicyOperation .WRITE )
2349
- .thenCompose (__ -> preValidation (authoritative )).thenCompose (__ -> {
2350
- // Set a topic-level replicated clusters that do not contain local cluster is not meaningful, except
2351
- // the following scenario: User has two clusters, which enabled Geo-Replication through a global
2352
- // metadata store, the resources named partitioned topic metadata and the resource namespace-level
2353
- // "replicated clusters" are shared between multi clusters. Pulsar can hardly delete a specify
2354
- // partitioned topic. To support this use case, the following steps can implement it:
2355
- // 1. set a global topic-level replicated clusters that do not contain local cluster.
2356
- // 2. the local cluster will remove the subtopics automatically, and remove the schemas and local
2357
- // topic policies. Just leave the global topic policies there, which prevents the namespace level
2358
- // replicated clusters policy taking affect.
2359
- // TODO But the API "pulsar-admin topics set-replication-clusters" does not support global policy,
2360
- // to support this scenario, a PIP is needed.
2361
- boolean clustersDoesNotContainsLocal = CollectionUtils .isEmpty (clusterIds )
2362
- || !clusterIds .contains (pulsar ().getConfig ().getClusterName ());
2363
- if (clustersDoesNotContainsLocal ) {
2364
- return FutureUtil .failedFuture (new RestException (Response .Status .PRECONDITION_FAILED ,
2365
- "Can not remove local cluster from the topic-level replication clusters policy" ));
2366
- }
2367
- return CompletableFuture .completedFuture (null );
2368
- })
2369
- .thenCompose (__ -> internalSetReplicationClusters (clusterIds ))
2378
+ .thenCompose (__ -> preValidation (authoritative ))
2379
+ .thenCompose (__ -> internalSetReplicationClusters (clusterIds , isGlobal ))
2370
2380
.thenRun (() -> asyncResponse .resume (Response .noContent ().build ()))
2371
2381
.exceptionally (ex -> {
2372
2382
handleTopicPolicyException ("setReplicationClusters" , ex , asyncResponse );
@@ -2387,12 +2397,13 @@ public void setReplicationClusters(
2387
2397
public void removeReplicationClusters (@ Suspended final AsyncResponse asyncResponse ,
2388
2398
@ PathParam ("tenant" ) String tenant , @ PathParam ("namespace" ) String namespace ,
2389
2399
@ PathParam ("topic" ) @ Encoded String encodedTopic ,
2400
+ @ QueryParam ("isGlobal" ) @ DefaultValue ("false" ) boolean isGlobal ,
2390
2401
@ ApiParam (value = "Whether leader broker redirected this call to this broker. For internal use." )
2391
2402
@ QueryParam ("authoritative" ) @ DefaultValue ("false" ) boolean authoritative ) {
2392
2403
validateTopicName (tenant , namespace , encodedTopic );
2393
2404
validateTopicPolicyOperationAsync (topicName , PolicyName .REPLICATION , PolicyOperation .WRITE )
2394
2405
.thenCompose (__ -> preValidation (authoritative ))
2395
- .thenCompose (__ -> internalRemoveReplicationClusters ())
2406
+ .thenCompose (__ -> internalRemoveReplicationClusters (isGlobal ))
2396
2407
.thenRun (() -> asyncResponse .resume (Response .noContent ().build ()))
2397
2408
.exceptionally (ex -> {
2398
2409
handleTopicPolicyException ("removeReplicationClusters" , ex , asyncResponse );
0 commit comments