You are viewing a plain text version of this content. The canonical link for it is here.
Posted to hdfs-commits@hadoop.apache.org by ar...@apache.org on 2014/08/23 08:36:47 UTC

svn commit: r1619975 - in /hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs: ./ src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/ src/test/java/org/apache/hadoop/hdfs/ src/test/java/org/apache/hadoop/hdfs/server/datano...

Author: arp
Date: Sat Aug 23 06:36:46 2014
New Revision: 1619975

URL: http://svn.apache.org/r1619975
Log:
HDFS-6899: Merging r1619970 from trunk to branch-2.

Modified:
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSClusterWithNodeGroup.java
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSafeMode.java
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockHasMultipleReplicasOnSameDN.java
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDnRespectsBlockReportSplitThreshold.java
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/metrics/TestNameNodeMetrics.java

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt?rev=1619975&r1=1619974&r2=1619975&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt Sat Aug 23 06:36:46 2014
@@ -159,6 +159,9 @@ Release 2.6.0 - UNRELEASED
     HDFS-6758. block writer should pass the expected block size to
     DataXceiverServer. (Arpit Agarwal)
 
+    HDFS-6899. Allow changing MiniDFSCluster volumes per DN and capacity
+    per volume. (Arpit Agarwal)
+
   OPTIMIZATIONS
 
     HDFS-6690. Deduplicate xattr names in memory. (wang)

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java?rev=1619975&r1=1619974&r2=1619975&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java Sat Aug 23 06:36:46 2014
@@ -29,6 +29,7 @@ import java.util.concurrent.ThreadFactor
 import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
 
+import com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.DF;
@@ -49,7 +50,8 @@ import com.google.common.util.concurrent
  * It uses the {@link FsDatasetImpl} object for synchronization.
  */
 @InterfaceAudience.Private
-class FsVolumeImpl implements FsVolumeSpi {
+@VisibleForTesting
+public class FsVolumeImpl implements FsVolumeSpi {
   private final FsDatasetImpl dataset;
   private final String storageID;
   private final StorageType storageType;
@@ -58,6 +60,12 @@ class FsVolumeImpl implements FsVolumeSp
   private final File currentDir;    // <StorageDirectory>/current
   private final DF usage;           
   private final long reserved;
+
+  // Capacity configured. This is useful when we want to
+  // limit the visible capacity for tests. If negative, then we just
+  // query from the filesystem.
+  protected long configuredCapacity;
+
   /**
    * Per-volume worker pool that processes new blocks to cache.
    * The maximum number of workers per volume is bounded (configurable via
@@ -77,20 +85,26 @@ class FsVolumeImpl implements FsVolumeSp
     File parent = currentDir.getParentFile();
     this.usage = new DF(parent, conf);
     this.storageType = storageType;
+    this.configuredCapacity = -1;
+    cacheExecutor = initializeCacheExecutor(parent);
+  }
+
+  protected ThreadPoolExecutor initializeCacheExecutor(File parent) {
     final int maxNumThreads = dataset.datanode.getConf().getInt(
         DFSConfigKeys.DFS_DATANODE_FSDATASETCACHE_MAX_THREADS_PER_VOLUME_KEY,
-        DFSConfigKeys.DFS_DATANODE_FSDATASETCACHE_MAX_THREADS_PER_VOLUME_DEFAULT
-        );
+        DFSConfigKeys.DFS_DATANODE_FSDATASETCACHE_MAX_THREADS_PER_VOLUME_DEFAULT);
+
     ThreadFactory workerFactory = new ThreadFactoryBuilder()
         .setDaemon(true)
         .setNameFormat("FsVolumeImplWorker-" + parent.toString() + "-%d")
         .build();
-    cacheExecutor = new ThreadPoolExecutor(
+    ThreadPoolExecutor executor = new ThreadPoolExecutor(
         1, maxNumThreads,
         60, TimeUnit.SECONDS,
         new LinkedBlockingQueue<Runnable>(),
         workerFactory);
-    cacheExecutor.allowCoreThreadTimeOut(true);
+    executor.allowCoreThreadTimeOut(true);
+    return executor;
   }
   
   File getCurrentDir() {
@@ -129,9 +143,24 @@ class FsVolumeImpl implements FsVolumeSp
    * reserved capacity.
    * @return the unreserved number of bytes left in this filesystem. May be zero.
    */
-  long getCapacity() {
-    long remaining = usage.getCapacity() - reserved;
-    return remaining > 0 ? remaining : 0;
+  @VisibleForTesting
+  public long getCapacity() {
+    if (configuredCapacity < 0) {
+      long remaining = usage.getCapacity() - reserved;
+      return remaining > 0 ? remaining : 0;
+    }
+
+    return configuredCapacity;
+  }
+
+  /**
+   * This function MUST NOT be used outside of tests.
+   *
+   * @param capacity
+   */
+  @VisibleForTesting
+  public void setCapacityForTesting(long capacity) {
+    this.configuredCapacity = capacity;
   }
 
   @Override

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java?rev=1619975&r1=1619974&r2=1619975&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java Sat Aug 23 06:36:46 2014
@@ -91,7 +91,9 @@ import org.apache.hadoop.hdfs.server.dat
 import org.apache.hadoop.hdfs.server.datanode.SecureDataNodeStarter.SecureResources;
 import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsDatasetUtil;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsVolumeImpl;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
@@ -132,11 +134,15 @@ public class MiniDFSCluster {
   public static final String  DFS_NAMENODE_SAFEMODE_EXTENSION_TESTING_KEY
       = DFS_NAMENODE_SAFEMODE_EXTENSION_KEY + ".testing";
 
-  // Changing this value may break some tests that assume it is 2.
-  public static final int DIRS_PER_DATANODE = 2;
+  // Changing this default may break some tests that assume it is 2.
+  private static final int DEFAULT_STORAGES_PER_DATANODE = 2;
 
   static { DefaultMetricsSystem.setMiniClusterMode(true); }
 
+  public int getStoragesPerDatanode() {
+    return storagesPerDatanode;
+  }
+
   /**
    * Class to construct instances of MiniDFSClusters with specific options.
    */
@@ -146,6 +152,8 @@ public class MiniDFSCluster {
     private final Configuration conf;
     private int numDataNodes = 1;
     private StorageType[][] storageTypes = null;
+    private StorageType[] storageTypes1D = null;
+    private int storagesPerDatanode = DEFAULT_STORAGES_PER_DATANODE;
     private boolean format = true;
     private boolean manageNameDfsDirs = true;
     private boolean manageNameDfsSharedDirs = true;
@@ -156,6 +164,8 @@ public class MiniDFSCluster {
     private String[] racks = null; 
     private String [] hosts = null;
     private long [] simulatedCapacities = null;
+    private long [][] storageCapacities = null;
+    private long [] storageCapacities1D = null;
     private String clusterId = null;
     private boolean waitSafeMode = true;
     private boolean setupHostsFile = false;
@@ -194,16 +204,20 @@ public class MiniDFSCluster {
     }
 
     /**
+     * Default: DEFAULT_STORAGES_PER_DATANODE
+     */
+    public Builder storagesPerDatanode(int numStorages) {
+      this.storagesPerDatanode = numStorages;
+      return this;
+    }
+
+    /**
      * Set the same storage type configuration for each datanode.
      * If storageTypes is uninitialized or passed null then
      * StorageType.DEFAULT is used.
      */
     public Builder storageTypes(StorageType[] types) {
-      assert types.length == DIRS_PER_DATANODE;
-      this.storageTypes = new StorageType[numDataNodes][types.length];
-      for (int i = 0; i < numDataNodes; ++i) {
-        this.storageTypes[i] = types;
-      }
+      this.storageTypes1D = types;
       return this;
     }
 
@@ -218,6 +232,26 @@ public class MiniDFSCluster {
     }
 
     /**
+     * Set the same storage capacity configuration for each datanode.
+     * If storageTypes is uninitialized or passed null then
+     * StorageType.DEFAULT is used.
+     */
+    public Builder storageCapacities(long[] capacities) {
+      this.storageCapacities1D = capacities;
+      return this;
+    }
+
+    /**
+     * Set custom storage capacity configuration for each datanode.
+     * If storageCapacities is uninitialized or passed null then
+     * capacity is limited by available disk space.
+     */
+    public Builder storageCapacities(long[][] capacities) {
+      this.storageCapacities = capacities;
+      return this;
+    }
+
+    /**
      * Default: true
      */
     public Builder format(boolean val) {
@@ -290,6 +324,11 @@ public class MiniDFSCluster {
     }
 
     /**
+     * Use SimulatedFSDataset and limit the capacity of each DN per
+     * the values passed in val.
+     *
+     * For limiting the capacity of volumes with real storage, see
+     * {@link FsVolumeImpl#setCapacityForTesting}
      * Default: null
      */
     public Builder simulatedCapacities(long[] val) {
@@ -392,7 +431,28 @@ public class MiniDFSCluster {
     LOG.info("starting cluster: numNameNodes=" + numNameNodes
         + ", numDataNodes=" + builder.numDataNodes);
     nameNodes = new NameNodeInfo[numNameNodes];
+    this.storagesPerDatanode = builder.storagesPerDatanode;
+
+    // Duplicate the storageType setting for each DN.
+    if (builder.storageTypes == null && builder.storageTypes1D != null) {
+      assert builder.storageTypes1D.length == storagesPerDatanode;
+      builder.storageTypes = new StorageType[builder.numDataNodes][storagesPerDatanode];
       
+      for (int i = 0; i < builder.numDataNodes; ++i) {
+        builder.storageTypes[i] = builder.storageTypes1D;
+      }
+    }
+
+    // Duplicate the storageCapacity setting for each DN.
+    if (builder.storageCapacities == null && builder.storageCapacities1D != null) {
+      assert builder.storageCapacities1D.length == storagesPerDatanode;
+      builder.storageCapacities = new long[builder.numDataNodes][storagesPerDatanode];
+
+      for (int i = 0; i < builder.numDataNodes; ++i) {
+        builder.storageCapacities[i] = builder.storageCapacities1D;
+      }
+    }
+
     initMiniDFSCluster(builder.conf,
                        builder.numDataNodes,
                        builder.storageTypes,
@@ -405,6 +465,7 @@ public class MiniDFSCluster {
                        builder.dnOption,
                        builder.racks,
                        builder.hosts,
+                       builder.storageCapacities,
                        builder.simulatedCapacities,
                        builder.clusterId,
                        builder.waitSafeMode,
@@ -447,6 +508,7 @@ public class MiniDFSCluster {
   private boolean waitSafeMode = true;
   private boolean federation;
   private boolean checkExitOnShutdown = true;
+  protected final int storagesPerDatanode;
   
   /**
    * A unique instance identifier for the cluster. This
@@ -485,6 +547,7 @@ public class MiniDFSCluster {
    */
   public MiniDFSCluster() {
     nameNodes = new NameNodeInfo[0]; // No namenode in the cluster
+    storagesPerDatanode = DEFAULT_STORAGES_PER_DATANODE;
     synchronized (MiniDFSCluster.class) {
       instanceId = instanceCount++;
     }
@@ -659,11 +722,12 @@ public class MiniDFSCluster {
                         String[] racks, String hosts[],
                         long[] simulatedCapacities) throws IOException {
     this.nameNodes = new NameNodeInfo[1]; // Single namenode in the cluster
+    this.storagesPerDatanode = DEFAULT_STORAGES_PER_DATANODE;
     initMiniDFSCluster(conf, numDataNodes, null, format,
-        manageNameDfsDirs, true, manageDataDfsDirs, manageDataDfsDirs, 
-        operation, null, racks, hosts,
-        simulatedCapacities, null, true, false,
-        MiniDFSNNTopology.simpleSingleNN(nameNodePort, 0), true, false, false, null);
+                       manageNameDfsDirs, true, manageDataDfsDirs, manageDataDfsDirs,
+                       operation, null, racks, hosts,
+                       null, simulatedCapacities, null, true, false,
+                       MiniDFSNNTopology.simpleSingleNN(nameNodePort, 0), true, false, false, null);
   }
 
   private void initMiniDFSCluster(
@@ -672,7 +736,8 @@ public class MiniDFSCluster {
       boolean manageNameDfsSharedDirs, boolean enableManagedDfsDirsRedundancy,
       boolean manageDataDfsDirs, StartupOption startOpt,
       StartupOption dnStartOpt, String[] racks,
-      String[] hosts, long[] simulatedCapacities, String clusterId,
+      String[] hosts,
+      long[][] storageCapacities, long[] simulatedCapacities, String clusterId,
       boolean waitSafeMode, boolean setupHostsFile,
       MiniDFSNNTopology nnTopology, boolean checkExitOnShutdown,
       boolean checkDataNodeAddrConfig,
@@ -746,7 +811,7 @@ public class MiniDFSCluster {
       // Start the DataNodes
       startDataNodes(conf, numDataNodes, storageTypes, manageDataDfsDirs,
           dnStartOpt != null ? dnStartOpt : startOpt,
-          racks, hosts, simulatedCapacities, setupHostsFile,
+          racks, hosts, storageCapacities, simulatedCapacities, setupHostsFile,
           checkDataNodeAddrConfig, checkDataNodeHostConfig, dnConfOverlays);
       waitClusterUp();
       //make sure ProxyUsers uses the latest conf
@@ -1121,8 +1186,8 @@ public class MiniDFSCluster {
 
   String makeDataNodeDirs(int dnIndex, StorageType[] storageTypes) throws IOException {
     StringBuilder sb = new StringBuilder();
-    assert storageTypes == null || storageTypes.length == DIRS_PER_DATANODE;
-    for (int j = 0; j < DIRS_PER_DATANODE; ++j) {
+    assert storageTypes == null || storageTypes.length == storagesPerDatanode;
+    for (int j = 0; j < storagesPerDatanode; ++j) {
       File dir = getInstanceStorageDir(dnIndex, j);
       dir.mkdirs();
       if (!dir.isDirectory()) {
@@ -1198,7 +1263,7 @@ public class MiniDFSCluster {
                              long[] simulatedCapacities,
                              boolean setupHostsFile) throws IOException {
     startDataNodes(conf, numDataNodes, null, manageDfsDirs, operation, racks, hosts,
-        simulatedCapacities, setupHostsFile, false, false, null);
+        null, simulatedCapacities, setupHostsFile, false, false, null);
   }
 
   public synchronized void startDataNodes(Configuration conf, int numDataNodes,
@@ -1208,7 +1273,7 @@ public class MiniDFSCluster {
       boolean setupHostsFile,
       boolean checkDataNodeAddrConfig) throws IOException {
     startDataNodes(conf, numDataNodes, null, manageDfsDirs, operation, racks, hosts,
-        simulatedCapacities, setupHostsFile, checkDataNodeAddrConfig, false, null);
+        null, simulatedCapacities, setupHostsFile, checkDataNodeAddrConfig, false, null);
   }
 
   /**
@@ -1242,12 +1307,15 @@ public class MiniDFSCluster {
   public synchronized void startDataNodes(Configuration conf, int numDataNodes,
       StorageType[][] storageTypes, boolean manageDfsDirs, StartupOption operation,
       String[] racks, String[] hosts,
+      long[][] storageCapacities,
       long[] simulatedCapacities,
       boolean setupHostsFile,
       boolean checkDataNodeAddrConfig,
       boolean checkDataNodeHostConfig,
       Configuration[] dnConfOverlays) throws IOException {
+    assert storageCapacities == null || simulatedCapacities == null;
     assert storageTypes == null || storageTypes.length == numDataNodes;
+    assert storageCapacities == null || storageCapacities.length == numDataNodes;
 
     if (operation == StartupOption.RECOVER) {
       return;
@@ -1300,7 +1368,7 @@ public class MiniDFSCluster {
                         operation != StartupOption.ROLLBACK) ?
         null : new String[] {operation.getName()};
     
-    
+    DataNode[] dns = new DataNode[numDataNodes];
     for (int i = curDatanodesNum; i < curDatanodesNum+numDataNodes; i++) {
       Configuration dnConf = new HdfsConfiguration(conf);
       if (dnConfOverlays != null) {
@@ -1391,10 +1459,24 @@ public class MiniDFSCluster {
       dn.runDatanodeDaemon();
       dataNodes.add(new DataNodeProperties(dn, newconf, dnArgs,
           secureResources, dn.getIpcPort()));
+      dns[i - curDatanodesNum] = dn;
     }
     curDatanodesNum += numDataNodes;
     this.numDataNodes += numDataNodes;
     waitActive();
+
+    if (storageCapacities != null) {
+      for (int i = curDatanodesNum; i < curDatanodesNum+numDataNodes; ++i) {
+        List<? extends FsVolumeSpi> volumes = dns[i].getFSDataset().getVolumes();
+        assert storageCapacities[i].length == storagesPerDatanode;
+        assert volumes.size() == storagesPerDatanode;
+
+        for (int j = 0; j < volumes.size(); ++j) {
+          FsVolumeImpl volume = (FsVolumeImpl) volumes.get(j);
+          volume.setCapacityForTesting(storageCapacities[i][j]);
+        }
+      }
+    }
   }
   
   

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSClusterWithNodeGroup.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSClusterWithNodeGroup.java?rev=1619975&r1=1619974&r2=1619975&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSClusterWithNodeGroup.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSClusterWithNodeGroup.java Sat Aug 23 06:36:46 2014
@@ -22,6 +22,7 @@ import static org.apache.hadoop.hdfs.ser
 
 import java.io.File;
 import java.io.IOException;
+import java.util.List;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -31,6 +32,8 @@ import org.apache.hadoop.hdfs.server.dat
 import org.apache.hadoop.hdfs.server.datanode.SecureDataNodeStarter;
 import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset;
 import org.apache.hadoop.hdfs.server.datanode.SecureDataNodeStarter.SecureResources;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsVolumeImpl;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.net.StaticMapping;
 import org.apache.hadoop.security.UserGroupInformation;
@@ -52,11 +55,15 @@ public class MiniDFSClusterWithNodeGroup
   public synchronized void startDataNodes(Configuration conf, int numDataNodes,
       StorageType[][] storageTypes, boolean manageDfsDirs, StartupOption operation,
       String[] racks, String[] nodeGroups, String[] hosts,
+      long[][] storageCapacities,
       long[] simulatedCapacities,
       boolean setupHostsFile,
       boolean checkDataNodeAddrConfig,
       boolean checkDataNodeHostConfig) throws IOException {
+
+    assert storageCapacities == null || simulatedCapacities == null;
     assert storageTypes == null || storageTypes.length == numDataNodes;
+    assert storageCapacities == null || storageCapacities.length == numDataNodes;
 
     if (operation == StartupOption.RECOVER) {
       return;
@@ -109,6 +116,7 @@ public class MiniDFSClusterWithNodeGroup
     operation != StartupOption.ROLLBACK) ?
         null : new String[] {operation.getName()};
 
+    DataNode[] dns = new DataNode[numDataNodes];
     for (int i = curDatanodesNum; i < curDatanodesNum+numDataNodes; i++) {
       Configuration dnConf = new HdfsConfiguration(conf);
       // Set up datanode address
@@ -181,10 +189,23 @@ public class MiniDFSClusterWithNodeGroup
       }
       dn.runDatanodeDaemon();
       dataNodes.add(new DataNodeProperties(dn, newconf, dnArgs, secureResources, dn.getIpcPort()));
+      dns[i - curDatanodesNum] = dn;
     }
     curDatanodesNum += numDataNodes;
     this.numDataNodes += numDataNodes;
     waitActive();
+
+    if (storageCapacities != null) {
+      for (int i = curDatanodesNum; i < curDatanodesNum+numDataNodes; ++i) {
+        List<? extends FsVolumeSpi> volumes = dns[i].getFSDataset().getVolumes();
+        assert volumes.size() == storagesPerDatanode;
+
+        for (int j = 0; j < volumes.size(); ++j) {
+          FsVolumeImpl volume = (FsVolumeImpl) volumes.get(j);
+          volume.setCapacityForTesting(storageCapacities[i][j]);
+        }
+      }
+    }
   }
 
   public synchronized void startDataNodes(Configuration conf, int numDataNodes, 
@@ -193,7 +214,7 @@ public class MiniDFSClusterWithNodeGroup
       long[] simulatedCapacities,
       boolean setupHostsFile) throws IOException {
     startDataNodes(conf, numDataNodes, null, manageDfsDirs, operation, racks, nodeGroups,
-        hosts, simulatedCapacities, setupHostsFile, false, false);
+        hosts, null, simulatedCapacities, setupHostsFile, false, false);
   }
 
   public void startDataNodes(Configuration conf, int numDataNodes, 
@@ -209,13 +230,14 @@ public class MiniDFSClusterWithNodeGroup
   public synchronized void startDataNodes(Configuration conf, int numDataNodes, 
       StorageType[][] storageTypes, boolean manageDfsDirs, StartupOption operation,
       String[] racks, String[] hosts,
+      long[][] storageCapacities,
       long[] simulatedCapacities,
       boolean setupHostsFile,
       boolean checkDataNodeAddrConfig,
       boolean checkDataNodeHostConfig,
       Configuration[] dnConfOverlays) throws IOException {
     startDataNodes(conf, numDataNodes, storageTypes, manageDfsDirs, operation, racks,
-        NODE_GROUPS, hosts, simulatedCapacities, setupHostsFile, 
+        NODE_GROUPS, hosts, storageCapacities, simulatedCapacities, setupHostsFile,
         checkDataNodeAddrConfig, checkDataNodeHostConfig);
   }
 

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSafeMode.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSafeMode.java?rev=1619975&r1=1619974&r2=1619975&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSafeMode.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSafeMode.java Sat Aug 23 06:36:46 2014
@@ -213,7 +213,7 @@ public class TestSafeMode {
       @Override
       public Boolean get() {
         return getLongCounter("StorageBlockReportOps", getMetrics(NN_METRICS)) ==
-            MiniDFSCluster.DIRS_PER_DATANODE;
+            cluster.getStoragesPerDatanode();
       }
     }, 10, 10000);
 

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockHasMultipleReplicasOnSameDN.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockHasMultipleReplicasOnSameDN.java?rev=1619975&r1=1619974&r2=1619975&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockHasMultipleReplicasOnSameDN.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockHasMultipleReplicasOnSameDN.java Sat Aug 23 06:36:46 2014
@@ -18,7 +18,6 @@
 package org.apache.hadoop.hdfs.server.datanode;
 
 import java.io.IOException;
-import java.net.InetSocketAddress;
 import java.util.ArrayList;
 
 
@@ -106,7 +105,7 @@ public class TestBlockHasMultipleReplica
     DataNode dn = cluster.getDataNodes().get(0);
     DatanodeRegistration dnReg = dn.getDNRegistrationForBP(bpid);
     StorageBlockReport reports[] =
-        new StorageBlockReport[MiniDFSCluster.DIRS_PER_DATANODE];
+        new StorageBlockReport[cluster.getStoragesPerDatanode()];
 
     ArrayList<Block> blocks = new ArrayList<Block>();
 
@@ -114,7 +113,7 @@ public class TestBlockHasMultipleReplica
       blocks.add(locatedBlock.getBlock().getLocalBlock());
     }
 
-    for (int i = 0; i < MiniDFSCluster.DIRS_PER_DATANODE; ++i) {
+    for (int i = 0; i < cluster.getStoragesPerDatanode(); ++i) {
       BlockListAsLongs bll = new BlockListAsLongs(blocks, null);
       FsVolumeSpi v = dn.getFSDataset().getVolumes().get(i);
       DatanodeStorage dns = new DatanodeStorage(v.getStorageID());

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDnRespectsBlockReportSplitThreshold.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDnRespectsBlockReportSplitThreshold.java?rev=1619975&r1=1619974&r2=1619975&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDnRespectsBlockReportSplitThreshold.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDnRespectsBlockReportSplitThreshold.java Sat Aug 23 06:36:46 2014
@@ -130,7 +130,7 @@ public class TestDnRespectsBlockReportSp
     ArgumentCaptor<StorageBlockReport[]> captor =
         ArgumentCaptor.forClass(StorageBlockReport[].class);
 
-    Mockito.verify(nnSpy, times(MiniDFSCluster.DIRS_PER_DATANODE)).blockReport(
+    Mockito.verify(nnSpy, times(cluster.getStoragesPerDatanode())).blockReport(
         any(DatanodeRegistration.class),
         anyString(),
         captor.capture());
@@ -167,7 +167,7 @@ public class TestDnRespectsBlockReportSp
         anyString(),
         captor.capture());
 
-    verifyCapturedArguments(captor, MiniDFSCluster.DIRS_PER_DATANODE, BLOCKS_IN_FILE);
+    verifyCapturedArguments(captor, cluster.getStoragesPerDatanode(), BLOCKS_IN_FILE);
   }
 
   /**
@@ -194,7 +194,7 @@ public class TestDnRespectsBlockReportSp
     ArgumentCaptor<StorageBlockReport[]> captor =
         ArgumentCaptor.forClass(StorageBlockReport[].class);
 
-    Mockito.verify(nnSpy, times(MiniDFSCluster.DIRS_PER_DATANODE)).blockReport(
+    Mockito.verify(nnSpy, times(cluster.getStoragesPerDatanode())).blockReport(
         any(DatanodeRegistration.class),
         anyString(),
         captor.capture());

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/metrics/TestNameNodeMetrics.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/metrics/TestNameNodeMetrics.java?rev=1619975&r1=1619974&r2=1619975&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/metrics/TestNameNodeMetrics.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/metrics/TestNameNodeMetrics.java Sat Aug 23 06:36:46 2014
@@ -444,7 +444,7 @@ public class TestNameNodeMetrics {
     assertCounter("SyncsNumOps", 1L, rb);
     // Each datanode reports in when the cluster comes up
     assertCounter("BlockReportNumOps",
-                  (long)DATANODE_COUNT*MiniDFSCluster.DIRS_PER_DATANODE, rb);
+                  (long)DATANODE_COUNT * cluster.getStoragesPerDatanode(), rb);
     
     // Sleep for an interval+slop to let the percentiles rollover
     Thread.sleep((PERCENTILES_INTERVAL+1)*1000);