Update exception handling in constructing bookkeeper client
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/BookKeeperClient.java b/distributedlog-core/src/main/java/org/apache/distributedlog/BookKeeperClient.java
index d1c1d6f..33baa9d 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/BookKeeperClient.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/BookKeeperClient.java
@@ -41,10 +41,8 @@
import org.apache.distributedlog.common.concurrent.FutureUtils;
import org.apache.distributedlog.exceptions.AlreadyClosedException;
import org.apache.distributedlog.exceptions.DLInterruptedException;
-import org.apache.distributedlog.exceptions.ZKException;
import org.apache.distributedlog.net.NetUtils;
import org.apache.distributedlog.util.ConfUtils;
-import org.apache.zookeeper.KeeperException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -85,7 +83,7 @@
String ledgersPath,
EventLoopGroup eventLoopGroup,
StatsLogger statsLogger, HashedWheelTimer requestTimer)
- throws IOException, InterruptedException, KeeperException {
+ throws IOException, InterruptedException {
ClientConfiguration bkConfig = new ClientConfiguration();
bkConfig.setAddEntryTimeout(conf.getBKClientWriteTimeout());
bkConfig.setReadTimeout(conf.getBKClientReadTimeout());
@@ -109,14 +107,18 @@
final DNSToSwitchMapping dnsResolver =
NetUtils.getDNSResolver(dnsResolverCls, conf.getBkDNSResolverOverrides());
- this.bkc = BookKeeper.forConfig(bkConfig)
- .setZookeeper(zkc.get())
- .setEventLoopGroup(eventLoopGroup)
- .setStatsLogger(statsLogger)
- .dnsResolver(dnsResolver)
- .requestTimer(requestTimer)
- .featureProvider(featureProvider.orNull())
- .build();
+ try {
+ this.bkc = BookKeeper.forConfig(bkConfig)
+ .setZookeeper(zkc.get())
+ .setEventLoopGroup(eventLoopGroup)
+ .setStatsLogger(statsLogger)
+ .dnsResolver(dnsResolver)
+ .requestTimer(requestTimer)
+ .featureProvider(featureProvider.orNull())
+ .build();
+ } catch (BKException bke) {
+ throw new IOException(bke);
+ }
}
BookKeeperClient(DistributedLogConfiguration conf,
@@ -167,8 +169,6 @@
commonInitialization(conf, ledgersPath, eventLoopGroup, statsLogger, requestTimer);
} catch (InterruptedException e) {
throw new DLInterruptedException("Interrupted on creating bookkeeper client " + name + " : ", e);
- } catch (KeeperException e) {
- throw new ZKException("Error on creating bookkeeper client " + name + " : ", e);
}
if (ownZK) {