mirror of https://github.com/apache/druid.git
Make the Announcer closing and unannouncing atomic.
This commit is contained in:
parent
c5e99bf6ec
commit
bd500fcf4b
|
@ -32,6 +32,8 @@ import io.druid.curator.ShutdownNowIgnoringExecutorService;
|
||||||
import io.druid.curator.cache.PathChildrenCacheFactory;
|
import io.druid.curator.cache.PathChildrenCacheFactory;
|
||||||
import io.druid.curator.cache.SimplePathChildrenCacheFactory;
|
import io.druid.curator.cache.SimplePathChildrenCacheFactory;
|
||||||
import org.apache.curator.framework.CuratorFramework;
|
import org.apache.curator.framework.CuratorFramework;
|
||||||
|
import org.apache.curator.framework.api.transaction.CuratorTransaction;
|
||||||
|
import org.apache.curator.framework.api.transaction.CuratorTransactionFinal;
|
||||||
import org.apache.curator.framework.recipes.cache.ChildData;
|
import org.apache.curator.framework.recipes.cache.ChildData;
|
||||||
import org.apache.curator.framework.recipes.cache.PathChildrenCache;
|
import org.apache.curator.framework.recipes.cache.PathChildrenCache;
|
||||||
import org.apache.curator.framework.recipes.cache.PathChildrenCacheEvent;
|
import org.apache.curator.framework.recipes.cache.PathChildrenCacheEvent;
|
||||||
|
@ -121,14 +123,23 @@ public class Announcer
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
if (!parentsIBuilt.isEmpty()) {
|
||||||
|
CuratorTransaction transaction = curator.inTransaction();
|
||||||
for (String parent : parentsIBuilt) {
|
for (String parent : parentsIBuilt) {
|
||||||
try {
|
try {
|
||||||
curator.delete().forPath(parent);
|
transaction = transaction.delete().forPath(parent).and();
|
||||||
}
|
}
|
||||||
catch (Exception e) {
|
catch (Exception e) {
|
||||||
log.info(e, "Unable to delete parent[%s], boooo.", parent);
|
log.info(e, "Unable to delete parent[%s], boooo.", parent);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
try {
|
||||||
|
((CuratorTransactionFinal) transaction).commit();
|
||||||
|
}
|
||||||
|
catch (Exception e) {
|
||||||
|
log.info(e, "Unable to commit transaction. Please feed the hamsters");
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -335,7 +346,7 @@ public class Announcer
|
||||||
}
|
}
|
||||||
|
|
||||||
try {
|
try {
|
||||||
curator.delete().guaranteed().forPath(path);
|
curator.inTransaction().delete().forPath(path).and().commit();
|
||||||
}
|
}
|
||||||
catch (KeeperException.NoNodeException e) {
|
catch (KeeperException.NoNodeException e) {
|
||||||
log.info("node[%s] didn't exist anyway...", path);
|
log.info("node[%s] didn't exist anyway...", path);
|
||||||
|
|
Loading…
Reference in New Issue