Skip to content

Commit dd2d40f

Browse files
authored
Use util code to identify root ZooKeeper path (apache#5120)
Avoid direct use of `Constants.ZROOT + "/" + instanceId` and use the existing `ZooUtil.getRoot(instanceId)` that was made for this purpose instead wherever possible. If a ServerContext is available, use `context.getZooKeeperRoot()` instead. * Use ZooUtil.getRoot() or ServerContext.getZooKeeperRoot() * Use Constants where not currently being used * Remove redundant ZKSecurityTool.getInstancePath * Remove redundant Manager methods that passthrough to ServerContext * Update related tests * Fix use of EasyMock in modified tests: RootTabletLocatorTest and ZookeeperLockCheckerTest * Avoid hard-coded "/accumulo/" in hdfs paths in some ITs that were false-positive potential uses of Constants.ZROOT when I was looking for possibility of replacing literals with constants. For these false-positives, retrieve the actual path from the MiniAccumuloConfig's "instance.volumes" property value, rather than make assumptions about the layout of MiniAccumuloCluster's setup.
1 parent e0168bb commit dd2d40f

File tree

53 files changed

+306
-237
lines changed

Some content is hidden

Large Commits have some content hidden by default. Use the searchbox below for content that may be hidden.

53 files changed

+306
-237
lines changed

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

+1-2
Original file line numberDiff line numberDiff line change
@@ -510,8 +510,7 @@ public String getRootTabletLocation() {
510510
*/
511511
public List<String> getManagerLocations() {
512512
ensureOpen();
513-
var zLockManagerPath =
514-
ServiceLock.path(Constants.ZROOT + "/" + getInstanceID() + Constants.ZMANAGER_LOCK);
513+
var zLockManagerPath = ServiceLock.path(getZooKeeperRoot() + Constants.ZMANAGER_LOCK);
515514

516515
Timer timer = null;
517516

core/src/test/java/org/apache/accumulo/core/clientImpl/RootTabletLocatorTest.java

+23-6
Original file line numberDiff line numberDiff line change
@@ -20,37 +20,54 @@
2020

2121
import static org.easymock.EasyMock.createMock;
2222
import static org.easymock.EasyMock.expect;
23+
import static org.easymock.EasyMock.expectLastCall;
2324
import static org.easymock.EasyMock.replay;
25+
import static org.easymock.EasyMock.reset;
2426
import static org.easymock.EasyMock.verify;
2527

28+
import java.util.UUID;
29+
2630
import org.apache.accumulo.core.Constants;
2731
import org.apache.accumulo.core.clientImpl.TabletLocatorImpl.TabletServerLockChecker;
32+
import org.apache.accumulo.core.data.InstanceId;
2833
import org.apache.accumulo.core.fate.zookeeper.ZooCache;
34+
import org.apache.accumulo.core.fate.zookeeper.ZooUtil;
35+
import org.junit.jupiter.api.AfterEach;
2936
import org.junit.jupiter.api.BeforeEach;
3037
import org.junit.jupiter.api.Test;
3138

3239
public class RootTabletLocatorTest {
40+
3341
private ClientContext context;
3442
private TabletServerLockChecker lockChecker;
3543
private ZooCache zc;
36-
private RootTabletLocator rtl;
3744

3845
@BeforeEach
3946
public void setUp() {
40-
context = createMock(ClientContext.class);
41-
expect(context.getZooKeeperRoot()).andReturn("/accumulo/iid").anyTimes();
47+
var instanceId = InstanceId.of(UUID.randomUUID());
4248
zc = createMock(ZooCache.class);
49+
context = createMock(ClientContext.class);
50+
expect(context.getZooKeeperRoot()).andReturn(ZooUtil.getRoot(instanceId)).anyTimes();
4351
expect(context.getZooCache()).andReturn(zc).anyTimes();
44-
replay(context);
4552
lockChecker = createMock(TabletServerLockChecker.class);
46-
rtl = new RootTabletLocator(lockChecker);
53+
replay(context, zc, lockChecker);
54+
}
55+
56+
@AfterEach
57+
public void tearDown() {
58+
verify(context, zc, lockChecker);
4759
}
4860

4961
@Test
5062
public void testInvalidateCache_Server() {
63+
var rtl = new RootTabletLocator(lockChecker);
64+
65+
verify(zc);
66+
reset(zc);
5167
zc.clear(context.getZooKeeperRoot() + Constants.ZTSERVERS + "/server");
68+
expectLastCall().once();
5269
replay(zc);
70+
5371
rtl.invalidateCache(context, "server");
54-
verify(zc);
5572
}
5673
}

core/src/test/java/org/apache/accumulo/core/clientImpl/ZookeeperLockCheckerTest.java

+23-6
Original file line numberDiff line numberDiff line change
@@ -20,34 +20,51 @@
2020

2121
import static org.easymock.EasyMock.createMock;
2222
import static org.easymock.EasyMock.expect;
23+
import static org.easymock.EasyMock.expectLastCall;
2324
import static org.easymock.EasyMock.replay;
25+
import static org.easymock.EasyMock.reset;
2426
import static org.easymock.EasyMock.verify;
2527

28+
import java.util.UUID;
29+
2630
import org.apache.accumulo.core.Constants;
31+
import org.apache.accumulo.core.data.InstanceId;
2732
import org.apache.accumulo.core.fate.zookeeper.ZooCache;
33+
import org.apache.accumulo.core.fate.zookeeper.ZooUtil;
34+
import org.junit.jupiter.api.AfterEach;
2835
import org.junit.jupiter.api.BeforeEach;
2936
import org.junit.jupiter.api.Test;
3037

3138
public class ZookeeperLockCheckerTest {
39+
3240
private ClientContext context;
3341
private ZooCache zc;
34-
private ZookeeperLockChecker zklc;
3542

3643
@BeforeEach
3744
public void setUp() {
38-
context = createMock(ClientContext.class);
39-
expect(context.getZooKeeperRoot()).andReturn("/accumulo/iid").anyTimes();
45+
var instanceId = InstanceId.of(UUID.randomUUID());
4046
zc = createMock(ZooCache.class);
47+
context = createMock(ClientContext.class);
48+
expect(context.getZooKeeperRoot()).andReturn(ZooUtil.getRoot(instanceId)).anyTimes();
4149
expect(context.getZooCache()).andReturn(zc).anyTimes();
42-
replay(context);
43-
zklc = new ZookeeperLockChecker(context);
50+
replay(context, zc);
51+
}
52+
53+
@AfterEach
54+
public void tearDown() {
55+
verify(context, zc);
4456
}
4557

4658
@Test
4759
public void testInvalidateCache() {
60+
var zklc = new ZookeeperLockChecker(context);
61+
62+
verify(zc);
63+
reset(zc);
4864
zc.clear(context.getZooKeeperRoot() + Constants.ZTSERVERS + "/server");
65+
expectLastCall().once();
4966
replay(zc);
67+
5068
zklc.invalidateCache("server");
51-
verify(zc);
5269
}
5370
}

minicluster/src/test/java/org/apache/accumulo/minicluster/MiniAccumuloClusterExistingZooKeepersTest.java

+5-2
Original file line numberDiff line numberDiff line change
@@ -26,8 +26,10 @@
2626
import java.io.File;
2727
import java.util.Map;
2828

29+
import org.apache.accumulo.core.Constants;
2930
import org.apache.accumulo.core.client.Accumulo;
3031
import org.apache.accumulo.core.client.AccumuloClient;
32+
import org.apache.accumulo.core.fate.zookeeper.ZooUtil;
3133
import org.apache.commons.io.FileUtils;
3234
import org.apache.curator.framework.CuratorFramework;
3335
import org.apache.curator.framework.CuratorFrameworkFactory;
@@ -74,8 +76,9 @@ public void canConnectViaExistingZooKeeper() throws Exception {
7476
Map<String,String> tableIds = client.tableOperations().tableIdMap();
7577
assertTrue(tableIds.containsKey(tableName));
7678

77-
String zkTablePath = String.format("/accumulo/%s/tables/%s/name",
78-
client.instanceOperations().getInstanceId().canonical(), tableIds.get(tableName));
79+
String zkTablePath = String.format("%s%s/%s/name",
80+
ZooUtil.getRoot(client.instanceOperations().getInstanceId()), Constants.ZTABLES,
81+
tableIds.get(tableName));
7982
try (CuratorFramework curatorClient =
8083
CuratorFrameworkFactory.newClient(zooKeeper.getConnectString(), new RetryOneTime(1))) {
8184
curatorClient.start();

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

+2-1
Original file line numberDiff line numberDiff line change
@@ -52,6 +52,7 @@
5252
import org.apache.accumulo.core.data.TableId;
5353
import org.apache.accumulo.core.fate.zookeeper.ZooReader;
5454
import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter;
55+
import org.apache.accumulo.core.fate.zookeeper.ZooUtil;
5556
import org.apache.accumulo.core.metadata.schema.Ample;
5657
import org.apache.accumulo.core.metrics.MetricsInfo;
5758
import org.apache.accumulo.core.rpc.SslConnectionParams;
@@ -118,7 +119,7 @@ private ServerContext(ServerInfo info) {
118119
serverDirs = info.getServerDirs();
119120

120121
propStore = memoize(() -> ZooPropStore.initialize(getInstanceID(), getZooReaderWriter()));
121-
zkUserPath = memoize(() -> Constants.ZROOT + "/" + getInstanceID() + Constants.ZUSERS);
122+
zkUserPath = memoize(() -> ZooUtil.getRoot(getInstanceID()) + Constants.ZUSERS);
122123

123124
tableManager = memoize(() -> new TableManager(this));
124125
nameAllocator = memoize(() -> new UniqueNameAllocator(this));

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

+2-1
Original file line numberDiff line numberDiff line change
@@ -36,6 +36,7 @@
3636
import org.apache.accumulo.core.data.InstanceId;
3737
import org.apache.accumulo.core.fate.zookeeper.ZooCache;
3838
import org.apache.accumulo.core.fate.zookeeper.ZooCacheFactory;
39+
import org.apache.accumulo.core.fate.zookeeper.ZooUtil;
3940
import org.apache.accumulo.core.singletons.SingletonManager;
4041
import org.apache.accumulo.core.singletons.SingletonManager.Mode;
4142
import org.apache.accumulo.server.fs.VolumeManager;
@@ -79,7 +80,7 @@ public class ServerInfo implements ClientInfo {
7980
+ "Run \"accumulo org.apache.accumulo.server.util.ListInstances\" to see a list.");
8081
}
8182
instanceID = InstanceId.of(new String(iidb, UTF_8));
82-
if (zooCache.get(Constants.ZROOT + "/" + instanceID) == null) {
83+
if (zooCache.get(ZooUtil.getRoot(instanceID)) == null) {
8384
if (instanceName == null) {
8485
throw new IllegalStateException(
8586
"Instance id " + instanceID + " does not exist in zookeeper");

server/base/src/main/java/org/apache/accumulo/server/init/ZooKeeperInitializer.java

+2-2
Original file line numberDiff line numberDiff line change
@@ -71,7 +71,7 @@ void initializeConfig(final InstanceId instanceId, final ZooReaderWriter zoo) {
7171
zoo.putPersistentData(Constants.ZROOT, new byte[0], ZooUtil.NodeExistsPolicy.SKIP,
7272
ZooDefs.Ids.OPEN_ACL_UNSAFE);
7373

74-
String zkInstanceRoot = Constants.ZROOT + "/" + instanceId;
74+
String zkInstanceRoot = ZooUtil.getRoot(instanceId);
7575
zoo.putPersistentData(zkInstanceRoot, EMPTY_BYTE_ARRAY, ZooUtil.NodeExistsPolicy.SKIP);
7676
var sysPropPath = SystemPropKey.of(instanceId).getPath();
7777
VersionedProperties vProps = new VersionedProperties();
@@ -109,7 +109,7 @@ void initialize(final ServerContext context, final boolean clearInstanceName,
109109
ZooUtil.NodeExistsPolicy.FAIL);
110110

111111
// setup the instance
112-
String zkInstanceRoot = Constants.ZROOT + "/" + instanceId;
112+
String zkInstanceRoot = context.getZooKeeperRoot();
113113
zoo.putPersistentData(zkInstanceRoot + Constants.ZTABLES, Constants.ZTABLES_INITIAL_ID,
114114
ZooUtil.NodeExistsPolicy.FAIL);
115115
zoo.putPersistentData(zkInstanceRoot + Constants.ZNAMESPACES,

server/base/src/main/java/org/apache/accumulo/server/security/handler/ZKPermHandler.java

+3-4
Original file line numberDiff line numberDiff line change
@@ -27,12 +27,12 @@
2727
import java.util.Set;
2828
import java.util.TreeSet;
2929

30+
import org.apache.accumulo.core.Constants;
3031
import org.apache.accumulo.core.client.AccumuloSecurityException;
3132
import org.apache.accumulo.core.client.NamespaceNotFoundException;
3233
import org.apache.accumulo.core.client.TableNotFoundException;
3334
import org.apache.accumulo.core.clientImpl.Namespace;
3435
import org.apache.accumulo.core.clientImpl.thrift.SecurityErrorCode;
35-
import org.apache.accumulo.core.data.InstanceId;
3636
import org.apache.accumulo.core.data.NamespaceId;
3737
import org.apache.accumulo.core.data.TableId;
3838
import org.apache.accumulo.core.fate.zookeeper.ZooCache;
@@ -66,10 +66,9 @@ public class ZKPermHandler implements PermissionHandler {
6666
public void initialize(ServerContext context) {
6767
zooCache = new ZooCache(context.getZooReader(), null);
6868
zoo = context.getZooReaderWriter();
69-
InstanceId instanceId = context.getInstanceID();
7069
zkUserPath = context.zkUserPath();
71-
ZKTablePath = ZKSecurityTool.getInstancePath(instanceId) + "/tables";
72-
ZKNamespacePath = ZKSecurityTool.getInstancePath(instanceId) + "/namespaces";
70+
ZKTablePath = context.getZooKeeperRoot() + Constants.ZTABLES;
71+
ZKNamespacePath = context.getZooKeeperRoot() + Constants.ZNAMESPACES;
7372
}
7473

7574
@Override

server/base/src/main/java/org/apache/accumulo/server/security/handler/ZKSecurityTool.java

-5
Original file line numberDiff line numberDiff line change
@@ -31,9 +31,7 @@
3131
import java.util.HashSet;
3232
import java.util.Set;
3333

34-
import org.apache.accumulo.core.Constants;
3534
import org.apache.accumulo.core.client.AccumuloException;
36-
import org.apache.accumulo.core.data.InstanceId;
3735
import org.apache.accumulo.core.security.Authorizations;
3836
import org.apache.accumulo.core.security.NamespacePermission;
3937
import org.apache.accumulo.core.security.SystemPermission;
@@ -191,7 +189,4 @@ public static Set<NamespacePermission> convertNamespacePermissions(byte[] namesp
191189
return toReturn;
192190
}
193191

194-
public static String getInstancePath(InstanceId instanceId) {
195-
return Constants.ZROOT + "/" + instanceId;
196-
}
197192
}

server/base/src/main/java/org/apache/accumulo/server/tables/TableManager.java

+6-5
Original file line numberDiff line numberDiff line change
@@ -37,6 +37,7 @@
3737
import org.apache.accumulo.core.data.TableId;
3838
import org.apache.accumulo.core.fate.zookeeper.ZooCache;
3939
import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter;
40+
import org.apache.accumulo.core.fate.zookeeper.ZooUtil;
4041
import org.apache.accumulo.core.fate.zookeeper.ZooUtil.NodeExistsPolicy;
4142
import org.apache.accumulo.core.fate.zookeeper.ZooUtil.NodeMissingPolicy;
4243
import org.apache.accumulo.core.manager.state.tables.TableState;
@@ -77,7 +78,7 @@ public static void prepareNewNamespaceState(final ServerContext context, Namespa
7778
final InstanceId instanceId = context.getInstanceID();
7879
log.debug("Creating ZooKeeper entries for new namespace {} (ID: {})", namespace, namespaceId);
7980
context.getZooReaderWriter().putPersistentData(
80-
Constants.ZROOT + "/" + instanceId + Constants.ZNAMESPACES + "/" + namespaceId, new byte[0],
81+
context.getZooKeeperRoot() + Constants.ZNAMESPACES + "/" + namespaceId, new byte[0],
8182
existsPolicy);
8283
var propKey = NamespacePropKey.of(instanceId, namespaceId);
8384
if (!propStore.exists(propKey)) {
@@ -94,7 +95,7 @@ public static void prepareNewTableState(ZooReaderWriter zoo, PropStore propStore
9495
tableName, tableId, namespaceId);
9596
Pair<String,String> qualifiedTableName = TableNameUtil.qualify(tableName);
9697
tableName = qualifiedTableName.getSecond();
97-
String zTablePath = Constants.ZROOT + "/" + instanceId + Constants.ZTABLES + "/" + tableId;
98+
String zTablePath = ZooUtil.getRoot(instanceId) + Constants.ZTABLES + "/" + tableId;
9899
zoo.putPersistentData(zTablePath, new byte[0], existsPolicy);
99100
zoo.putPersistentData(zTablePath + Constants.ZTABLE_NAMESPACE,
100101
namespaceId.canonical().getBytes(UTF_8), existsPolicy);
@@ -220,10 +221,10 @@ public void cloneTable(TableId srcTableId, TableId tableId, String tableName,
220221
prepareNewTableState(zoo, context.getPropStore(), instanceID, tableId, namespaceId, tableName,
221222
TableState.NEW, NodeExistsPolicy.OVERWRITE);
222223

223-
String srcTablePath = Constants.ZROOT + "/" + instanceID + Constants.ZTABLES + "/" + srcTableId
224-
+ Constants.ZCONFIG;
224+
String srcTablePath =
225+
context.getZooKeeperRoot() + Constants.ZTABLES + "/" + srcTableId + Constants.ZCONFIG;
225226
String newTablePath =
226-
Constants.ZROOT + "/" + instanceID + Constants.ZTABLES + "/" + tableId + Constants.ZCONFIG;
227+
context.getZooKeeperRoot() + Constants.ZTABLES + "/" + tableId + Constants.ZCONFIG;
227228
zoo.recursiveCopyPersistentOverwrite(srcTablePath, newTablePath);
228229

229230
PropUtil.setProperties(context, TablePropKey.of(context, tableId), propertiesToSet);

server/base/src/main/java/org/apache/accumulo/server/tablets/UniqueNameAllocator.java

+1-1
Original file line numberDiff line numberDiff line change
@@ -50,7 +50,7 @@ public class UniqueNameAllocator {
5050

5151
public UniqueNameAllocator(ServerContext context) {
5252
this.context = context;
53-
nextNamePath = Constants.ZROOT + "/" + context.getInstanceID() + Constants.ZNEXT_FILE;
53+
nextNamePath = context.getZooKeeperRoot() + Constants.ZNEXT_FILE;
5454
}
5555

5656
public synchronized String getNextName() {

server/base/src/main/java/org/apache/accumulo/server/util/ChangeSecret.java

+3-2
Original file line numberDiff line numberDiff line change
@@ -26,6 +26,7 @@
2626
import java.util.List;
2727
import java.util.UUID;
2828

29+
import org.apache.accumulo.core.Constants;
2930
import org.apache.accumulo.core.conf.AccumuloConfiguration;
3031
import org.apache.accumulo.core.data.InstanceId;
3132
import org.apache.accumulo.core.fate.zookeeper.ZooReader;
@@ -145,7 +146,7 @@ private static void rewriteZooKeeperInstance(final ServerContext context,
145146
}
146147
}
147148
});
148-
String path = "/accumulo/instances/" + context.getInstanceName();
149+
String path = Constants.ZROOT + Constants.ZINSTANCES + "/" + context.getInstanceName();
149150
orig.recursiveDelete(path, NodeMissingPolicy.SKIP);
150151
new_.putPersistentData(path, newInstanceId.canonical().getBytes(UTF_8),
151152
NodeExistsPolicy.OVERWRITE);
@@ -201,6 +202,6 @@ private static void checkHdfsAccessPermissions(FileStatus stat, FsAction mode) t
201202

202203
private static void deleteInstance(ServerContext context, String oldPass) throws Exception {
203204
ZooReaderWriter orig = context.getZooReader().asWriter(oldPass);
204-
orig.recursiveDelete("/accumulo/" + context.getInstanceID(), NodeMissingPolicy.SKIP);
205+
orig.recursiveDelete(context.getZooKeeperRoot(), NodeMissingPolicy.SKIP);
205206
}
206207
}

server/base/src/main/java/org/apache/accumulo/server/util/ListInstances.java

+2-2
Original file line numberDiff line numberDiff line change
@@ -35,6 +35,7 @@
3535
import org.apache.accumulo.core.data.InstanceId;
3636
import org.apache.accumulo.core.fate.zookeeper.ZooCache;
3737
import org.apache.accumulo.core.fate.zookeeper.ZooReader;
38+
import org.apache.accumulo.core.fate.zookeeper.ZooUtil;
3839
import org.apache.accumulo.core.lock.ServiceLock;
3940
import org.apache.accumulo.core.lock.ServiceLockData;
4041
import org.apache.accumulo.core.lock.ServiceLockData.ThriftService;
@@ -165,8 +166,7 @@ private static String getManager(ZooCache cache, InstanceId iid, boolean printEr
165166
}
166167

167168
try {
168-
var zLockManagerPath =
169-
ServiceLock.path(Constants.ZROOT + "/" + iid + Constants.ZMANAGER_LOCK);
169+
var zLockManagerPath = ServiceLock.path(ZooUtil.getRoot(iid) + Constants.ZMANAGER_LOCK);
170170
Optional<ServiceLockData> sld = ServiceLock.getLockData(cache, zLockManagerPath, null);
171171
if (sld.isEmpty()) {
172172
return null;

server/base/src/main/java/org/apache/accumulo/server/util/ZooKeeperMain.java

+1-1
Original file line numberDiff line numberDiff line change
@@ -69,7 +69,7 @@ public void execute(final String[] args) throws Exception {
6969
}
7070
System.out.println("The accumulo instance id is " + context.getInstanceID());
7171
if (!opts.servers.contains("/")) {
72-
opts.servers += "/accumulo/" + context.getInstanceID();
72+
opts.servers += context.getZooKeeperRoot();
7373
}
7474
org.apache.zookeeper.ZooKeeperMain
7575
.main(new String[] {"-server", opts.servers, "-timeout", "" + (opts.timeout * 1000)});

0 commit comments

Comments
 (0)