mirror of https://github.com/apache/druid.git
another effort on fixing the transient error for Broker/CoordinatorServerViewTest happend on Travis build
This commit is contained in:
parent
78215a2053
commit
f12cfa99fa
|
@ -20,6 +20,7 @@
|
|||
package io.druid.curator;
|
||||
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.google.api.client.repackaged.com.google.common.base.Throwables;
|
||||
import com.metamx.common.guava.CloseQuietly;
|
||||
import io.druid.client.DruidServer;
|
||||
import io.druid.server.initialization.ZkPathsConfig;
|
||||
|
@ -29,6 +30,7 @@ import org.apache.curator.retry.RetryOneTime;
|
|||
import org.apache.curator.test.TestingServer;
|
||||
import org.apache.curator.test.Timing;
|
||||
import org.apache.curator.utils.ZKPaths;
|
||||
import org.apache.zookeeper.KeeperException;
|
||||
|
||||
/**
|
||||
*/
|
||||
|
@ -54,35 +56,45 @@ public class CuratorTestBase
|
|||
}
|
||||
|
||||
protected void setupZNodeForServer(DruidServer server, ZkPathsConfig zkPathsConfig, ObjectMapper jsonMapper)
|
||||
throws Exception
|
||||
{
|
||||
final String announcementsPath = zkPathsConfig.getAnnouncementsPath();
|
||||
final String inventoryPath = zkPathsConfig.getLiveSegmentsPath();
|
||||
|
||||
final String zNodePathAnnounce = ZKPaths.makePath(announcementsPath, server.getHost());
|
||||
final String zNodePathSegment = ZKPaths.makePath(inventoryPath, server.getHost());
|
||||
|
||||
/*
|
||||
* Explicitly check whether the zNodes we are about to create exist or not,
|
||||
* if exist, delete them to make sure we have a clean state on zookeeper.
|
||||
* Address issue: https://github.com/druid-io/druid/issues/1512
|
||||
*/
|
||||
if (curator.checkExists().forPath(zNodePathAnnounce) != null) {
|
||||
curator.delete().guaranteed().forPath(zNodePathAnnounce);
|
||||
try {
|
||||
curator.create()
|
||||
.creatingParentsIfNeeded()
|
||||
.forPath(
|
||||
ZKPaths.makePath(announcementsPath, server.getHost()),
|
||||
jsonMapper.writeValueAsBytes(server.getMetadata())
|
||||
);
|
||||
curator.create()
|
||||
.creatingParentsIfNeeded()
|
||||
.forPath(ZKPaths.makePath(inventoryPath, server.getHost()));
|
||||
}
|
||||
if (curator.checkExists().forPath(zNodePathSegment) != null) {
|
||||
curator.delete().guaranteed().forPath(zNodePathSegment);
|
||||
catch (KeeperException.NodeExistsException e) {
|
||||
/*
|
||||
* For some reason, Travis build sometimes fails here because of
|
||||
* org.apache.zookeeper.KeeperException$NodeExistsException: KeeperErrorCode = NodeExists, though it should never
|
||||
* happen because zookeeper should be in a clean state for each run of tests.
|
||||
* Address issue: https://github.com/druid-io/druid/issues/1512
|
||||
*/
|
||||
try {
|
||||
curator.setData()
|
||||
.forPath(
|
||||
ZKPaths.makePath(announcementsPath, server.getHost()),
|
||||
jsonMapper.writeValueAsBytes(server.getMetadata())
|
||||
);
|
||||
curator.setData()
|
||||
.forPath(ZKPaths.makePath(inventoryPath, server.getHost()));
|
||||
}
|
||||
catch (Exception e1) {
|
||||
Throwables.propagate(e1);
|
||||
}
|
||||
}
|
||||
catch (Exception e) {
|
||||
Throwables.propagate(e);
|
||||
}
|
||||
|
||||
curator.create()
|
||||
.creatingParentsIfNeeded()
|
||||
.forPath(
|
||||
ZKPaths.makePath(announcementsPath, server.getHost()),
|
||||
jsonMapper.writeValueAsBytes(server.getMetadata())
|
||||
);
|
||||
curator.create()
|
||||
.creatingParentsIfNeeded()
|
||||
.forPath(ZKPaths.makePath(inventoryPath, server.getHost()));
|
||||
}
|
||||
|
||||
protected void tearDownServerAndCurator()
|
||||
|
|
Loading…
Reference in New Issue