mirror of https://github.com/apache/druid.git
also remove curator listeners on cleanup
This commit is contained in:
parent
69a9e04563
commit
00cfb32541
|
@ -153,7 +153,7 @@ public class ServerInventoryView implements ServerView, InventoryView
|
|||
|
||||
if (segment != null) {
|
||||
log.warn(
|
||||
"Not running callbacks for existing segment[%s] on server[%s]",
|
||||
"Not adding or running callbacks for existing segment[%s] on server[%s]",
|
||||
inventoryKey,
|
||||
container.getName()
|
||||
);
|
||||
|
@ -185,7 +185,7 @@ public class ServerInventoryView implements ServerView, InventoryView
|
|||
|
||||
if (segment == null) {
|
||||
log.warn(
|
||||
"Not running callbacks or cleanup for non-existing segment[%s] on server[%s]",
|
||||
"Not running cleanup or callbacks for non-existing segment[%s] on server[%s]",
|
||||
inventoryKey,
|
||||
container.getName()
|
||||
);
|
||||
|
|
|
@ -216,7 +216,7 @@ public class CuratorInventoryManager<ContainerClass, InventoryClass>
|
|||
|
||||
containers.put(containerKey, new ContainerHolder(container, inventoryCache));
|
||||
|
||||
log.info("Starting inventory cache for %s, inventoryPath %s", container, inventoryPath);
|
||||
log.info("Starting inventory cache for %s, inventoryPath %s", containerKey, inventoryPath);
|
||||
inventoryCache.start();
|
||||
strategy.newContainer(container);
|
||||
}
|
||||
|
@ -233,8 +233,8 @@ public class CuratorInventoryManager<ContainerClass, InventoryClass>
|
|||
|
||||
// This close() call actually calls shutdownNow() on the executor registered with the Cache object, it
|
||||
// better have its own executor or ignore shutdownNow() calls...
|
||||
log.info("Closing inventory cache for %s", containerKey);
|
||||
removed.getCache().clear();
|
||||
log.info("Closing inventory cache for %s. Also removing listeners.", containerKey);
|
||||
removed.getCache().getListenable().clear();
|
||||
removed.getCache().close();
|
||||
strategy.deadContainer(removed.getContainer());
|
||||
|
||||
|
|
Loading…
Reference in New Issue