HDDS-609. On restart, SCM does not exit chill mode as it expects DNs to report containers in ALLOCATED state. Contributed by Hanisha Koneru.

This commit is contained in:
Bharat Viswanadham 2018-10-11 21:03:01 -07:00
parent c0118105b7
commit 74db993a61
2 changed files with 61 additions and 4 deletions

View File

@ -29,6 +29,7 @@ import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicLong;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdds.HddsConfigKeys; import org.apache.hadoop.hdds.HddsConfigKeys;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ScmOps; import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ScmOps;
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo; import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
import org.apache.hadoop.hdds.scm.events.SCMEvents; import org.apache.hadoop.hdds.scm.events.SCMEvents;
@ -162,7 +163,7 @@ public class SCMChillModeManager implements
// Required cutoff % for containers with at least 1 reported replica. // Required cutoff % for containers with at least 1 reported replica.
private double chillModeCutoff; private double chillModeCutoff;
// Containers read from scm db. // Containers read from scm db (excluding containers in ALLOCATED state).
private Map<Long, ContainerInfo> containerMap; private Map<Long, ContainerInfo> containerMap;
private double maxContainer; private double maxContainer;
@ -174,11 +175,16 @@ public class SCMChillModeManager implements
containerMap = new ConcurrentHashMap<>(); containerMap = new ConcurrentHashMap<>();
if(containers != null) { if(containers != null) {
containers.forEach(c -> { containers.forEach(c -> {
if (c != null) { // Containers in ALLOCATED state should not be included while
// calculating the total number of containers here. They are not
// reported by DNs and hence should not affect the chill mode exit
// rule.
if (c != null && c.getState() != null &&
!c.getState().equals(HddsProtos.LifeCycleState.ALLOCATED)) {
containerMap.put(c.getContainerID(), c); containerMap.put(c.getContainerID(), c);
} }
}); });
maxContainer = containers.size(); maxContainer = containerMap.size();
} }
} }

View File

@ -22,6 +22,7 @@ import java.util.List;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdds.HddsConfigKeys; import org.apache.hadoop.hdds.HddsConfigKeys;
import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import org.apache.hadoop.hdds.scm.HddsTestUtils; import org.apache.hadoop.hdds.scm.HddsTestUtils;
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo; import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
import org.apache.hadoop.hdds.scm.events.SCMEvents; import org.apache.hadoop.hdds.scm.events.SCMEvents;
@ -70,6 +71,11 @@ public class TestSCMChillModeManager {
private void testChillMode(int numContainers) throws Exception { private void testChillMode(int numContainers) throws Exception {
containers = new ArrayList<>(); containers = new ArrayList<>();
containers.addAll(HddsTestUtils.getContainerInfo(numContainers)); containers.addAll(HddsTestUtils.getContainerInfo(numContainers));
// Assign open state to containers to be included in the chill mode
// container list
for (ContainerInfo container : containers) {
container.setState(HddsProtos.LifeCycleState.OPEN);
}
scmChillModeManager = new SCMChillModeManager(config, containers, queue); scmChillModeManager = new SCMChillModeManager(config, containers, queue);
queue.addHandler(SCMEvents.NODE_REGISTRATION_CONT_REPORT, queue.addHandler(SCMEvents.NODE_REGISTRATION_CONT_REPORT,
scmChillModeManager); scmChillModeManager);
@ -85,6 +91,11 @@ public class TestSCMChillModeManager {
public void testChillModeExitRule() throws Exception { public void testChillModeExitRule() throws Exception {
containers = new ArrayList<>(); containers = new ArrayList<>();
containers.addAll(HddsTestUtils.getContainerInfo(25 * 4)); containers.addAll(HddsTestUtils.getContainerInfo(25 * 4));
// Assign open state to containers to be included in the chill mode
// container list
for (ContainerInfo container : containers) {
container.setState(HddsProtos.LifeCycleState.OPEN);
}
scmChillModeManager = new SCMChillModeManager(config, containers, queue); scmChillModeManager = new SCMChillModeManager(config, containers, queue);
queue.addHandler(SCMEvents.NODE_REGISTRATION_CONT_REPORT, queue.addHandler(SCMEvents.NODE_REGISTRATION_CONT_REPORT,
scmChillModeManager); scmChillModeManager);
@ -119,6 +130,46 @@ public class TestSCMChillModeManager {
testChillModeDataNodes(5); testChillModeDataNodes(5);
} }
/**
* Check that containers in Allocated state are not considered while
* computing percentage of containers with at least 1 reported replica in
* chill mode exit rule.
*/
@Test
public void testContainerChillModeRule() throws Exception {
containers = new ArrayList<>();
// Add 100 containers to the list of containers in SCM
containers.addAll(HddsTestUtils.getContainerInfo(25 * 4));
// Assign OPEN state to first 25 containers and ALLLOCATED state to rest
// of the containers
for (ContainerInfo container : containers.subList(0, 25)) {
container.setState(HddsProtos.LifeCycleState.OPEN);
}
for (ContainerInfo container : containers.subList(25, 100)) {
container.setState(HddsProtos.LifeCycleState.ALLOCATED);
}
scmChillModeManager = new SCMChillModeManager(config, containers, queue);
queue.addHandler(SCMEvents.NODE_REGISTRATION_CONT_REPORT,
scmChillModeManager);
assertTrue(scmChillModeManager.getInChillMode());
// When 10 OPEN containers are reported by DNs, the computed container
// threshold should be 10/25 as there are only 25 open containers.
// Containers in ALLOCATED state should not contribute towards list of
// containers while calculating container threshold in SCMChillNodeManager
testContainerThreshold(containers.subList(0, 10), 0.4);
assertTrue(scmChillModeManager.getInChillMode());
// When remaining 15 OPEN containers are reported by DNs, the container
// threshold should be (10+15)/25.
testContainerThreshold(containers.subList(10, 25), 1.0);
GenericTestUtils.waitFor(() -> {
return !scmChillModeManager.getInChillMode();
}, 100, 1000 * 5);
}
private void testChillModeDataNodes(int numOfDns) throws Exception { private void testChillModeDataNodes(int numOfDns) throws Exception {
OzoneConfiguration conf = new OzoneConfiguration(config); OzoneConfiguration conf = new OzoneConfiguration(config);
conf.setInt(HddsConfigKeys.HDDS_SCM_CHILLMODE_MIN_DATANODE, numOfDns); conf.setInt(HddsConfigKeys.HDDS_SCM_CHILLMODE_MIN_DATANODE, numOfDns);
@ -146,7 +197,7 @@ public class TestSCMChillModeManager {
queue.fireEvent(SCMEvents.NODE_REGISTRATION_CONT_REPORT, queue.fireEvent(SCMEvents.NODE_REGISTRATION_CONT_REPORT,
HddsTestUtils.createNodeRegistrationContainerReport(containers)); HddsTestUtils.createNodeRegistrationContainerReport(containers));
GenericTestUtils.waitFor(() -> { GenericTestUtils.waitFor(() -> {
return scmChillModeManager.getInChillMode(); return !scmChillModeManager.getInChillMode();
}, 10, 1000 * 10); }, 10, 1000 * 10);
} }