Selaa lähdekoodia

HDFS-9282. Make data directory count and storage raw capacity related tests FsDataset-agnostic. (Tony Wu via lei)

(cherry picked from commit 5a1a54c9f5d9a8f7585000fe33ef106e62dbb43a)
Lei Xu 9 vuotta sitten
vanhempi
commit
489c920c9a

+ 3 - 0
hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt

@@ -827,6 +827,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-9372. Remove dead code in DataStorage.recoverTransitionRead.
     (Duo Zhang via wheat9)
 
+    HDFS-9282. Make data directory count and storage raw capacity related tests
+    FsDataset-agnostic. (Tony Wu via lei)
+
   BUG FIXES
 
     HDFS-8091: ACLStatus and XAttributes should be presented to

+ 2 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java

@@ -192,6 +192,8 @@ public class MiniDFSCluster {
     
     public Builder(Configuration conf) {
       this.conf = conf;
+      this.storagesPerDatanode =
+          FsDatasetTestUtils.Factory.getFactory(conf).getDefaultNumOfDataDirs();
     }
     
     /**

+ 5 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/FsDatasetImplTestUtilsFactory.java

@@ -29,4 +29,9 @@ public final class FsDatasetImplTestUtilsFactory
   public FsDatasetTestUtils newInstance(DataNode datanode) {
     return new FsDatasetImplTestUtils(datanode);
   }
+
+  @Override
+  public int getDefaultNumOfDataDirs() {
+    return FsDatasetImplTestUtils.DEFAULT_NUM_OF_DATA_DIRS;
+  }
 }

+ 19 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/FsDatasetTestUtils.java

@@ -67,6 +67,14 @@ public interface FsDatasetTestUtils {
     public boolean isSimulated() {
       return false;
     }
+
+    /**
+     * Get the default number of data directories for underlying storage per
+     * DataNode.
+     *
+     * @return The default number of data dirs per DataNode.
+     */
+    abstract public int getDefaultNumOfDataDirs();
   }
 
   /**
@@ -213,4 +221,15 @@ public interface FsDatasetTestUtils {
    * @return Replica for the block.
    */
   Replica fetchReplica(ExtendedBlock block);
+
+  /**
+   * @return The default value of number of data dirs per DataNode in
+   * MiniDFSCluster.
+   */
+  int getDefaultNumOfDataDirs();
+
+  /**
+   * Obtain the raw capacity of underlying storage per DataNode.
+   */
+  long getRawCapacity() throws IOException;
 }

+ 6 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeMultipleRegistrations.java

@@ -97,7 +97,9 @@ public class TestDataNodeMultipleRegistrations {
         LOG.info("vol " + i++ + ") " + e.getKey() + ": " + e.getValue());
       }
       // number of volumes should be 2 - [data1, data2]
-      assertEquals("number of volumes is wrong", 2, volInfos.size());
+      assertEquals("number of volumes is wrong",
+          cluster.getFsDatasetTestUtils(0).getDefaultNumOfDataDirs(),
+          volInfos.size());
 
       for (BPOfferService bpos : dn.getAllBpOs()) {
         LOG.info("BP: " + bpos);
@@ -164,7 +166,9 @@ public class TestDataNodeMultipleRegistrations {
         LOG.info("vol " + i++ + ") " + e.getKey() + ": " + e.getValue());
       }
       // number of volumes should be 2 - [data1, data2]
-      assertEquals("number of volumes is wrong", 2, volInfos.size());
+      assertEquals("number of volumes is wrong",
+          cluster.getFsDatasetTestUtils(0).getDefaultNumOfDataDirs(),
+          volInfos.size());
 
       for (BPOfferService bpos : dn.getAllBpOs()) {
         LOG.info("reg: bpid=" + "; name=" + bpos.bpRegistration + "; sid="

+ 21 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImplTestUtils.java

@@ -24,6 +24,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.DF;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
@@ -57,6 +58,11 @@ public class FsDatasetImplTestUtils implements FsDatasetTestUtils {
       LogFactory.getLog(FsDatasetImplTestUtils.class);
   private final FsDatasetImpl dataset;
 
+  /**
+   * By default we assume 2 data directories (volumes) per DataNode.
+   */
+  public static final int DEFAULT_NUM_OF_DATA_DIRS = 2;
+
   /**
    * A reference to the replica that is used to corrupt block / meta later.
    */
@@ -322,4 +328,19 @@ public class FsDatasetImplTestUtils implements FsDatasetTestUtils {
   public Replica fetchReplica(ExtendedBlock block) {
     return dataset.fetchReplicaInfo(block.getBlockPoolId(), block.getBlockId());
   }
+
+  @Override
+  public int getDefaultNumOfDataDirs() {
+    return this.DEFAULT_NUM_OF_DATA_DIRS;
+  }
+
+  @Override
+  public long getRawCapacity() throws IOException {
+    try (FsVolumeReferences volRefs = dataset.getFsVolumeReferences()) {
+      Preconditions.checkState(volRefs.size() != 0);
+      DF df = new DF(new File(volRefs.get(0).getBasePath()),
+          dataset.datanode.getConf());
+      return df.getCapacity();
+    }
+  }
 }

+ 6 - 8
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNamenodeCapacityReport.java

@@ -21,7 +21,6 @@ package org.apache.hadoop.hdfs.server.namenode;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
-import java.io.File;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
@@ -29,7 +28,6 @@ import java.util.List;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.DF;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSOutputStream;
@@ -45,6 +43,7 @@ import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
+import org.apache.hadoop.hdfs.server.datanode.FsDatasetTestUtils;
 import org.junit.Test;
 
 
@@ -110,9 +109,7 @@ public class TestNamenodeCapacityReport {
         assertTrue(percentBpUsed == DFSUtilClient.getPercentUsed(bpUsed,
                                                                  configCapacity));
       }   
-      
-      DF df = new DF(new File(cluster.getDataDirectory()), conf);
-     
+
       //
       // Currently two data directories are created by the data node
       // in the MiniDFSCluster. This results in each data directory having
@@ -123,9 +120,10 @@ public class TestNamenodeCapacityReport {
       // So multiply the disk capacity and reserved space by two 
       // for accommodating it
       //
-      int numOfDataDirs = 2;
-      
-      long diskCapacity = numOfDataDirs * df.getCapacity();
+      final FsDatasetTestUtils utils = cluster.getFsDatasetTestUtils(0);
+      int numOfDataDirs = utils.getDefaultNumOfDataDirs();
+
+      long diskCapacity = numOfDataDirs * utils.getRawCapacity();
       reserved *= numOfDataDirs;
       
       configCapacity = namesystem.getCapacityTotal();