Skip to content

Commit 579b512

Browse files
authored
Allow providing a chroot to ZK suppliers (apache#5281)
* Add support to ClientInfo (and implementations) for providing a chroot path to the ZooKeeper (ZooSession) supplier * Fix typo in two ZooSession client names to lookup the instanceId or instanceName from the other (the client names were the reverse of the operation being performed, creating a little confusion in the logs)
1 parent d08a924 commit 579b512

File tree

4 files changed

+19
-15
lines changed

4 files changed

+19
-15
lines changed

core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java

+3-2
Original file line numberDiff line numberDiff line change
@@ -224,7 +224,8 @@ public ClientContext(SingletonReservation reservation, ClientInfo info,
224224

225225
this.zooSession = memoize(() -> {
226226
var zk = info
227-
.getZooKeeperSupplier(getClass().getSimpleName() + "(" + info.getPrincipal() + ")").get();
227+
.getZooKeeperSupplier(getClass().getSimpleName() + "(" + info.getPrincipal() + ")", "")
228+
.get();
228229
zooKeeperOpened.set(true);
229230
return zk;
230231
});
@@ -1100,7 +1101,7 @@ public synchronized ZookeeperLockChecker getTServerLockChecker() {
11001101
// so, it can't rely on being able to continue to use the same client's ZooCache,
11011102
// because that client could be closed, and its ZooSession also closed
11021103
// this needs to be fixed; TODO https://github.com/apache/accumulo/issues/2301
1103-
var zk = info.getZooKeeperSupplier(ZookeeperLockChecker.class.getSimpleName()).get();
1104+
var zk = info.getZooKeeperSupplier(ZookeeperLockChecker.class.getSimpleName(), "").get();
11041105
this.zkLockChecker =
11051106
new ZookeeperLockChecker(new ZooCache(zk), getZooKeeperRoot() + Constants.ZTSERVERS);
11061107
}

core/src/main/java/org/apache/accumulo/core/clientImpl/ClientInfo.java

+1-1
Original file line numberDiff line numberDiff line change
@@ -50,7 +50,7 @@ public interface ClientInfo {
5050
/**
5151
* @return a Supplier for creating new ZooKeeper client instances based on the configuration
5252
*/
53-
Supplier<ZooSession> getZooKeeperSupplier(String clientName);
53+
Supplier<ZooSession> getZooKeeperSupplier(String clientName, String rootPath);
5454

5555
/**
5656
* @return Zookeeper connection information for Accumulo instance

core/src/main/java/org/apache/accumulo/core/clientImpl/ClientInfoImpl.java

+8-7
Original file line numberDiff line numberDiff line change
@@ -29,7 +29,7 @@
2929
import java.nio.file.Paths;
3030
import java.util.Optional;
3131
import java.util.Properties;
32-
import java.util.function.Function;
32+
import java.util.function.BiFunction;
3333
import java.util.function.Supplier;
3434

3535
import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
@@ -52,18 +52,19 @@ public class ClientInfoImpl implements ClientInfo {
5252
private final Supplier<AuthenticationToken> tokenSupplier;
5353
private final Supplier<Configuration> hadoopConf;
5454
private final Supplier<InstanceId> instanceId;
55-
private final Function<String,ZooSession> zooSessionForName;
55+
private final BiFunction<String,String,ZooSession> zooSessionForName;
5656

5757
public ClientInfoImpl(Properties properties, Optional<AuthenticationToken> tokenOpt) {
5858
this.properties = requireNonNull(properties);
5959
// convert the optional to a supplier to delay retrieval from the properties unless needed
6060
this.tokenSupplier = requireNonNull(tokenOpt).map(Suppliers::ofInstance)
6161
.orElse(memoize(() -> ClientProperty.getAuthenticationToken(properties)));
6262
this.hadoopConf = memoize(Configuration::new);
63-
this.zooSessionForName =
64-
name -> new ZooSession(name, getZooKeepers(), getZooKeepersSessionTimeOut(), null);
63+
this.zooSessionForName = (name, rootPath) -> new ZooSession(name, getZooKeepers() + rootPath,
64+
getZooKeepersSessionTimeOut(), null);
6565
this.instanceId = memoize(() -> {
66-
try (var zk = getZooKeeperSupplier(getClass().getSimpleName() + ".getInstanceName()").get()) {
66+
try (var zk =
67+
getZooKeeperSupplier(getClass().getSimpleName() + ".getInstanceId()", "").get()) {
6768
return ZooUtil.getInstanceId(zk, getInstanceName());
6869
}
6970
});
@@ -80,8 +81,8 @@ public InstanceId getInstanceId() {
8081
}
8182

8283
@Override
83-
public Supplier<ZooSession> getZooKeeperSupplier(String clientName) {
84-
return () -> zooSessionForName.apply(clientName);
84+
public Supplier<ZooSession> getZooKeeperSupplier(String clientName, String rootPath) {
85+
return () -> zooSessionForName.apply(requireNonNull(clientName), requireNonNull(rootPath));
8586
}
8687

8788
@Override

server/base/src/main/java/org/apache/accumulo/server/ServerInfo.java

+7-5
Original file line numberDiff line numberDiff line change
@@ -24,6 +24,7 @@
2424
import java.io.IOException;
2525
import java.io.UncheckedIOException;
2626
import java.util.Properties;
27+
import java.util.function.BiFunction;
2728
import java.util.function.Function;
2829
import java.util.function.Supplier;
2930
import java.util.function.ToIntFunction;
@@ -57,7 +58,8 @@ public class ServerInfo implements ClientInfo {
5758
static ServerInfo fromServerConfig(SiteConfiguration siteConfig) {
5859
final Function<ServerInfo,String> instanceNameFromZk = si -> {
5960
try (var zk =
60-
si.getZooKeeperSupplier(ServerInfo.class.getSimpleName() + ".getInstanceId()").get()) {
61+
si.getZooKeeperSupplier(ServerInfo.class.getSimpleName() + ".getInstanceName()", "")
62+
.get()) {
6163
return ZooUtil.getInstanceName(zk, si.getInstanceId());
6264
}
6365
};
@@ -109,7 +111,7 @@ static ServerInfo forTesting(SiteConfiguration siteConfig, String instanceName,
109111
private final Supplier<InstanceId> instanceId;
110112
private final Supplier<String> instanceName;
111113
private final Supplier<Credentials> credentials;
112-
private final Function<String,ZooSession> zooSessionForName;
114+
private final BiFunction<String,String,ZooSession> zooSessionForName;
113115

114116
// set up everything to be lazily loaded with memoized suppliers, so if nothing is used, the cost
115117
// is low; to support different scenarios, plug in the functionality to retrieve certain items
@@ -139,7 +141,7 @@ private ServerInfo(SiteConfiguration siteConfig, Function<ServerInfo,String> zkH
139141
this.credentials =
140142
memoize(() -> SystemCredentials.get(getInstanceId(), getSiteConfiguration()));
141143

142-
this.zooSessionForName = name -> new ZooSession(name, getZooKeepers(),
144+
this.zooSessionForName = (name, rootPath) -> new ZooSession(name, getZooKeepers() + rootPath,
143145
getZooKeepersSessionTimeOut(), getSiteConfiguration().get(Property.INSTANCE_SECRET));
144146

145147
// from here on, set up the suppliers based on what was passed in, to support different cases
@@ -163,8 +165,8 @@ public InstanceId getInstanceId() {
163165
}
164166

165167
@Override
166-
public Supplier<ZooSession> getZooKeeperSupplier(String clientName) {
167-
return () -> zooSessionForName.apply(clientName);
168+
public Supplier<ZooSession> getZooKeeperSupplier(String clientName, String rootPath) {
169+
return () -> zooSessionForName.apply(requireNonNull(clientName), requireNonNull(rootPath));
168170
}
169171

170172
@Override

0 commit comments

Comments
 (0)