Browse Source

fix Zookeeper does not start, printed logs have no error messages, Api services can start but cannot be accessed

pull/3/MERGE
dailidong 4 years ago
parent
commit
2ee5df42b4
  1. 19
      dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/zk/CuratorZookeeperClient.java
  2. 3
      dolphinscheduler-service/src/test/java/org/apache/dolphinscheduler/service/zk/CuratorZookeeperClientTest.java

19
dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/zk/CuratorZookeeperClient.java

@ -24,6 +24,7 @@ import org.apache.curator.framework.state.ConnectionState;
import org.apache.curator.retry.ExponentialBackoffRetry;
import org.apache.zookeeper.ZooDefs;
import org.apache.zookeeper.data.ACL;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.springframework.beans.factory.InitializingBean;
@ -32,6 +33,7 @@ import org.springframework.stereotype.Component;
import java.nio.charset.StandardCharsets;
import java.util.List;
import java.util.concurrent.TimeUnit;
import static org.apache.dolphinscheduler.common.utils.Preconditions.checkNotNull;
@ -55,9 +57,10 @@ public class CuratorZookeeperClient implements InitializingBean {
}
private CuratorFramework buildClient() {
logger.info("zookeeper registry center init, server lists is: {}.", zookeeperConfig.getServerList());
logger.info("zookeeper registry center init, server lists is: [{}]", zookeeperConfig.getServerList());
CuratorFrameworkFactory.Builder builder = CuratorFrameworkFactory.builder().ensembleProvider(new DefaultEnsembleProvider(checkNotNull(zookeeperConfig.getServerList(),"zookeeper quorum can't be null")))
CuratorFrameworkFactory.Builder builder = CuratorFrameworkFactory.builder()
.ensembleProvider(new DefaultEnsembleProvider(checkNotNull(zookeeperConfig.getServerList(), "zookeeper quorum can't be null")))
.retryPolicy(new ExponentialBackoffRetry(zookeeperConfig.getBaseSleepTimeMs(), zookeeperConfig.getMaxRetries(), zookeeperConfig.getMaxSleepMs()));
//these has default value
@ -84,7 +87,9 @@ public class CuratorZookeeperClient implements InitializingBean {
zkClient = builder.build();
zkClient.start();
try {
zkClient.blockUntilConnected();
logger.info("trying to connect zookeeper server list:{}", zookeeperConfig.getServerList());
zkClient.blockUntilConnected(30, TimeUnit.SECONDS);
} catch (final Exception ex) {
throw new RuntimeException(ex);
}
@ -95,12 +100,14 @@ public class CuratorZookeeperClient implements InitializingBean {
checkNotNull(zkClient);
zkClient.getConnectionStateListenable().addListener((client, newState) -> {
if(newState == ConnectionState.LOST){
if (newState == ConnectionState.LOST) {
logger.error("connection lost from zookeeper");
} else if(newState == ConnectionState.RECONNECTED){
} else if (newState == ConnectionState.RECONNECTED) {
logger.info("reconnected to zookeeper");
} else if(newState == ConnectionState.SUSPENDED){
} else if (newState == ConnectionState.SUSPENDED) {
logger.warn("connection SUSPENDED to zookeeper");
} else if (newState == ConnectionState.CONNECTED) {
logger.info("connected to zookeeper server list:[{}]", zookeeperConfig.getServerList());
}
});
}

3
dolphinscheduler-service/src/test/java/org/apache/dolphinscheduler/service/zk/CuratorZookeeperClientTest.java

@ -59,9 +59,8 @@ public class CuratorZookeeperClientTest {
zookeeperConfig.setDsRoot("/dolphinscheduler");
zookeeperConfig.setMaxWaitTime(30000);
zookeeperClient.setZookeeperConfig(zookeeperConfig);
System.out.println("start");
zookeeperClient.afterPropertiesSet();
System.out.println("end");
Assert.assertNotNull(zookeeperClient.getZkClient());
}
}
Loading…
Cancel
Save