|
@@ -17,59 +17,63 @@
|
|
|
*/
|
|
|
package org.apache.hadoop.ozone;
|
|
|
|
|
|
-import java.io.IOException;
|
|
|
+import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_CONTAINER_REPORT_INTERVAL;
|
|
|
+import static org.junit.Assert.assertFalse;
|
|
|
+import static org.junit.Assert.assertTrue;
|
|
|
+import static org.junit.Assert.fail;
|
|
|
|
|
|
+import com.google.common.collect.Lists;
|
|
|
+import com.google.common.collect.Maps;
|
|
|
+import com.google.common.util.concurrent.AtomicDouble;
|
|
|
+import java.io.IOException;
|
|
|
+import java.nio.file.Path;
|
|
|
+import java.nio.file.Paths;
|
|
|
+import java.util.Collections;
|
|
|
+import java.util.HashSet;
|
|
|
+import java.util.List;
|
|
|
+import java.util.Map;
|
|
|
+import java.util.Set;
|
|
|
+import java.util.UUID;
|
|
|
+import java.util.concurrent.TimeUnit;
|
|
|
+import java.util.concurrent.atomic.AtomicReference;
|
|
|
import org.apache.commons.lang3.RandomUtils;
|
|
|
+import org.apache.hadoop.hdds.HddsConfigKeys;
|
|
|
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
|
|
|
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
|
|
|
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleEvent;
|
|
|
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState;
|
|
|
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeType;
|
|
|
+import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.DeletedBlocksTransaction;
|
|
|
+import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.SCMCommandProto;
|
|
|
+import org.apache.hadoop.hdds.scm.ScmConfigKeys;
|
|
|
+import org.apache.hadoop.hdds.scm.ScmInfo;
|
|
|
+import org.apache.hadoop.hdds.scm.XceiverClientManager;
|
|
|
+import org.apache.hadoop.hdds.scm.block.DeletedBlockLog;
|
|
|
+import org.apache.hadoop.hdds.scm.block.SCMBlockDeletingService;
|
|
|
+import org.apache.hadoop.hdds.scm.container.ContainerMapping;
|
|
|
+import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
|
|
|
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline;
|
|
|
+import org.apache.hadoop.hdds.scm.exceptions.SCMException;
|
|
|
+import org.apache.hadoop.hdds.scm.node.NodeManager;
|
|
|
+import org.apache.hadoop.hdds.scm.server.SCMChillModeManager;
|
|
|
import org.apache.hadoop.hdds.scm.server.SCMClientProtocolServer;
|
|
|
import org.apache.hadoop.hdds.scm.server.SCMStorage;
|
|
|
-import org.apache.hadoop.hdds.scm.node.NodeManager;
|
|
|
+import org.apache.hadoop.hdds.scm.server.StorageContainerManager;
|
|
|
+import org.apache.hadoop.hdds.scm.server.StorageContainerManager.StartupOption;
|
|
|
import org.apache.hadoop.ozone.container.ContainerTestHelper;
|
|
|
import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
|
|
|
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
|
|
|
import org.apache.hadoop.ozone.protocol.commands.DeleteBlocksCommand;
|
|
|
import org.apache.hadoop.ozone.protocol.commands.SCMCommand;
|
|
|
-import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
|
|
|
-import org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeType;
|
|
|
-import org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState;
|
|
|
-import org.apache.hadoop.hdds.protocol.proto
|
|
|
- .StorageContainerDatanodeProtocolProtos.DeletedBlocksTransaction;
|
|
|
-import org.apache.hadoop.hdds.protocol.proto
|
|
|
- .StorageContainerDatanodeProtocolProtos.SCMCommandProto;
|
|
|
-import org.apache.hadoop.hdds.scm.server.StorageContainerManager;
|
|
|
-import org.apache.hadoop.hdds.scm.server.StorageContainerManager.StartupOption;
|
|
|
-import org.apache.hadoop.hdds.scm.block.DeletedBlockLog;
|
|
|
-import org.apache.hadoop.hdds.scm.block.SCMBlockDeletingService;
|
|
|
-import org.apache.hadoop.hdds.scm.exceptions.SCMException;
|
|
|
-import org.apache.hadoop.hdds.scm.XceiverClientManager;
|
|
|
-import org.apache.hadoop.hdds.scm.ScmInfo;
|
|
|
-import org.junit.Rule;
|
|
|
+import org.apache.hadoop.test.GenericTestUtils;
|
|
|
import org.junit.Assert;
|
|
|
+import org.junit.Rule;
|
|
|
import org.junit.Test;
|
|
|
import org.junit.rules.ExpectedException;
|
|
|
-
|
|
|
-import java.nio.file.Path;
|
|
|
-import java.nio.file.Paths;
|
|
|
-import java.util.List;
|
|
|
-import java.util.HashSet;
|
|
|
-import java.util.Set;
|
|
|
-import java.util.Map;
|
|
|
-import java.util.Collections;
|
|
|
-import java.util.UUID;
|
|
|
-import java.util.concurrent.TimeUnit;
|
|
|
-
|
|
|
-import com.google.common.collect.Lists;
|
|
|
-import com.google.common.collect.Maps;
|
|
|
-import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
|
|
|
-import org.apache.hadoop.hdds.scm.ScmConfigKeys;
|
|
|
-
|
|
|
import org.junit.rules.Timeout;
|
|
|
import org.mockito.Mockito;
|
|
|
-import org.apache.hadoop.test.GenericTestUtils;
|
|
|
-
|
|
|
-import static org.apache.hadoop.hdds
|
|
|
- .HddsConfigKeys.HDDS_CONTAINER_REPORT_INTERVAL;
|
|
|
-import static org.junit.Assert.fail;
|
|
|
+import org.slf4j.Logger;
|
|
|
+import org.slf4j.LoggerFactory;
|
|
|
|
|
|
/**
|
|
|
* Test class that exercises the StorageContainerManager.
|
|
@@ -78,6 +82,8 @@ public class TestStorageContainerManager {
|
|
|
private static XceiverClientManager xceiverClientManager =
|
|
|
new XceiverClientManager(
|
|
|
new OzoneConfiguration());
|
|
|
+ private static final Logger LOG = LoggerFactory
|
|
|
+ .getLogger(TestStorageContainerManager.class);
|
|
|
/**
|
|
|
* Set the timeout for every test.
|
|
|
*/
|
|
@@ -457,4 +463,92 @@ public class TestStorageContainerManager {
|
|
|
Assert.assertEquals(clusterId, scmInfo.getClusterId());
|
|
|
Assert.assertEquals(scmId, scmInfo.getScmId());
|
|
|
}
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testSCMChillMode() throws Exception {
|
|
|
+ OzoneConfiguration conf = new OzoneConfiguration();
|
|
|
+ MiniOzoneCluster.Builder builder = MiniOzoneCluster.newBuilder(conf)
|
|
|
+ .setHbInterval(1000)
|
|
|
+ .setNumDatanodes(3)
|
|
|
+ .setStartDataNodes(false)
|
|
|
+ .setHbProcessorInterval(500);
|
|
|
+ MiniOzoneClusterImpl cluster = (MiniOzoneClusterImpl) builder.build();
|
|
|
+ // Test1: Test chill mode when there are no containers in system.
|
|
|
+ assertTrue(cluster.getStorageContainerManager().isInChillMode());
|
|
|
+ cluster.startHddsDatanodes();
|
|
|
+ cluster.waitForClusterToBeReady();
|
|
|
+ assertFalse(cluster.getStorageContainerManager().isInChillMode());
|
|
|
+
|
|
|
+ // Test2: Test chill mode when containers are there in system.
|
|
|
+ // Create {numKeys} random names keys.
|
|
|
+ TestStorageContainerManagerHelper helper =
|
|
|
+ new TestStorageContainerManagerHelper(cluster, conf);
|
|
|
+ Map<String, OmKeyInfo> keyLocations = helper.createKeys(100*2, 4096);
|
|
|
+ final List<ContainerInfo> containers = cluster.getStorageContainerManager()
|
|
|
+ .getScmContainerManager().getStateManager().getAllContainers();
|
|
|
+ GenericTestUtils.waitFor(() -> {
|
|
|
+ return containers.size() > 10;
|
|
|
+ }, 100, 1000);
|
|
|
+
|
|
|
+ // Removing some container to keep them open.
|
|
|
+ containers.remove(0);
|
|
|
+ containers.remove(1);
|
|
|
+ containers.remove(2);
|
|
|
+ containers.remove(3);
|
|
|
+
|
|
|
+ // Close remaining containers
|
|
|
+ ContainerMapping mapping = (ContainerMapping) cluster
|
|
|
+ .getStorageContainerManager().getScmContainerManager();
|
|
|
+ containers.forEach(c -> {
|
|
|
+ try {
|
|
|
+ mapping.updateContainerState(c.getContainerID(),
|
|
|
+ HddsProtos.LifeCycleEvent.FINALIZE);
|
|
|
+ mapping.updateContainerState(c.getContainerID(),
|
|
|
+ LifeCycleEvent.CLOSE);
|
|
|
+ } catch (IOException e) {
|
|
|
+ LOG.info("Failed to change state of open containers.", e);
|
|
|
+ }
|
|
|
+ });
|
|
|
+ cluster.stop();
|
|
|
+
|
|
|
+ GenericTestUtils.LogCapturer logCapturer = GenericTestUtils.LogCapturer
|
|
|
+ .captureLogs(SCMChillModeManager.getLogger());
|
|
|
+ logCapturer.clearOutput();
|
|
|
+ AtomicReference<MiniOzoneCluster> miniCluster = new AtomicReference<>();
|
|
|
+ new Thread(() -> {
|
|
|
+ try {
|
|
|
+ miniCluster.set(builder.setStartDataNodes(false).build());
|
|
|
+ } catch (IOException e) {
|
|
|
+ fail("failed");
|
|
|
+ }
|
|
|
+ }).start();
|
|
|
+
|
|
|
+ StorageContainerManager scm;
|
|
|
+ GenericTestUtils.waitFor(() -> {
|
|
|
+ return miniCluster.get() != null;
|
|
|
+ }, 100, 1000 * 3);
|
|
|
+
|
|
|
+ scm = miniCluster.get().getStorageContainerManager();
|
|
|
+ assertTrue(scm.isInChillMode());
|
|
|
+ assertFalse(logCapturer.getOutput().contains("SCM exiting chill mode."));
|
|
|
+ assertTrue(scm.getCurrentContainerThreshold() == 0);
|
|
|
+ AtomicDouble curThreshold = new AtomicDouble();
|
|
|
+ AtomicDouble lastReportedThreshold = new AtomicDouble();
|
|
|
+ for(HddsDatanodeService dn:miniCluster.get().getHddsDatanodes()){
|
|
|
+ dn.start(null);
|
|
|
+ GenericTestUtils.waitFor(() -> {
|
|
|
+ curThreshold.set(scm.getCurrentContainerThreshold());
|
|
|
+ return curThreshold.get() > lastReportedThreshold.get();
|
|
|
+ }, 100, 1000 * 5);
|
|
|
+ lastReportedThreshold.set(curThreshold.get());
|
|
|
+ }
|
|
|
+ double chillModeCutoff = conf
|
|
|
+ .getDouble(HddsConfigKeys.HDDS_SCM_CHILLMODE_THRESHOLD_PCT,
|
|
|
+ HddsConfigKeys.HDDS_SCM_CHILLMODE_THRESHOLD_PCT_DEFAULT);
|
|
|
+ assertTrue(scm.getCurrentContainerThreshold() >= chillModeCutoff);
|
|
|
+ assertTrue(logCapturer.getOutput().contains("SCM exiting chill mode."));
|
|
|
+ assertFalse(scm.isInChillMode());
|
|
|
+ cluster.shutdown();
|
|
|
+ }
|
|
|
+
|
|
|
}
|