瀏覽代碼

HDDS-1758. Add replication and key deletion tests to MiniOzoneChaosCluster. Contributed by Mukul Kumar Singh. (#1049)

Mukul Kumar Singh 5 年之前
父節點
當前提交
c9c3429a16

+ 50 - 7
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/MiniOzoneChaosCluster.java

@@ -22,6 +22,7 @@ import org.apache.commons.lang3.RandomUtils;
 import org.apache.hadoop.conf.StorageUnit;
 import org.apache.hadoop.hdds.HddsConfigKeys;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import org.apache.hadoop.hdds.scm.ScmConfigKeys;
 import org.apache.hadoop.hdds.scm.server.StorageContainerManager;
 import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
@@ -55,7 +56,8 @@ public class MiniOzoneChaosCluster extends MiniOzoneClusterImpl {
   private ScheduledFuture scheduledFuture;
 
   private enum FailureMode {
-    NODES
+    NODES_RESTART,
+    NODES_SHUTDOWN
   }
 
   public MiniOzoneChaosCluster(OzoneConfiguration conf,
@@ -81,21 +83,55 @@ public class MiniOzoneChaosCluster extends MiniOzoneClusterImpl {
     return RandomUtils.nextBoolean();
   }
 
+  // Should the selected node be stopped or started.
+  private boolean shouldStop() {
+    return RandomUtils.nextBoolean();
+  }
+
   // Get the datanode index of the datanode to fail.
   private int getNodeToFail() {
     return RandomUtils.nextInt() % numDatanodes;
   }
 
-  private void failNodes() {
+  private void restartNodes() {
     final int numNodesToFail = getNumberOfNodesToFail();
     LOG.info("Will restart {} nodes to simulate failure", numNodesToFail);
     for (int i = 0; i < numNodesToFail; i++) {
       boolean failureMode = isFastRestart();
       int failedNodeIndex = getNodeToFail();
+      String failString = failureMode ? "Fast" : "Slow";
+      DatanodeDetails dn =
+          getHddsDatanodes().get(failedNodeIndex).getDatanodeDetails();
       try {
-        LOG.info("Restarting DataNodeIndex {}", failedNodeIndex);
+        LOG.info("{} Restarting DataNode: {}", failString, dn.getUuid());
         restartHddsDatanode(failedNodeIndex, failureMode);
-        LOG.info("Completed restarting DataNodeIndex {}", failedNodeIndex);
+        LOG.info("{} Completed restarting Datanode: {}", failString,
+            dn.getUuid());
+      } catch (Exception e) {
+
+      }
+    }
+  }
+
+  private void shutdownNodes() {
+    final int numNodesToFail = getNumberOfNodesToFail();
+    LOG.info("Will shutdown {} nodes to simulate failure", numNodesToFail);
+    for (int i = 0; i < numNodesToFail; i++) {
+      boolean shouldStop = shouldStop();
+      int failedNodeIndex = getNodeToFail();
+      String stopString = shouldStop ? "Stopping" : "Starting";
+      DatanodeDetails dn =
+          getHddsDatanodes().get(failedNodeIndex).getDatanodeDetails();
+      try {
+        LOG.info("{} DataNode {}", stopString, dn.getUuid());
+
+        if (shouldStop) {
+          shutdownHddsDatanode(failedNodeIndex);
+        } else {
+          restartHddsDatanode(failedNodeIndex, true);
+        }
+        LOG.info("Completed {} DataNode {}", stopString, dn.getUuid());
+
       } catch (Exception e) {
 
       }
@@ -111,8 +147,11 @@ public class MiniOzoneChaosCluster extends MiniOzoneClusterImpl {
   private void fail() {
     FailureMode mode = getFailureMode();
     switch (mode) {
-    case NODES:
-      failNodes();
+    case NODES_RESTART:
+      restartNodes();
+      break;
+    case NODES_SHUTDOWN:
+      shutdownNodes();
       break;
 
     default:
@@ -190,7 +229,9 @@ public class MiniOzoneChaosCluster extends MiniOzoneClusterImpl {
           1, StorageUnit.MB);
       conf.setTimeDuration(ScmConfigKeys.HDDS_SCM_WATCHER_TIMEOUT, 1000,
           TimeUnit.MILLISECONDS);
-      conf.setTimeDuration(ScmConfigKeys.OZONE_SCM_STALENODE_INTERVAL, 5,
+      conf.setTimeDuration(ScmConfigKeys.OZONE_SCM_STALENODE_INTERVAL, 10,
+          TimeUnit.SECONDS);
+      conf.setTimeDuration(ScmConfigKeys.OZONE_SCM_DEADNODE_INTERVAL, 20,
           TimeUnit.SECONDS);
       conf.setTimeDuration(HddsConfigKeys.HDDS_CONTAINER_REPORT_INTERVAL, 1,
           TimeUnit.SECONDS);
@@ -204,6 +245,8 @@ public class MiniOzoneChaosCluster extends MiniOzoneClusterImpl {
       conf.setTimeDuration(HddsConfigKeys.HDDS_HEARTBEAT_INTERVAL, 1,
           TimeUnit.SECONDS);
       conf.setInt(OzoneConfigKeys.OZONE_CONTAINER_CACHE_SIZE, 8);
+      conf.setInt("hdds.scm.replication.thread.interval", 10 * 1000);
+      conf.setInt("hdds.scm.replication.event.timeout", 20 * 1000);
     }
 
     @Override

+ 5 - 0
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/MiniOzoneLoadGenerator.java

@@ -130,6 +130,11 @@ public class MiniOzoneLoadGenerator {
         break;
       }
 
+      try {
+        bucket.deleteKey(keyName);
+      } catch (Exception e) {
+        LOG.error("LOADGEN: Unable to delete key:{}", keyName, e);
+      }
     }
     // This will terminate other threads too.
     isWriteThreadRunning.set(false);

+ 1 - 1
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestMiniChaosOzoneCluster.java

@@ -62,7 +62,7 @@ public class TestMiniChaosOzoneCluster implements Runnable {
 
   @Option(names = {"-i", "--failureInterval"},
       description = "time between failure events in seconds")
-  private static int failureInterval = 5; // 5 second period between failures.
+  private static int failureInterval = 300; // 5 second period between failures.
 
   private static MiniOzoneChaosCluster cluster;
   private static MiniOzoneLoadGenerator loadGenerator;

+ 2 - 1
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/KeyManagerImpl.java

@@ -711,7 +711,8 @@ public class KeyManagerImpl implements KeyManager {
                   k.setPipeline(cp.getPipeline());
                 }
               } catch (IOException e) {
-                LOG.debug("Unable to update pipeline for container");
+                LOG.error("Unable to update pipeline for container:{}",
+                    k.getContainerID());
               }
             }
           });