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 cm...@apache.org on 2013/09/09 20:53:01 UTC

svn commit: r1521240 - in /hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs: ./ src/main/java/org/apache/hadoop/hdfs/ src/main/java/org/apache/hadoop/hdfs/protocol/ src/main/java/org/apache/hadoop/hdfs/server/namenode/ src/main/java/org...

Author: cmccabe
Date: Mon Sep  9 18:53:01 2013
New Revision: 1521240

URL: http://svn.apache.org/r1521240
Log:
HDFS-5120. Add command-line support for manipulating cache pools.

Modified:
    hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-4949.txt
    hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
    hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
    hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/CacheManager.java
    hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/CachePool.java
    hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java
    hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testHDFSConf.xml

Modified: hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-4949.txt
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-4949.txt?rev=1521240&r1=1521239&r2=1521240&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-4949.txt (original)
+++ hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-4949.txt Mon Sep  9 18:53:01 2013
@@ -27,6 +27,10 @@ HDFS-4949 (Unreleased)
     HDFS-5163. Miscellaneous cache pool RPC fixes.  (Contributed by Colin
     Patrick McCabe)
 
+    HDFS-5120. Add command-line support for manipulating cache pools.
+    (Contributed by Colin Patrick McCabe)
+
+
   OPTIMIZATIONS
 
   BUG FIXES

Modified: hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java?rev=1521240&r1=1521239&r2=1521240&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java (original)
+++ hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java Mon Sep  9 18:53:01 2013
@@ -57,6 +57,7 @@ import org.apache.hadoop.fs.permission.F
 import org.apache.hadoop.hdfs.client.HdfsAdmin;
 import org.apache.hadoop.hdfs.client.HdfsDataInputStream;
 import org.apache.hadoop.hdfs.client.HdfsDataOutputStream;
+import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
@@ -1578,5 +1579,52 @@ public class DistributedFileSystem exten
       }
     }.resolve(this, absF);
   }
-  
+
+  /**
+   * Add a cache pool.
+   *
+   * @param req
+   *          The request to add a cache pool.
+   * @throws IOException 
+   *          If the request could not be completed.
+   */
+  public void addCachePool(CachePoolInfo info) throws IOException {
+    dfs.namenode.addCachePool(info);
+  }
+
+  /**
+   * Modify an existing cache pool.
+   *
+   * @param req
+   *          The request to modify a cache pool.
+   * @throws IOException 
+   *          If the request could not be completed.
+   */
+  public void modifyCachePool(CachePoolInfo info) throws IOException {
+    dfs.namenode.modifyCachePool(info);
+  }
+    
+  /**
+   * Remove a cache pool.
+   *
+   * @param cachePoolName
+   *          Name of the cache pool to remove.
+   * @throws IOException 
+   *          if the cache pool did not exist, or could not be removed.
+   */
+  public void removeCachePool(String name) throws IOException {
+    dfs.namenode.removeCachePool(name);
+  }
+
+  /**
+   * List all cache pools.
+   *
+   * @return A remote iterator from which you can get CachePoolInfo objects.
+   *          Requests will be made as needed.
+   * @throws IOException
+   *          If there was an error listing cache pools.
+   */
+  public RemoteIterator<CachePoolInfo> listCachePools() throws IOException {
+    return dfs.namenode.listCachePools("");
+  }
 }

Modified: hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java?rev=1521240&r1=1521239&r2=1521240&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java (original)
+++ hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java Mon Sep  9 18:53:01 2013
@@ -1144,7 +1144,7 @@ public interface ClientProtocol {
   public void addCachePool(CachePoolInfo info) throws IOException;
 
   /**
-   * Modify a cache pool.
+   * Modify an existing cache pool.
    *
    * @param req
    *          The request to modify a cache pool.

Modified: hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/CacheManager.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/CacheManager.java?rev=1521240&r1=1521239&r2=1521240&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/CacheManager.java (original)
+++ hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/CacheManager.java Mon Sep  9 18:53:01 2013
@@ -254,9 +254,7 @@ final class CacheManager {
   public synchronized void addCachePool(CachePoolInfo info)
       throws IOException {
     String poolName = info.getPoolName();
-    if (poolName.isEmpty()) {
-      throw new IOException("invalid empty cache pool name");
-    }
+    CachePool.validateName(poolName);
     CachePool pool = cachePools.get(poolName);
     if (pool != null) {
       throw new IOException("cache pool " + poolName + " already exists.");

Modified: hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/CachePool.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/CachePool.java?rev=1521240&r1=1521239&r2=1521240&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/CachePool.java (original)
+++ hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/CachePool.java Mon Sep  9 18:53:01 2013
@@ -40,6 +40,8 @@ import org.apache.hadoop.security.UserGr
 public final class CachePool {
   public static final Log LOG = LogFactory.getLog(CachePool.class);
 
+  public static final int DEFAULT_WEIGHT = 100;
+  
   @Nonnull
   private final String poolName;
 
@@ -152,4 +154,13 @@ public final class CachePool {
         append(", weight:").append(weight).
         append(" }").toString();
   }
+
+  public static void validateName(String name) throws IOException {
+    if (name.isEmpty()) {
+      // Empty pool names are not allowed because they would be highly
+      // confusing.  They would also break the ability to list all pools
+      // by starting with prevKey = ""
+      throw new IOException("invalid empty cache pool name");
+    }
+  }
 }

Modified: hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java?rev=1521240&r1=1521239&r2=1521240&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java (original)
+++ hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java Mon Sep  9 18:53:01 2013
@@ -24,6 +24,7 @@ import java.security.PrivilegedException
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashMap;
+import java.util.LinkedList;
 import java.util.List;
 import java.util.TreeSet;
 
@@ -36,6 +37,8 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FsShell;
 import org.apache.hadoop.fs.FsStatus;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.shell.Command;
 import org.apache.hadoop.fs.shell.CommandFormat;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
@@ -44,12 +47,14 @@ import org.apache.hadoop.hdfs.Distribute
 import org.apache.hadoop.hdfs.HAUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.NameNodeProxies;
+import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
 import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.SnapshotException;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
+import org.apache.hadoop.hdfs.server.namenode.CachePool;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.TransferFsImage;
 import org.apache.hadoop.ipc.RPC;
@@ -62,6 +67,8 @@ import org.apache.hadoop.security.author
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.ToolRunner;
 
+import com.google.common.base.Joiner;
+
 /**
  * This class provides some DFS administrative access shell commands.
  */
@@ -455,6 +462,234 @@ public class DFSAdmin extends FsShell {
     return exitCode;
   }
 
+  final private static String ADD_CACHE_POOL_USAGE =
+      "-addCachePool <name> [-owner <owner>] " +
+          "[-group <group>] [-mode <mode>] [-weight <weight>]";
+
+  public int addCachePool(String argsArray[], int idx) throws IOException {
+    List<String> args= new LinkedList<String>();
+    for (int i = idx; i < argsArray.length; i++) {
+      args.add(argsArray[i]);
+    }
+    String owner = StringUtils.popOptionWithArgument("-owner", args);
+    if (owner == null) {
+      owner = UserGroupInformation.getCurrentUser().getShortUserName();
+    }
+    String group = StringUtils.popOptionWithArgument("-group", args);
+    if (group == null) {
+      group = UserGroupInformation.getCurrentUser().getGroupNames()[0];
+    }
+    String modeString = StringUtils.popOptionWithArgument("-mode", args);
+    int mode;
+    if (modeString == null) {
+      mode = FsPermission.getCachePoolDefault().toShort();
+    } else {
+      mode = Integer.parseInt(modeString, 8);
+    }
+    String weightString = StringUtils.popOptionWithArgument("-weight", args);
+    int weight;
+    if (weightString == null) {
+      weight = CachePool.DEFAULT_WEIGHT;
+    } else {
+      weight = Integer.parseInt(weightString);
+    }
+    String name = StringUtils.popFirstNonOption(args);
+    if (name == null) {
+      System.err.println("You must specify a name when creating a " +
+          "cache pool.");
+      return 1;
+    }
+    if (!args.isEmpty()) {
+      System.err.print("Can't understand arguments: " +
+        Joiner.on(" ").join(args) + "\n");
+      System.err.println("Usage is " + ADD_CACHE_POOL_USAGE);
+      return 1;
+    }
+    DistributedFileSystem dfs = getDFS();
+    CachePoolInfo info = new CachePoolInfo(name).
+        setOwnerName(owner).
+        setGroupName(group).
+        setMode(new FsPermission((short)mode)).
+        setWeight(weight);
+    try {
+      dfs.addCachePool(info);
+    } catch (IOException e) {
+      throw new RemoteException(e.getClass().getName(), e.getMessage());
+    }
+    System.out.println("Successfully added cache pool " + name + ".");
+    return 0;
+  }
+
+  final private static String MODIFY_CACHE_POOL_USAGE =
+      "-modifyCachePool <name> [-owner <owner>] " +
+          "[-group <group>] [-mode <mode>] [-weight <weight>]";
+
+  public int modifyCachePool(String argsArray[], int idx) throws IOException {
+    List<String> args = new LinkedList<String>();
+    for (int i = idx; i < argsArray.length; i++) {
+      args.add(argsArray[i]);
+    }
+    String owner = StringUtils.popOptionWithArgument("-owner", args);
+    String group = StringUtils.popOptionWithArgument("-group", args);
+    String modeString = StringUtils.popOptionWithArgument("-mode", args);
+    Integer mode = (modeString == null) ?
+        null : Integer.parseInt(modeString, 8);
+    String weightString = StringUtils.popOptionWithArgument("-weight", args);
+    Integer weight = (weightString == null) ?
+        null : Integer.parseInt(weightString);
+    String name = StringUtils.popFirstNonOption(args);
+    if (name == null) {
+      System.err.println("You must specify a name when creating a " +
+          "cache pool.");
+      return 1;
+    }
+    if (!args.isEmpty()) {
+      System.err.print("Can't understand arguments: " +
+        Joiner.on(" ").join(args) + "\n");
+      System.err.println("usage is " + MODIFY_CACHE_POOL_USAGE);
+      return 1;
+    }
+    boolean changed = false;
+    CachePoolInfo info = new CachePoolInfo(name);
+    if (owner != null) {
+      info.setOwnerName(owner);
+      changed = true;
+    }
+    if (group != null) {
+      info.setGroupName(group);
+      changed = true;
+    }
+    if (mode != null) {
+      info.setMode(new FsPermission(mode.shortValue()));
+      changed = true;
+    }
+    if (weight != null) {
+      info.setWeight(weight);
+      changed = true;
+    }
+    if (!changed) {
+      System.err.println("You must specify at least one attribute to " +
+          "change in the cache pool.");
+      return 1;
+    }
+    DistributedFileSystem dfs = getDFS();
+    try {
+      dfs.modifyCachePool(info);
+    } catch (IOException e) {
+      throw new RemoteException(e.getClass().getName(), e.getMessage());
+    }
+    System.out.print("Successfully modified cache pool " + name);
+    String prefix = " to have ";
+    if (owner != null) {
+      System.out.print(prefix + "owner name " + owner);
+      prefix = "and ";
+    }
+    if (group != null) {
+      System.out.print(prefix + "group name " + group);
+      prefix = "and ";
+    }
+    if (mode != null) {
+      System.out.print(prefix + "mode " + new FsPermission(mode.shortValue()));
+      prefix = "and ";
+    }
+    if (weight != null) {
+      System.out.print(prefix + "weight " + weight);
+      prefix = "and ";
+    }
+    System.out.print("\n");
+    return 0;
+  }
+
+  final private static String REMOVE_CACHE_POOL_USAGE =
+      "-removeCachePool <name>";
+
+  public int removeCachePool(String argsArray[], int idx) throws IOException {
+    List<String> args = new LinkedList<String>();
+    for (int i = idx; i < argsArray.length; i++) {
+      args.add(argsArray[i]);
+    }
+    String name = StringUtils.popFirstNonOption(args);
+    if (name == null) {
+      System.err.println("You must specify a name when deleting a " +
+          "cache pool.");
+      return 1;
+    }
+    if (!args.isEmpty()) {
+      System.err.print("Can't understand arguments: " +
+        Joiner.on(" ").join(args) + "\n");
+      System.err.println("Usage is " + REMOVE_CACHE_POOL_USAGE);
+      return 1;
+    }
+    DistributedFileSystem dfs = getDFS();
+    try {
+      dfs.removeCachePool(name);
+    } catch (IOException e) {
+      dfs.removeCachePool(name);
+      throw new RemoteException(e.getClass().getName(), e.getMessage());
+    }
+    System.out.println("Successfully removed cache pool " + name + ".");
+    return 0;
+  }
+
+  final private static String LIST_CACHE_POOLS_USAGE =
+      "-listCachePools] [-verbose] [name]";
+
+  private void listCachePool(CachePoolInfo info) {
+    System.out.print(String.format("%s\n", info.getPoolName()));
+    System.out.print(String.format("owner:\t%s\n", info.getOwnerName()));
+    System.out.print(String.format("group:\t%s\n", info.getGroupName()));
+    System.out.print(String.format("mode:\t%s\n", info.getMode()));
+    System.out.print(String.format("weight:\t%d\n", info.getWeight()));
+    System.out.print("\n");
+  }
+
+  public int listCachePools(String argsArray[], int idx) throws IOException {
+    List<String> args = new LinkedList<String>();
+    for (int i = idx; i < argsArray.length; i++) {
+      args.add(argsArray[i]);
+    }
+    String name = StringUtils.popFirstNonOption(args);
+    if (!args.isEmpty()) {
+      System.err.print("Can't understand arguments: " +
+        Joiner.on(" ").join(args) + "\n");
+      System.err.println("usage is " + LIST_CACHE_POOLS_USAGE);
+      return 1;
+    }
+    boolean gotResults = false;
+    DistributedFileSystem dfs = getDFS();
+    try {
+      RemoteIterator<CachePoolInfo> iter = dfs.listCachePools();
+      if (name != null) {
+        while (iter.hasNext()) {
+          CachePoolInfo info = iter.next();
+          if (info.getPoolName().equals(name)) {
+            listCachePool(info);
+            gotResults = true;
+            return 0;
+          }
+        }
+      } else {
+        while (iter.hasNext()) {
+          listCachePool(iter.next());
+          gotResults = true;
+        }
+      }
+    } catch (IOException e) {
+      throw new RemoteException(e.getClass().getName(), e.getMessage());
+    }
+    int ret = 0;
+    if (!gotResults) {
+      if (name != null) {
+        System.out.println("No cache pool named " + name + " found.");
+        ret = 1;
+      } else {
+        System.out.println("No cache pools found.");
+        ret = 1;
+      }
+    }
+    return ret;
+  }
+
   public int rollEdits() throws IOException {
     DistributedFileSystem dfs = getDFS();
     long txid = dfs.rollEdits();
@@ -582,6 +817,10 @@ public class DFSAdmin extends FsShell {
       "\t[-fetchImage <local directory>]\n" +
       "\t[-allowSnapshot <snapshotDir>]\n" +
       "\t[-disallowSnapshot <snapshotDir>]\n" +
+      "\t[" + ADD_CACHE_POOL_USAGE + "]\n" +
+      "\t[" + MODIFY_CACHE_POOL_USAGE + "]\n" +
+      "\t[" + REMOVE_CACHE_POOL_USAGE + "]\n" +
+      "\t[" + LIST_CACHE_POOLS_USAGE + "]\n" +
       "\t[-help [cmd]]\n";
 
     String report ="-report: \tReports basic filesystem information and statistics.\n";
@@ -679,6 +918,42 @@ public class DFSAdmin extends FsShell {
     String disallowSnapshot = "-disallowSnapshot <snapshotDir>:\n" +
         "\tDo not allow snapshots to be taken on a directory any more.\n";
     
+    String addCachePool = ADD_CACHE_POOL_USAGE  + ": \n" +
+      "\tAdd a new cache pool.\n" + 
+      "\t<name> is the name of the new pool.  It must not already be used.\n" +
+      "\t<owner> is the owner of the pool.  It defaults to the current\n" +
+      "\tuser name.\n" +
+      "\t<group> is the group of the pool.  It defaults to the primary\n" +
+      "\tgroup name of the current user.\n" +
+      "\t<mode> is the mode of the pool.  This is a UNIX-style numeric mode\n" +
+      "\targument, supplied as an octal number.  For example, mode 0755\n" +
+      "\tgrants the owner all permissions, and grants everyone else\n" +
+      "\tonly read and list permissions.\n" +
+      "\tThe mode defaults to " +
+      String.format("0%03o",
+          FsPermission.getCachePoolDefault().toShort()) + "\n" +
+      "\t<weight> is the weight of the pool.  This determines what share \n" +
+      "\tof cluster resources the pool will get.  It defaults to " +
+      CachePool.DEFAULT_WEIGHT + "\n";
+
+    String modifyCachePool = MODIFY_CACHE_POOL_USAGE + ": \n" +
+      "\tAdd a new cache pool with the given name.\n" +
+      "\t<name> is the name of the pool to modify.\n" +
+      "\t<owner> is the new owner of the pool.\n" + 
+      "\t<group> is the new group of the pool.\n" + 
+      "\t<mode> is the new mode of the pool.\n" + 
+      "\t<weight> is the new weight of the pool.\n";
+
+    String removeCachePool = REMOVE_CACHE_POOL_USAGE + ": \n" +
+      "\tRemove a cache pool.\n" +
+      "\t<name> is the name of the pool to remove.\n";
+
+    String listCachePools = " -listCachePools [-name <name>] [-verbose]\n" + 
+      "\tList cache pools.\n" +
+      "\tIf <name> is specified, we will list only the cache pool with\n" +
+      "\tthat name.  If <verbose> is specified, we will list detailed\n" +
+      "\tinformation about each pool\n";
+
     String help = "-help [cmd]: \tDisplays help for the given command or all commands if none\n" +
       "\t\tis specified.\n";
 
@@ -726,6 +1001,14 @@ public class DFSAdmin extends FsShell {
       System.out.println(allowSnapshot);
     } else if ("disallowSnapshot".equalsIgnoreCase(cmd)) {
       System.out.println(disallowSnapshot);
+    } else if ("addCachePool".equalsIgnoreCase(cmd)) {
+      System.out.println(addCachePool);
+    } else if ("modifyCachePool".equalsIgnoreCase(cmd)) {
+      System.out.println(modifyCachePool);
+    } else if ("removeCachePool".equalsIgnoreCase(cmd)) {
+      System.out.println(removeCachePool);
+    } else if ("listCachePools".equalsIgnoreCase(cmd)) {
+      System.out.println(listCachePools);
     } else if ("help".equals(cmd)) {
       System.out.println(help);
     } else {
@@ -752,6 +1035,13 @@ public class DFSAdmin extends FsShell {
       System.out.println(fetchImage);
       System.out.println(allowSnapshot);
       System.out.println(disallowSnapshot);
+      System.out.println(addCachePool);
+      System.out.println(modifyCachePool);
+      System.out.println(removeCachePool);
+      System.out.println(listCachePools);
+
+      System.out.println(disallowSnapshot);
+
       System.out.println(help);
       System.out.println();
       ToolRunner.printGenericCommandUsage(System.out);
@@ -988,6 +1278,18 @@ public class DFSAdmin extends FsShell {
     } else if ("-fetchImage".equals(cmd)) {
       System.err.println("Usage: java DFSAdmin"
           + " [-fetchImage <local directory>]");
+    } else if ("-addCachePool".equals(cmd)) {
+      System.err.println("Usage: java DFSAdmin"
+          + " [" + ADD_CACHE_POOL_USAGE  + "]");
+    } else if ("-modifyCachePool".equals(cmd)) {
+      System.err.println("Usage: java DFSAdmin"
+          + " [" + MODIFY_CACHE_POOL_USAGE + "]");
+    } else if ("-removeCachePool".equals(cmd)) {
+      System.err.println("Usage: java DFSAdmin"
+          + " [" + REMOVE_CACHE_POOL_USAGE + "]");
+    } else if ("-listCachePools".equals(cmd)) {
+      System.err.println("Usage: java DFSAdmin"
+          + " [" + LIST_CACHE_POOLS_USAGE + "]");
     } else {
       System.err.println("Usage: java DFSAdmin");
       System.err.println("Note: Administrative commands can only be run as the HDFS superuser.");
@@ -1013,6 +1315,10 @@ public class DFSAdmin extends FsShell {
       System.err.println("           ["+ClearSpaceQuotaCommand.USAGE+"]");      
       System.err.println("           [-setBalancerBandwidth <bandwidth in bytes per second>]");
       System.err.println("           [-fetchImage <local directory>]");
+      System.err.println("           [" + ADD_CACHE_POOL_USAGE + "]");
+      System.err.println("           [" + MODIFY_CACHE_POOL_USAGE + "]");
+      System.err.println("           [" + REMOVE_CACHE_POOL_USAGE + "]");
+      System.err.println("           [" + LIST_CACHE_POOLS_USAGE + "]");
       System.err.println("           [-help [cmd]]");
       System.err.println();
       ToolRunner.printGenericCommandUsage(System.err);
@@ -1185,6 +1491,14 @@ public class DFSAdmin extends FsShell {
         exitCode = setBalancerBandwidth(argv, i);
       } else if ("-fetchImage".equals(cmd)) {
         exitCode = fetchImage(argv, i);
+      } else if ("-addCachePool".equals(cmd)) {
+        exitCode = addCachePool(argv, i);
+      } else if ("-modifyCachePool".equals(cmd)) {
+        exitCode = modifyCachePool(argv, i);
+      } else if ("-removeCachePool".equals(cmd)) {
+        exitCode = removeCachePool(argv, i);
+      } else if ("-listCachePools".equals(cmd)) {
+        exitCode = listCachePools(argv, i);
       } else if ("-help".equals(cmd)) {
         if (i < argv.length) {
           printHelp(argv[i]);

Modified: hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testHDFSConf.xml
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testHDFSConf.xml?rev=1521240&r1=1521239&r2=1521240&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testHDFSConf.xml (original)
+++ hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testHDFSConf.xml Mon Sep  9 18:53:01 2013
@@ -16356,7 +16356,7 @@
       </comparators>
     </test>
   
-  <test> <!--Tested -->
+    <test> <!--Tested -->
       <description>Verifying clrSpaceQuota operation is not permitted in safemode</description>
       <test-commands>
         <command>-fs NAMENODE -mkdir /test </command>
@@ -16374,5 +16374,70 @@
         </comparator>
       </comparators>
     </test>
+
+    <test> <!--Tested -->
+      <description>Testing listing no cache pools</description>
+      <test-commands>
+        <dfs-admin-command>-fs NAMENODE -listCachePools</dfs-admin-command>
+      </test-commands>
+      <cleanup-commands>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>No cache pools found.</expected-output>
+        </comparator>
+      </comparators>
+    </test>
+
+    <test> <!--Tested -->
+      <description>Testing adding a cache pool</description>
+      <test-commands>
+        <dfs-admin-command>-fs NAMENODE -addCachePool foo</dfs-admin-command>
+      </test-commands>
+      <cleanup-commands>
+        <dfs-admin-command>-fs NAMENODE -removeCachePool foo</dfs-admin-command>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>Successfully added cache pool foo.</expected-output>
+        </comparator>
+      </comparators>
+    </test>
+
+    <test> <!--Tested -->
+      <description>Testing deleting a cache pool</description>
+      <test-commands>
+        <dfs-admin-command>-fs NAMENODE -addCachePool foo</dfs-admin-command>
+        <dfs-admin-command>-fs NAMENODE -removeCachePool foo</dfs-admin-command>
+      </test-commands>
+      <cleanup-commands>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>Successfully removed cache pool foo.</expected-output>
+        </comparator>
+      </comparators>
+    </test>
+
+    <test> <!--Tested -->
+      <description>Testing listing a cache pool</description>
+      <test-commands>
+        <dfs-admin-command>-fs NAMENODE -addCachePool foo -owner bob -group bob -mode 0664</dfs-admin-command>
+        <dfs-admin-command>-fs NAMENODE -listCachePools foo</dfs-admin-command>
+      </test-commands>
+      <cleanup-commands>
+        <dfs-admin-command>-fs NAMENODE -removeCachePool foo</dfs-admin-command>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>foo</expected-output>
+        </comparator>
+      </comparators>
+    </test>
+
   </tests>
 </configuration>