48
48
import java .util .concurrent .TimeUnit ;
49
49
import java .util .concurrent .atomic .AtomicBoolean ;
50
50
import java .util .concurrent .atomic .AtomicLong ;
51
+ import java .util .concurrent .atomic .AtomicReference ;
51
52
import java .util .function .BiFunction ;
52
53
import java .util .stream .Collectors ;
53
54
import javax .annotation .Nonnull ;
@@ -258,6 +259,37 @@ protected TopicStatsHelper initialValue() {
258
259
@ Getter
259
260
private final ExecutorService orderedExecutor ;
260
261
262
+ private volatile CloseFutures closeFutures ;
263
+
264
+ /***
265
+ * We use 2 futures to prevent a new closing if there is an in-progress deletion or closing. We make Pulsar return
266
+ * the in-progress one when it is called the second time.
267
+ *
268
+ * The topic closing will be called the below scenarios:
269
+ * 1. Calling "pulsar-admin topics unload". Relate to {@link CloseFutures#waitDisconnectClients}.
270
+ * 2. Namespace bundle unloading. The unloading topic triggered by unloading namespace bundles will not wait for
271
+ * clients disconnect. See {@link CloseFutures#notWaitDisconnectClients}.
272
+ *
273
+ * The two futures will be setting as the below rule:
274
+ * Event: Topic close.
275
+ * - If the first one closing is called by "close and not wait for clients disconnect":
276
+ * - {@link CloseFutures#waitDisconnectClients} will be initialized as "waiting for clients disconnect".
277
+ * - If the first one closing is called by "close and wait for clients disconnect", the two futures will be
278
+ * initialized as "waiting for clients disconnect".
279
+ * Event: Topic delete.
280
+ * the three futures will be initialized as "waiting for clients disconnect".
281
+ */
282
+ private class CloseFutures {
283
+ private final CompletableFuture <Void > notWaitDisconnectClients ;
284
+ private final CompletableFuture <Void > waitDisconnectClients ;
285
+
286
+ public CloseFutures (CompletableFuture <Void > waitDisconnectClients ,
287
+ CompletableFuture <Void > notWaitDisconnectClients ) {
288
+ this .waitDisconnectClients = waitDisconnectClients ;
289
+ this .notWaitDisconnectClients = notWaitDisconnectClients ;
290
+ }
291
+ }
292
+
261
293
private static class TopicStatsHelper {
262
294
public double averageMsgSize ;
263
295
public double aggMsgRateIn ;
@@ -1349,8 +1381,10 @@ private CompletableFuture<Void> delete(boolean failIfHasSubscriptions,
1349
1381
}
1350
1382
1351
1383
fenceTopicToCloseOrDelete (); // Avoid clients reconnections while deleting
1384
+ // Mark the progress of close to prevent close calling concurrently.
1385
+ this .closeFutures = new CloseFutures (new CompletableFuture (), new CompletableFuture ());
1352
1386
1353
- return getBrokerService ().getPulsar ().getPulsarResources ().getNamespaceResources ()
1387
+ CompletableFuture < Void > res = getBrokerService ().getPulsar ().getPulsarResources ().getNamespaceResources ()
1354
1388
.getPartitionedTopicResources ().runWithMarkDeleteAsync (TopicName .get (topic ), () -> {
1355
1389
CompletableFuture <Void > deleteFuture = new CompletableFuture <>();
1356
1390
@@ -1453,6 +1487,10 @@ public void deleteLedgerComplete(Object ctx) {
1453
1487
unfenceTopicToResume ();
1454
1488
}
1455
1489
});
1490
+
1491
+ FutureUtil .completeAfter (closeFutures .notWaitDisconnectClients , res );
1492
+ FutureUtil .completeAfter (closeFutures .waitDisconnectClients , res );
1493
+ return res ;
1456
1494
} finally {
1457
1495
lock .writeLock ().unlock ();
1458
1496
}
@@ -1463,6 +1501,11 @@ public CompletableFuture<Void> close() {
1463
1501
return close (false );
1464
1502
}
1465
1503
1504
+ private enum CloseTypes {
1505
+ notWaitDisconnectClients ,
1506
+ waitDisconnectClients ;
1507
+ }
1508
+
1466
1509
/**
1467
1510
* Close this topic - close all producers and subscriptions associated with this topic.
1468
1511
*
@@ -1471,19 +1514,32 @@ public CompletableFuture<Void> close() {
1471
1514
*/
1472
1515
@ Override
1473
1516
public CompletableFuture <Void > close (boolean closeWithoutWaitingClientDisconnect ) {
1474
- CompletableFuture <Void > closeFuture = new CompletableFuture <>();
1475
1517
1476
- lock .writeLock ().lock ();
1477
- try {
1518
+ CloseTypes closeType ;
1519
+ if (closeWithoutWaitingClientDisconnect ) {
1520
+ closeType = CloseTypes .notWaitDisconnectClients ;
1521
+ } else {
1478
1522
// closing managed-ledger waits until all producers/consumers/replicators get closed. Sometimes, broker
1479
1523
// forcefully wants to close managed-ledger without waiting all resources to be closed.
1480
- if (!isClosingOrDeleting || closeWithoutWaitingClientDisconnect ) {
1481
- fenceTopicToCloseOrDelete ();
1482
- } else {
1483
- log .warn ("[{}] Topic is already being closed or deleted" , topic );
1484
- closeFuture .completeExceptionally (new TopicFencedException ("Topic is already fenced" ));
1485
- return closeFuture ;
1524
+ closeType = CloseTypes .waitDisconnectClients ;
1525
+ }
1526
+
1527
+ lock .writeLock ().lock ();
1528
+ try {
1529
+ // Return in-progress future if exists.
1530
+ if (isClosingOrDeleting ) {
1531
+ switch (closeType ) {
1532
+ case notWaitDisconnectClients -> {
1533
+ return closeFutures .notWaitDisconnectClients ;
1534
+ }
1535
+ case waitDisconnectClients -> {
1536
+ return closeFutures .waitDisconnectClients ;
1537
+ }
1538
+ }
1486
1539
}
1540
+ // No in-progress closing.
1541
+ fenceTopicToCloseOrDelete ();
1542
+ this .closeFutures = new CloseFutures (new CompletableFuture (), new CompletableFuture ());
1487
1543
} finally {
1488
1544
lock .writeLock ().unlock ();
1489
1545
}
@@ -1513,11 +1569,22 @@ public CompletableFuture<Void> close(boolean closeWithoutWaitingClientDisconnect
1513
1569
});
1514
1570
}
1515
1571
1516
- CompletableFuture <Void > clientCloseFuture = closeWithoutWaitingClientDisconnect
1517
- ? CompletableFuture .completedFuture (null )
1518
- : FutureUtil .waitForAll (futures );
1572
+ CompletableFuture <Void > disconnectClientsInCurrentCall = null ;
1573
+ AtomicReference <CompletableFuture <Void >> disconnectClientsToCache = new AtomicReference <>();
1574
+ switch (closeType ) {
1575
+ case notWaitDisconnectClients -> {
1576
+ disconnectClientsInCurrentCall = CompletableFuture .completedFuture (null );
1577
+ disconnectClientsToCache .set (FutureUtil .waitForAll (futures ));
1578
+ break ;
1579
+ }
1580
+ case waitDisconnectClients -> {
1581
+ disconnectClientsInCurrentCall = FutureUtil .waitForAll (futures );
1582
+ disconnectClientsToCache .set (disconnectClientsInCurrentCall );
1583
+ }
1584
+ }
1585
+ CompletableFuture <Void > closeFuture = new CompletableFuture <>();
1519
1586
1520
- clientCloseFuture . thenRun ( () -> {
1587
+ Runnable closeLedgerAfterCloseClients = () -> {
1521
1588
// After having disconnected all producers/consumers, close the managed ledger
1522
1589
ledger .asyncClose (new CloseCallback () {
1523
1590
@ Override
@@ -1532,13 +1599,32 @@ public void closeFailed(ManagedLedgerException exception, Object ctx) {
1532
1599
disposeTopic (closeFuture );
1533
1600
}
1534
1601
}, null );
1535
- }).exceptionally (exception -> {
1602
+ };
1603
+ disconnectClientsInCurrentCall .thenRun (closeLedgerAfterCloseClients ).exceptionally (exception -> {
1536
1604
log .error ("[{}] Error closing topic" , topic , exception );
1537
1605
unfenceTopicToResume ();
1538
1606
closeFuture .completeExceptionally (exception );
1539
1607
return null ;
1540
1608
});
1541
1609
1610
+ switch (closeType ) {
1611
+ case notWaitDisconnectClients -> {
1612
+ FutureUtil .completeAfter (closeFutures .notWaitDisconnectClients , closeFuture );
1613
+ FutureUtil .completeAfterAll (closeFutures .waitDisconnectClients ,
1614
+ closeFuture .thenCompose (ignore -> disconnectClientsToCache .get ().exceptionally (ex -> {
1615
+ // Since the managed ledger has been closed, eat the error of clients disconnection.
1616
+ log .error ("[{}] Closed managed ledger, but disconnect clients failed,"
1617
+ + " this topic will be marked closed" , topic , ex );
1618
+ return null ;
1619
+ })));
1620
+ break ;
1621
+ }
1622
+ case waitDisconnectClients -> {
1623
+ FutureUtil .completeAfter (closeFutures .notWaitDisconnectClients , closeFuture );
1624
+ FutureUtil .completeAfterAll (closeFutures .waitDisconnectClients , closeFuture );
1625
+ }
1626
+ }
1627
+
1542
1628
return closeFuture ;
1543
1629
}
1544
1630
@@ -1824,10 +1910,10 @@ protected CompletableFuture<Void> addReplicationCluster(String remoteCluster, Ma
1824
1910
lock .readLock ().lock ();
1825
1911
try {
1826
1912
if (isClosingOrDeleting ) {
1827
- // Whether is "transferring" or not, do not create new replicator.
1913
+ // Do not create new replicator.
1828
1914
log .info ("[{}] Skip to create replicator because this topic is closing."
1829
- + " remote cluster: {}. State of transferring : {} " ,
1830
- topic , remoteCluster , transferring );
1915
+ + " remote cluster: {}." ,
1916
+ topic , remoteCluster );
1831
1917
return ;
1832
1918
}
1833
1919
Replicator replicator = replicators .computeIfAbsent (remoteCluster , r -> {
0 commit comments