diff --git a/hbase-client/pom.xml b/hbase-client/pom.xml index f3e27bc82d7..f61c6d02e4d 100644 --- a/hbase-client/pom.xml +++ b/hbase-client/pom.xml @@ -203,6 +203,18 @@ mockito-all test + + org.apache.curator + curator-recipes + + + org.apache.curator + curator-framework + + + org.apache.curator + curator-client + diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java index 7a8fd9a5571..121a16bbaef 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java @@ -66,7 +66,7 @@ class AsyncConnectionImpl implements AsyncConnection { private final User user; - private final ClusterRegistry registry; + private final AsyncRegistry registry; private final String clusterId; diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterRegistry.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegistry.java similarity index 56% rename from hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterRegistry.java rename to hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegistry.java index c1918a7fdbd..731cf09b6db 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterRegistry.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegistry.java @@ -18,16 +18,28 @@ package org.apache.hadoop.hbase.client; import java.io.Closeable; +import java.util.concurrent.CompletableFuture; +import org.apache.hadoop.hbase.RegionLocations; +import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.classification.InterfaceAudience; /** - * Implementations hold cluster information such as this cluster's id. + * Implementations hold cluster information such as this cluster's id, location of hbase:meta, etc.. + * All stuffs that may be related to zookeeper at client side are placed here. + *

+ * Most methods are executed asynchronously except getClusterId. It will be executed synchronously + * and should be called only once when initialization. *

* Internal use only. */ @InterfaceAudience.Private -interface ClusterRegistry extends Closeable { +interface AsyncRegistry extends Closeable { + + /** + * Get the location of meta region. + */ + CompletableFuture getMetaRegionLocation(); /** * Should only be called once. @@ -36,6 +48,24 @@ interface ClusterRegistry extends Closeable { */ String getClusterId(); + /** + * Get the number of 'running' regionservers. + */ + CompletableFuture getCurrentNrHRS(); + + /** + * Get the address of HMaster. + */ + CompletableFuture getMasterAddress(); + + /** + * Get the info port of HMaster. + */ + CompletableFuture getMasterInfoPort(); + + /** + * Closes this instance and releases any system resources associated with it + */ @Override void close(); } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterRegistryFactory.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterRegistryFactory.java index a6b3e39f50e..48bfb18bf7c 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterRegistryFactory.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterRegistryFactory.java @@ -35,9 +35,9 @@ final class ClusterRegistryFactory { /** * @return The cluster registry implementation to use. */ - static ClusterRegistry getRegistry(Configuration conf) { - Class clazz = - conf.getClass(REGISTRY_IMPL_CONF_KEY, ZKClusterRegistry.class, ClusterRegistry.class); + static AsyncRegistry getRegistry(Configuration conf) { + Class clazz = + conf.getClass(REGISTRY_IMPL_CONF_KEY, ZKAsyncRegistry.class, AsyncRegistry.class); return ReflectionUtils.newInstance(clazz, conf); } } \ No newline at end of file diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZKAsyncRegistry.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZKAsyncRegistry.java new file mode 100644 index 00000000000..c76aa3ec0a1 --- /dev/null +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZKAsyncRegistry.java @@ -0,0 +1,252 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.client; + +import static org.apache.hadoop.hbase.HConstants.DEFAULT_ZK_SESSION_TIMEOUT; +import static org.apache.hadoop.hbase.HConstants.ZK_SESSION_TIMEOUT; +import static org.apache.hadoop.hbase.HRegionInfo.DEFAULT_REPLICA_ID; +import static org.apache.hadoop.hbase.HRegionInfo.FIRST_META_REGIONINFO; +import static org.apache.hadoop.hbase.client.RegionReplicaUtil.getRegionInfoForDefaultReplica; +import static org.apache.hadoop.hbase.client.RegionReplicaUtil.getRegionInfoForReplica; +import static org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil.lengthOfPBMagic; +import static org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper.removeMetaData; + +import java.io.IOException; +import java.util.concurrent.CompletableFuture; + +import org.apache.commons.lang.mutable.MutableInt; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.curator.framework.CuratorFramework; +import org.apache.curator.framework.CuratorFrameworkFactory; +import org.apache.curator.framework.api.BackgroundPathable; +import org.apache.curator.framework.api.CuratorEvent; +import org.apache.curator.retry.RetryNTimes; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.ClusterId; +import org.apache.hadoop.hbase.HRegionLocation; +import org.apache.hadoop.hbase.RegionLocations; +import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.master.RegionState; +import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos; +import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos; +import org.apache.hadoop.hbase.util.Pair; +import org.apache.hadoop.hbase.util.Threads; +import org.apache.hadoop.hbase.zookeeper.ZKConfig; +import org.apache.hadoop.hbase.zookeeper.ZNodePaths; +import org.apache.zookeeper.data.Stat; + +/** + * Cache the cluster registry data in memory and use zk watcher to update. The only exception is + * {@link #getClusterId()}, it will fetch the data from zk directly. + */ +@InterfaceAudience.Private +class ZKAsyncRegistry implements AsyncRegistry { + + private static final Log LOG = LogFactory.getLog(ZKAsyncRegistry.class); + + private final CuratorFramework zk; + + private final ZNodePaths znodePaths; + + ZKAsyncRegistry(Configuration conf) { + this.znodePaths = new ZNodePaths(conf); + int zkSessionTimeout = conf.getInt(ZK_SESSION_TIMEOUT, DEFAULT_ZK_SESSION_TIMEOUT); + int zkRetry = conf.getInt("zookeeper.recovery.retry", 3); + int zkRetryIntervalMs = conf.getInt("zookeeper.recovery.retry.intervalmill", 1000); + this.zk = CuratorFrameworkFactory.builder() + .connectString(ZKConfig.getZKQuorumServersString(conf)).sessionTimeoutMs(zkSessionTimeout) + .retryPolicy(new RetryNTimes(zkRetry, zkRetryIntervalMs)) + .threadFactory( + Threads.newDaemonThreadFactory(String.format("ZKClusterRegistry-0x%08x", hashCode()))) + .build(); + this.zk.start(); + } + + @Override + public String getClusterId() { + try { + byte[] data = zk.getData().forPath(znodePaths.clusterIdZNode); + if (data == null || data.length == 0) { + return null; + } + data = removeMetaData(data); + return ClusterId.parseFrom(data).toString(); + } catch (Exception e) { + LOG.warn("failed to get cluster id", e); + return null; + } + } + + @Override + public void close() { + zk.close(); + } + + private interface CuratorEventProcessor { + T process(CuratorEvent event) throws Exception; + } + + private static CompletableFuture exec(BackgroundPathable opBuilder, String path, + CuratorEventProcessor processor) { + CompletableFuture future = new CompletableFuture<>(); + try { + opBuilder.inBackground((client, event) -> { + try { + future.complete(processor.process(event)); + } catch (Exception e) { + future.completeExceptionally(e); + } + }).withUnhandledErrorListener((msg, e) -> future.completeExceptionally(e)).forPath(path); + } catch (Exception e) { + future.completeExceptionally(e); + } + return future; + } + + private static ZooKeeperProtos.MetaRegionServer getMetaProto(CuratorEvent event) + throws IOException { + byte[] data = event.getData(); + if (data == null || data.length == 0) { + return null; + } + data = removeMetaData(data); + int prefixLen = lengthOfPBMagic(); + return ZooKeeperProtos.MetaRegionServer.parser().parseFrom(data, prefixLen, + data.length - prefixLen); + } + + private static void tryComplete(MutableInt remaining, HRegionLocation[] locs, + CompletableFuture future) { + remaining.decrement(); + if (remaining.intValue() > 0) { + return; + } + future.complete(new RegionLocations(locs)); + } + + private Pair getStateAndServerName( + ZooKeeperProtos.MetaRegionServer proto) { + RegionState.State state; + if (proto.hasState()) { + state = RegionState.State.convert(proto.getState()); + } else { + state = RegionState.State.OPEN; + } + HBaseProtos.ServerName snProto = proto.getServer(); + return Pair.newPair(state, + ServerName.valueOf(snProto.getHostName(), snProto.getPort(), snProto.getStartCode())); + } + + @Override + public CompletableFuture getMetaRegionLocation() { + CompletableFuture future = new CompletableFuture<>(); + HRegionLocation[] locs = new HRegionLocation[znodePaths.metaReplicaZNodes.size()]; + MutableInt remaining = new MutableInt(locs.length); + znodePaths.metaReplicaZNodes.forEach((replicaId, path) -> { + if (replicaId == DEFAULT_REPLICA_ID) { + exec(zk.getData(), path, ZKAsyncRegistry::getMetaProto).whenComplete((proto, error) -> { + if (error != null) { + future.completeExceptionally(error); + return; + } + if (proto == null) { + future.completeExceptionally(new IOException("Meta znode is null")); + return; + } + Pair stateAndServerName = getStateAndServerName(proto); + if (stateAndServerName.getFirst() != RegionState.State.OPEN) { + future.completeExceptionally( + new IOException("Meta region is in state " + stateAndServerName.getFirst())); + return; + } + locs[DEFAULT_REPLICA_ID] = new HRegionLocation( + getRegionInfoForDefaultReplica(FIRST_META_REGIONINFO), + stateAndServerName.getSecond()); + tryComplete(remaining, locs, future); + }); + } else { + exec(zk.getData(), path, ZKAsyncRegistry::getMetaProto).whenComplete((proto, error) -> { + if (future.isDone()) { + return; + } + if (error != null) { + LOG.warn("Failed to fetch " + path, error); + locs[replicaId] = null; + } else if (proto == null) { + LOG.warn("Meta znode for replica " + replicaId + " is null"); + locs[replicaId] = null; + } else { + Pair stateAndServerName = getStateAndServerName(proto); + if (stateAndServerName.getFirst() != RegionState.State.OPEN) { + LOG.warn("Meta region for replica " + replicaId + " is in state " + + stateAndServerName.getFirst()); + locs[replicaId] = null; + } else { + locs[replicaId] = new HRegionLocation( + getRegionInfoForReplica(FIRST_META_REGIONINFO, replicaId), + stateAndServerName.getSecond()); + } + } + tryComplete(remaining, locs, future); + }); + } + }); + return future; + } + + private static int getCurrentNrHRS(CuratorEvent event) { + Stat stat = event.getStat(); + return stat != null ? stat.getNumChildren() : 0; + } + + @Override + public CompletableFuture getCurrentNrHRS() { + return exec(zk.checkExists(), znodePaths.rsZNode, ZKAsyncRegistry::getCurrentNrHRS); + } + + private static ZooKeeperProtos.Master getMasterProto(CuratorEvent event) throws IOException { + byte[] data = event.getData(); + if (data == null || data.length == 0) { + return null; + } + data = removeMetaData(data); + int prefixLen = lengthOfPBMagic(); + return ZooKeeperProtos.Master.parser().parseFrom(data, prefixLen, data.length - prefixLen); + } + + @Override + public CompletableFuture getMasterAddress() { + return exec(zk.getData(), znodePaths.masterAddressZNode, ZKAsyncRegistry::getMasterProto) + .thenApply(proto -> { + if (proto == null) { + return null; + } + HBaseProtos.ServerName snProto = proto.getMaster(); + return ServerName.valueOf(snProto.getHostName(), snProto.getPort(), + snProto.getStartCode()); + }); + } + + @Override + public CompletableFuture getMasterInfoPort() { + return exec(zk.getData(), znodePaths.masterAddressZNode, ZKAsyncRegistry::getMasterProto) + .thenApply(proto -> proto != null ? proto.getInfoPort() : 0); + } +} \ No newline at end of file diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZKClusterRegistry.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZKClusterRegistry.java deleted file mode 100644 index d385136e649..00000000000 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZKClusterRegistry.java +++ /dev/null @@ -1,78 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.hbase.client; - -import static org.apache.hadoop.hbase.HConstants.DEFAULT_ZK_SESSION_TIMEOUT; -import static org.apache.hadoop.hbase.HConstants.ZK_SESSION_TIMEOUT; - -import java.io.IOException; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.ClusterId; -import org.apache.hadoop.hbase.classification.InterfaceAudience; -import org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper; -import org.apache.hadoop.hbase.zookeeper.ZKConfig; -import org.apache.hadoop.hbase.zookeeper.ZNodePaths; - -/** - * Cache the cluster registry data in memory and use zk watcher to update. The only exception is - * {@link #getClusterId()}, it will fetch the data from zk directly. - */ -@InterfaceAudience.Private -class ZKClusterRegistry implements ClusterRegistry { - - private static final Log LOG = LogFactory.getLog(ZKClusterRegistry.class); - - private final RecoverableZooKeeper zk; - - private final ZNodePaths znodePaths; - - ZKClusterRegistry(Configuration conf) throws IOException { - this.znodePaths = new ZNodePaths(conf); - int zkSessionTimeout = conf.getInt(ZK_SESSION_TIMEOUT, DEFAULT_ZK_SESSION_TIMEOUT); - int zkRetry = conf.getInt("zookeeper.recovery.retry", 3); - int zkRetryIntervalMs = conf.getInt("zookeeper.recovery.retry.intervalmill", 1000); - this.zk = new RecoverableZooKeeper(ZKConfig.getZKQuorumServersString(conf), zkSessionTimeout, - null, zkRetry, zkRetryIntervalMs); - } - - @Override - public String getClusterId() { - try { - byte[] data = zk.getData(znodePaths.clusterIdZNode, false, null); - if (data == null || data.length == 0) { - return null; - } - return ClusterId.parseFrom(data).toString(); - } catch (Exception e) { - LOG.warn("failed to get cluster id", e); - return null; - } - } - - @Override - public void close() { - try { - zk.close(); - } catch (InterruptedException e) { - LOG.warn("close zookeeper failed", e); - } - } -} \ No newline at end of file diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/RecoverableZooKeeper.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/RecoverableZooKeeper.java index 371279e9c60..14532cfd839 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/RecoverableZooKeeper.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/RecoverableZooKeeper.java @@ -352,7 +352,7 @@ public class RecoverableZooKeeper { while (true) { try { byte[] revData = checkZk().getData(path, watcher, stat); - return this.removeMetaData(revData); + return removeMetaData(revData); } catch (KeeperException e) { switch (e.code()) { case CONNECTIONLOSS: @@ -384,7 +384,7 @@ public class RecoverableZooKeeper { while (true) { try { byte[] revData = checkZk().getData(path, watch, stat); - return this.removeMetaData(revData); + return removeMetaData(revData); } catch (KeeperException e) { switch (e.code()) { case CONNECTIONLOSS: @@ -707,7 +707,7 @@ public class RecoverableZooKeeper { return null; } - public byte[] removeMetaData(byte[] data) { + public static byte[] removeMetaData(byte[] data) { if(data == null || data.length == 0) { return data; } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableNoncedRetry.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableNoncedRetry.java index 8fc0f60adb0..840f8440ec3 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableNoncedRetry.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableNoncedRetry.java @@ -17,7 +17,8 @@ */ package org.apache.hadoop.hbase.client; -import static org.junit.Assert.*; +import static org.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertEquals; import java.io.IOException; import java.util.concurrent.ExecutionException; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestZKAsyncRegistry.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestZKAsyncRegistry.java new file mode 100644 index 00000000000..a94df61624e --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestZKAsyncRegistry.java @@ -0,0 +1,78 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.client; + +import static org.apache.hadoop.hbase.HConstants.META_REPLICAS_NUM; +import static org.junit.Assert.*; + +import java.io.IOException; +import java.util.concurrent.ExecutionException; +import java.util.stream.IntStream; + +import org.apache.commons.io.IOUtils; +import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.HRegionLocation; +import org.apache.hadoop.hbase.RegionLocations; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.testclassification.ClientTests; +import org.apache.hadoop.hbase.testclassification.MediumTests; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +@Category({ MediumTests.class, ClientTests.class }) +public class TestZKAsyncRegistry { + + private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); + + private static ZKAsyncRegistry REGISTRY; + + @BeforeClass + public static void setUp() throws Exception { + TEST_UTIL.getConfiguration().setInt(META_REPLICAS_NUM, 3); + TEST_UTIL.startMiniCluster(3); + TEST_UTIL.waitUntilAllRegionsAssigned(TableName.META_TABLE_NAME); + REGISTRY = new ZKAsyncRegistry(TEST_UTIL.getConfiguration()); + } + + @AfterClass + public static void tearDown() throws Exception { + IOUtils.closeQuietly(REGISTRY); + TEST_UTIL.shutdownMiniCluster(); + } + + @Test + public void test() throws InterruptedException, ExecutionException, IOException { + assertEquals(TEST_UTIL.getHBaseCluster().getClusterStatus().getClusterId(), + REGISTRY.getClusterId()); + assertEquals(TEST_UTIL.getHBaseCluster().getClusterStatus().getServersSize(), + REGISTRY.getCurrentNrHRS().get().intValue()); + assertEquals(TEST_UTIL.getHBaseCluster().getMaster().getServerName(), + REGISTRY.getMasterAddress().get()); + assertEquals(-1, REGISTRY.getMasterInfoPort().get().intValue()); + RegionLocations locs = REGISTRY.getMetaRegionLocation().get(); + assertEquals(3, locs.getRegionLocations().length); + IntStream.range(0, 3).forEach(i -> { + HRegionLocation loc = locs.getRegionLocation(i); + assertNotNull(loc); + assertTrue(loc.getRegionInfo().getTable().equals(TableName.META_TABLE_NAME)); + assertEquals(i, loc.getRegionInfo().getReplicaId()); + }); + } +} diff --git a/pom.xml b/pom.xml index ad77f058ac5..65b8adbd14b 100644 --- a/pom.xml +++ b/pom.xml @@ -1230,6 +1230,7 @@ 1.46 1.0.0-RC2 1.0.0 + 2.11.0 2.4 1.8 @@ -1786,39 +1787,60 @@ disruptor ${disruptor.version} - + net.spy spymemcached ${spy.version} true - - - org.bouncycastle - bcprov-jdk16 - ${bouncycastle.version} - test - - - org.apache.kerby - kerb-client - ${kerby.version} - - - org.apache.kerby - kerb-simplekdc - ${kerby.version} - - - org.apache.commons - commons-crypto - ${commons-crypto.version} - - - net.java.dev.jna - jna - - - + + + org.bouncycastle + bcprov-jdk16 + ${bouncycastle.version} + test + + + org.apache.kerby + kerb-client + ${kerby.version} + + + org.apache.kerby + kerb-simplekdc + ${kerby.version} + + + org.apache.commons + commons-crypto + ${commons-crypto.version} + + + net.java.dev.jna + jna + + + + + org.apache.curator + curator-recipes + ${curator.version} + + + org.apache.curator + curator-framework + ${curator.version} + + + org.apache.curator + curator-client + ${curator.version} + + + com.google.guava + guava + + +