Skip to content

Commit 335592c

Browse files
authored
Resolved timeout issue in MemoryStarvedMajCIT (apache#4267)
It appears that the test was stopping the Compactors, but not waiting for their ZooKeeper locks to be removed, before moving on and checking the number of zookeeper locks. I think what was happening is that the test code moved forward thinking that the Compactors were stopped, but the Compactors were actually stopping after this point and during the next step in the test.
1 parent b5afd3c commit 335592c

File tree

3 files changed

+23
-4
lines changed

3 files changed

+23
-4
lines changed

core/src/main/java/org/apache/accumulo/core/util/compaction/ExternalCompactionUtil.java

+1-1
Original file line numberDiff line numberDiff line change
@@ -114,7 +114,6 @@ public static Map<String,List<HostAndPort>> getCompactorAddrs(ClientContext cont
114114
ZooReader zooReader = context.getZooReader();
115115
List<String> groups = zooReader.getChildren(compactorGroupsPath);
116116
for (String group : groups) {
117-
groupsAndAddresses.putIfAbsent(group, new ArrayList<>());
118117
try {
119118
List<String> compactors = zooReader.getChildren(compactorGroupsPath + "/" + group);
120119
for (String compactor : compactors) {
@@ -124,6 +123,7 @@ public static Map<String,List<HostAndPort>> getCompactorAddrs(ClientContext cont
124123
zooReader.getChildren(compactorGroupsPath + "/" + group + "/" + compactor);
125124
if (!children.isEmpty()) {
126125
LOG.trace("Found live compactor {} ", compactor);
126+
groupsAndAddresses.putIfAbsent(group, new ArrayList<>());
127127
groupsAndAddresses.get(group).add(HostAndPort.fromString(compactor));
128128
}
129129
}

minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloClusterControl.java

+4
Original file line numberDiff line numberDiff line change
@@ -519,6 +519,10 @@ public void kill(ServerType server, String hostname) throws IOException {
519519
stop(server, hostname);
520520
}
521521

522+
public List<Process> getCompactors(String resourceGroup) {
523+
return compactorProcesses.get(resourceGroup);
524+
}
525+
522526
public List<Process> getTabletServers(String resourceGroup) {
523527
return tabletServerProcesses.get(resourceGroup);
524528
}

test/src/main/java/org/apache/accumulo/test/functional/MemoryStarvedMajCIT.java

+18-3
Original file line numberDiff line numberDiff line change
@@ -22,6 +22,7 @@
2222
import static org.junit.jupiter.api.Assertions.assertEquals;
2323
import static org.junit.jupiter.api.Assertions.assertNull;
2424
import static org.junit.jupiter.api.Assertions.assertTrue;
25+
import static org.junit.jupiter.api.Assertions.fail;
2526

2627
import java.util.List;
2728
import java.util.Map;
@@ -66,6 +67,7 @@ public void configureMiniCluster(MiniAccumuloConfigImpl cfg, Configuration coreS
6667
cfg.getClusterServerConfiguration().setNumDefaultTabletServers(1);
6768
cfg.getClusterServerConfiguration().setNumDefaultScanServers(0);
6869
cfg.getClusterServerConfiguration().setNumDefaultCompactors(1);
70+
cfg.setProperty(Property.INSTANCE_ZK_TIMEOUT, "15s");
6971
cfg.setProperty(Property.GENERAL_LOW_MEM_DETECTOR_INTERVAL, "5s");
7072
cfg.setProperty(Property.GENERAL_LOW_MEM_DETECTOR_THRESHOLD,
7173
Double.toString(MemoryStarvedScanIT.FREE_MEMORY_THRESHOLD));
@@ -132,10 +134,23 @@ public void testMajCPauses() throws Exception {
132134

133135
ClientContext ctx = (ClientContext) client;
134136

135-
// Stop the normal compactors and start the version that will consume
136-
// and free memory when we need it to
137-
getCluster().getClusterControl().stopAllServers(ServerType.COMPACTOR);
137+
// Kill the normal compactors and wait until their addresses in ZK are cleared
138+
getCluster().getConfig().getClusterServerConfiguration().getCompactorConfiguration().keySet()
139+
.forEach(resourceGroup -> {
140+
List<Process> procs = getCluster().getClusterControl().getCompactors(resourceGroup);
141+
for (int i = 0; i < procs.size(); i++) {
142+
LOG.info("Stopping compactor process: {}", procs.get(i).pid());
143+
try {
144+
procs.get(i).destroyForcibly().waitFor();
145+
} catch (InterruptedException e) {
146+
fail("Interrupted trying to stop compactor process");
147+
}
148+
}
149+
getCluster().getClusterControl().getCompactors(resourceGroup).clear();
150+
});
151+
Wait.waitFor(() -> ExternalCompactionUtil.getCompactorAddrs(ctx).size() == 0, 60_000);
138152

153+
// Start the Compactors that will consume and free memory when we need it to
139154
getCluster().getClusterControl().start(ServerType.COMPACTOR, null, 1,
140155
MemoryConsumingCompactor.class);
141156
Wait.waitFor(() -> ExternalCompactionUtil.getCompactorAddrs(ctx).size() == 4, 60_000);

0 commit comments

Comments
 (0)