Forráskód Böngészése

HDFS-5471. CacheAdmin -listPools fails when user lacks permissions to view all pools (Andrew Wang via Colin Patrick McCabe)

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1541323 13f79535-47bb-0310-9956-ffa450edef68
Colin McCabe 11 éve
szülő
commit
ce35e0950c

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

@@ -277,6 +277,9 @@ Trunk (Unreleased)
     HADOOP-9740. Fix FsShell '-text' command to be able to read Avro
     files stored in HDFS and other filesystems. (Allan Yan via cutting)
 
+    HDFS-5471. CacheAdmin -listPools fails when user lacks permissions to view
+    all pools (Andrew Wang via Colin Patrick McCabe)
+
   OPTIMIZATIONS
 
     HADOOP-7761. Improve the performance of raw comparisons. (todd)

+ 4 - 3
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/IdNotFoundException.java → hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/InvalidRequestException.java

@@ -20,12 +20,13 @@ package org.apache.hadoop.fs;
 import java.io.IOException;
 
 /**
- * Exception corresponding to ID not found - EINVAL
+ * Thrown when the user makes a malformed request, for example missing required
+ * parameters or parameters that are not valid.
  */
-public class IdNotFoundException extends IOException {
+public class InvalidRequestException extends IOException {
   static final long serialVersionUID = 0L;
 
-  public IdNotFoundException(String str) {
+  public InvalidRequestException(String str) {
     super(str);
   }
 }

+ 3 - 43
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java

@@ -1104,21 +1104,7 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
   public AddCachePoolResponseProto addCachePool(RpcController controller,
       AddCachePoolRequestProto request) throws ServiceException {
     try {
-      CachePoolInfo info =
-          new CachePoolInfo(request.getPoolName());
-      if (request.hasOwnerName()) {
-        info.setOwnerName(request.getOwnerName());
-      }
-      if (request.hasGroupName()) {
-        info.setGroupName(request.getGroupName());
-      }
-      if (request.hasMode()) {
-        info.setMode(new FsPermission((short)request.getMode()));
-      }
-      if (request.hasWeight()) {
-        info.setWeight(request.getWeight());
-      }
-      server.addCachePool(info);
+      server.addCachePool(PBHelper.convert(request.getInfo()));
       return AddCachePoolResponseProto.newBuilder().build();
     } catch (IOException e) {
       throw new ServiceException(e);
@@ -1129,21 +1115,7 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
   public ModifyCachePoolResponseProto modifyCachePool(RpcController controller,
       ModifyCachePoolRequestProto request) throws ServiceException {
     try {
-      CachePoolInfo info =
-          new CachePoolInfo(request.getPoolName());
-      if (request.hasOwnerName()) {
-        info.setOwnerName(request.getOwnerName());
-      }
-      if (request.hasGroupName()) {
-        info.setGroupName(request.getGroupName());
-      }
-      if (request.hasMode()) {
-        info.setMode(new FsPermission((short)request.getMode()));
-      }
-      if (request.hasWeight()) {
-        info.setWeight(request.getWeight());
-      }
-      server.modifyCachePool(info);
+      server.modifyCachePool(PBHelper.convert(request.getInfo()));
       return ModifyCachePoolResponseProto.newBuilder().build();
     } catch (IOException e) {
       throw new ServiceException(e);
@@ -1174,19 +1146,7 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
         CachePoolInfo pool = iter.next();
         ListCachePoolsResponseElementProto.Builder elemBuilder = 
             ListCachePoolsResponseElementProto.newBuilder();
-        elemBuilder.setPoolName(pool.getPoolName());
-        if (pool.getOwnerName() != null) {
-          elemBuilder.setOwnerName(pool.getOwnerName());
-        }
-        if (pool.getGroupName() != null) {
-          elemBuilder.setGroupName(pool.getGroupName());
-        }
-        if (pool.getMode() != null) {
-          elemBuilder.setMode(pool.getMode().toShort());
-        }
-        if (pool.getWeight() != null) {
-          elemBuilder.setWeight(pool.getWeight());
-        }
+        elemBuilder.setInfo(PBHelper.convert(pool));
         responseBuilder.addElements(elemBuilder.build());
         prevPoolName = pool.getPoolName();
       }

+ 3 - 31
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java

@@ -1109,19 +1109,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
   public void addCachePool(CachePoolInfo info) throws IOException {
     AddCachePoolRequestProto.Builder builder = 
         AddCachePoolRequestProto.newBuilder();
-    builder.setPoolName(info.getPoolName());
-    if (info.getOwnerName() != null) {
-      builder.setOwnerName(info.getOwnerName());
-    }
-    if (info.getGroupName() != null) {
-      builder.setGroupName(info.getGroupName());
-    }
-    if (info.getMode() != null) {
-      builder.setMode(info.getMode().toShort());
-    }
-    if (info.getWeight() != null) {
-      builder.setWeight(info.getWeight());
-    }
+    builder.setInfo(PBHelper.convert(info));
     try {
       rpcProxy.addCachePool(null, builder.build());
     } catch (ServiceException e) {
@@ -1133,19 +1121,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
   public void modifyCachePool(CachePoolInfo req) throws IOException {
     ModifyCachePoolRequestProto.Builder builder = 
         ModifyCachePoolRequestProto.newBuilder();
-    builder.setPoolName(req.getPoolName());
-    if (req.getOwnerName() != null) {
-      builder.setOwnerName(req.getOwnerName());
-    }
-    if (req.getGroupName() != null) {
-      builder.setGroupName(req.getGroupName());
-    }
-    if (req.getMode() != null) {
-      builder.setMode(req.getMode().toShort());
-    }
-    if (req.getWeight() != null) {
-      builder.setWeight(req.getWeight());
-    }
+    builder.setInfo(PBHelper.convert(req));
     try {
       rpcProxy.modifyCachePool(null, builder.build());
     } catch (ServiceException e) {
@@ -1175,11 +1151,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
     @Override
     public CachePoolInfo get(int i) {
       ListCachePoolsResponseElementProto elem = proto.getElements(i);
-      return new CachePoolInfo(elem.getPoolName()).
-          setOwnerName(elem.getOwnerName()).
-          setGroupName(elem.getGroupName()).
-          setMode(new FsPermission((short)elem.getMode())).
-          setWeight(elem.getWeight());
+      return PBHelper.convert(elem.getInfo());
     }
 
     @Override

+ 41 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java

@@ -17,6 +17,8 @@
  */
 package org.apache.hadoop.hdfs.protocolPB;
 
+import static com.google.common.base.Preconditions.checkNotNull;
+
 import java.io.EOFException;
 import java.io.IOException;
 import java.io.InputStream;
@@ -34,6 +36,7 @@ import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
 import org.apache.hadoop.ha.proto.HAServiceProtocolProtos;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.CorruptFileBlocks;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
@@ -53,6 +56,7 @@ import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CachePoolInfoProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateFlagProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.DatanodeReportTypeProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFsStatsResponseProto;
@@ -1602,6 +1606,43 @@ public class PBHelper {
     return builder.build();
   }
   
+  public static CachePoolInfoProto convert(CachePoolInfo info) {
+    CachePoolInfoProto.Builder builder = CachePoolInfoProto.newBuilder();
+    builder.setPoolName(info.getPoolName());
+    if (info.getOwnerName() != null) {
+      builder.setOwnerName(info.getOwnerName());
+    }
+    if (info.getGroupName() != null) {
+      builder.setGroupName(info.getGroupName());
+    }
+    if (info.getMode() != null) {
+      builder.setMode(info.getMode().toShort());
+    }
+    if (info.getWeight() != null) {
+      builder.setWeight(info.getWeight());
+    }
+    return builder.build();
+  }
+
+  public static CachePoolInfo convert (CachePoolInfoProto proto) {
+    // Pool name is a required field, the rest are optional
+    String poolName = checkNotNull(proto.getPoolName());
+    CachePoolInfo info = new CachePoolInfo(poolName);
+    if (proto.hasOwnerName()) {
+        info.setOwnerName(proto.getOwnerName());
+    }
+    if (proto.hasGroupName()) {
+      info.setGroupName(proto.getGroupName());
+    }
+    if (proto.hasMode()) {
+      info.setMode(new FsPermission((short)proto.getMode()));
+    }
+    if (proto.hasWeight()) {
+      info.setWeight(proto.getWeight());
+    }
+    return info;
+  }
+
   public static HdfsProtos.ChecksumTypeProto convert(DataChecksum.Type type) {
     return HdfsProtos.ChecksumTypeProto.valueOf(type.id);
   }

+ 124 - 118
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/CacheManager.java

@@ -43,7 +43,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.IdNotFoundException;
+import org.apache.hadoop.fs.InvalidRequestException;
 import org.apache.hadoop.fs.BatchedRemoteIterator.BatchedListEntries;
 import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.hdfs.DFSUtil;
@@ -250,11 +250,87 @@ public final class CacheManager {
   private long getNextEntryId() throws IOException {
     assert namesystem.hasWriteLock();
     if (nextEntryId == Long.MAX_VALUE) {
-      throw new IOException("No more available IDs");
+      throw new IOException("No more available IDs.");
     }
     return nextEntryId++;
   }
 
+  // Helper getter / validation methods
+
+  private static void checkWritePermission(FSPermissionChecker pc,
+      CachePool pool) throws AccessControlException {
+    if ((pc != null)) {
+      pc.checkPermission(pool, FsAction.WRITE);
+    }
+  }
+
+  private static String validatePoolName(PathBasedCacheDirective directive)
+      throws InvalidRequestException {
+    String pool = directive.getPool();
+    if (pool == null) {
+      throw new InvalidRequestException("No pool specified.");
+    }
+    if (pool.isEmpty()) {
+      throw new InvalidRequestException("Invalid empty pool name.");
+    }
+    return pool;
+  }
+
+  private static String validatePath(PathBasedCacheDirective directive)
+      throws InvalidRequestException {
+    if (directive.getPath() == null) {
+      throw new InvalidRequestException("No path specified.");
+    }
+    String path = directive.getPath().toUri().getPath();
+    if (!DFSUtil.isValidName(path)) {
+      throw new InvalidRequestException("Invalid path '" + path + "'.");
+    }
+    return path;
+  }
+
+  private static short validateReplication(PathBasedCacheDirective directive,
+      short defaultValue) throws InvalidRequestException {
+    short repl = (directive.getReplication() != null)
+        ? directive.getReplication() : defaultValue;
+    if (repl <= 0) {
+      throw new InvalidRequestException("Invalid replication factor " + repl
+          + " <= 0");
+    }
+    return repl;
+  }
+
+  /**
+   * Get a PathBasedCacheEntry by ID, validating the ID and that the entry
+   * exists.
+   */
+  private PathBasedCacheEntry getById(long id) throws InvalidRequestException {
+    // Check for invalid IDs.
+    if (id <= 0) {
+      throw new InvalidRequestException("Invalid negative ID.");
+    }
+    // Find the entry.
+    PathBasedCacheEntry entry = entriesById.get(id);
+    if (entry == null) {
+      throw new InvalidRequestException("No directive with ID " + id
+          + " found.");
+    }
+    return entry;
+  }
+
+  /**
+   * Get a CachePool by name, validating that it exists.
+   */
+  private CachePool getCachePool(String poolName)
+      throws InvalidRequestException {
+    CachePool pool = cachePools.get(poolName);
+    if (pool == null) {
+      throw new InvalidRequestException("Unknown pool " + poolName);
+    }
+    return pool;
+  }
+
+  // RPC handlers
+
   private void addInternal(PathBasedCacheEntry entry) {
     entriesById.put(entry.getEntryId(), entry);
     String path = entry.getPath();
@@ -272,34 +348,10 @@ public final class CacheManager {
     assert namesystem.hasWriteLock();
     PathBasedCacheEntry entry;
     try {
-      if (directive.getPool() == null) {
-        throw new IdNotFoundException("addDirective: no pool was specified.");
-      }
-      if (directive.getPool().isEmpty()) {
-        throw new IdNotFoundException("addDirective: pool name was empty.");
-      }
-      CachePool pool = cachePools.get(directive.getPool());
-      if (pool == null) {
-        throw new IdNotFoundException("addDirective: no such pool as " +
-            directive.getPool());
-      }
-      if ((pc != null) && (!pc.checkPermission(pool, FsAction.WRITE))) {
-        throw new AccessControlException("addDirective: write " +
-            "permission denied for pool " + directive.getPool());
-      }
-      if (directive.getPath() == null) {
-        throw new IOException("addDirective: no path was specified.");
-      }
-      String path = directive.getPath().toUri().getPath();
-      if (!DFSUtil.isValidName(path)) {
-        throw new IOException("addDirective: path '" + path + "' is invalid.");
-      }
-      short replication = directive.getReplication() == null ? 
-          (short)1 : directive.getReplication();
-      if (replication <= 0) {
-        throw new IOException("addDirective: replication " + replication +
-            " is invalid.");
-      }
+      CachePool pool = getCachePool(validatePoolName(directive));
+      checkWritePermission(pc, pool);
+      String path = validatePath(directive);
+      short replication = validateReplication(directive, (short)1);
       long id;
       if (directive.getId() != null) {
         // We are loading an entry from the edit log.
@@ -312,10 +364,10 @@ public final class CacheManager {
       entry = new PathBasedCacheEntry(id, path, replication, pool);
       addInternal(entry);
     } catch (IOException e) {
-      LOG.warn("addDirective " + directive + ": failed.", e);
+      LOG.warn("addDirective of " + directive + " failed: ", e);
       throw e;
     }
-    LOG.info("addDirective " + directive + ": succeeded.");
+    LOG.info("addDirective of " + directive + " successful.");
     if (monitor != null) {
       monitor.kick();
     }
@@ -332,75 +384,43 @@ public final class CacheManager {
       // Check for invalid IDs.
       Long id = directive.getId();
       if (id == null) {
-        throw new IdNotFoundException("modifyDirective: " +
-            "no ID to modify was supplied.");
-      }
-      if (id <= 0) {
-        throw new IdNotFoundException("modifyDirective " + id +
-            ": invalid non-positive directive ID.");
-      }
-      // Find the entry.
-      PathBasedCacheEntry prevEntry = entriesById.get(id);
-      if (prevEntry == null) {
-        throw new IdNotFoundException("modifyDirective " + id +
-            ": id not found.");
-      }
-      if ((pc != null) &&
-          (!pc.checkPermission(prevEntry.getPool(), FsAction.WRITE))) {
-        throw new AccessControlException("modifyDirective " + id +
-            ": permission denied for initial pool " + prevEntry.getPool());
+        throw new InvalidRequestException("Must supply an ID.");
       }
+      PathBasedCacheEntry prevEntry = getById(id);
+      checkWritePermission(pc, prevEntry.getPool());
       String path = prevEntry.getPath();
       if (directive.getPath() != null) {
-        path = directive.getPath().toUri().getPath();
-        if (!DFSUtil.isValidName(path)) {
-          throw new IOException("modifyDirective " + id + ": new path " +
-              path + " is not valid.");
-        }
+        path = validatePath(directive);
       }
-      short replication = (directive.getReplication() != null) ?
-          directive.getReplication() : prevEntry.getReplication();
-      if (replication <= 0) {
-        throw new IOException("modifyDirective: replication " + replication +
-            " is invalid.");
+      short replication = prevEntry.getReplication();
+      if (directive.getReplication() != null) {
+        replication = validateReplication(directive, replication);
       }
       CachePool pool = prevEntry.getPool();
       if (directive.getPool() != null) {
-        pool = cachePools.get(directive.getPool());
-        if (pool == null) {
-          throw new IdNotFoundException("modifyDirective " + id +
-              ": pool " + directive.getPool() + " not found.");
-        }
-        if (directive.getPool().isEmpty()) {
-          throw new IdNotFoundException("modifyDirective: pool name was " +
-              "empty.");
-        }
-        if ((pc != null) &&
-            (!pc.checkPermission(pool, FsAction.WRITE))) {
-          throw new AccessControlException("modifyDirective " + id +
-              ": permission denied for target pool " + pool);
-        }
+        pool = getCachePool(validatePoolName(directive));
+        checkWritePermission(pc, pool);
       }
       removeInternal(prevEntry);
       PathBasedCacheEntry newEntry =
           new PathBasedCacheEntry(id, path, replication, pool);
       addInternal(newEntry);
     } catch (IOException e) {
-      LOG.warn("modifyDirective " + idString + ": failed.", e);
+      LOG.warn("modifyDirective of " + idString + " failed: ", e);
       throw e;
     }
-    LOG.info("modifyDirective " + idString + ": successfully applied " +
-        directive);
+    LOG.info("modifyDirective of " + idString + " successfully applied " +
+        directive + ".");
   }
 
   public void removeInternal(PathBasedCacheEntry existing)
-      throws IOException {
+      throws InvalidRequestException {
     assert namesystem.hasWriteLock();
     // Remove the corresponding entry in entriesByPath.
     String path = existing.getPath();
     List<PathBasedCacheEntry> entries = entriesByPath.get(path);
     if (entries == null || !entries.remove(existing)) {
-      throw new IdNotFoundException("removeInternal: failed to locate entry " +
+      throw new InvalidRequestException("Failed to locate entry " +
           existing.getEntryId() + " by path " + existing.getPath());
     }
     if (entries.size() == 0) {
@@ -413,32 +433,17 @@ public final class CacheManager {
       throws IOException {
     assert namesystem.hasWriteLock();
     try {
-      // Check for invalid IDs.
-      if (id <= 0) {
-        throw new IdNotFoundException("removeDirective " + id + ": invalid " +
-            "non-positive directive ID.");
-      }
-      // Find the entry.
-      PathBasedCacheEntry existing = entriesById.get(id);
-      if (existing == null) {
-        throw new IdNotFoundException("removeDirective " + id +
-            ": id not found.");
-      }
-      if ((pc != null) &&
-          (!pc.checkPermission(existing.getPool(), FsAction.WRITE))) {
-        throw new AccessControlException("removeDirective " + id +
-            ": write permission denied on pool " +
-            existing.getPool().getPoolName());
-      }
+      PathBasedCacheEntry existing = getById(id);
+      checkWritePermission(pc, existing.getPool());
       removeInternal(existing);
     } catch (IOException e) {
-      LOG.warn("removeDirective " + id + " failed.", e);
+      LOG.warn("removeDirective of " + id + " failed: ", e);
       throw e;
     }
     if (monitor != null) {
       monitor.kick();
     }
-    LOG.info("removeDirective " + id + ": succeeded.");
+    LOG.info("removeDirective of " + id + " successful.");
   }
 
   public BatchedListEntries<PathBasedCacheDirective> 
@@ -449,18 +454,13 @@ public final class CacheManager {
     final int NUM_PRE_ALLOCATED_ENTRIES = 16;
     String filterPath = null;
     if (filter.getId() != null) {
-      throw new IOException("we currently don't support filtering by ID");
+      throw new IOException("Filtering by ID is unsupported.");
     }
     if (filter.getPath() != null) {
-      filterPath = filter.getPath().toUri().getPath();
-      if (!DFSUtil.isValidName(filterPath)) {
-        throw new IOException("listPathBasedCacheDirectives: invalid " +
-            "path name '" + filterPath + "'");
-      }
+      filterPath = validatePath(filter);
     }
     if (filter.getReplication() != null) {
-      throw new IOException("we currently don't support filtering " +
-          "by replication");
+      throw new IOException("Filtering by replication is unsupported.");
     }
     ArrayList<PathBasedCacheDirective> replies =
         new ArrayList<PathBasedCacheDirective>(NUM_PRE_ALLOCATED_ENTRIES);
@@ -481,8 +481,15 @@ public final class CacheManager {
           !directive.getPath().toUri().getPath().equals(filterPath)) {
         continue;
       }
-      if ((pc == null) ||
-          (pc.checkPermission(curEntry.getPool(), FsAction.READ))) {
+      boolean hasPermission = true;
+      if (pc != null) {
+        try {
+          pc.checkPermission(curEntry.getPool(), FsAction.READ);
+        } catch (AccessControlException e) {
+          hasPermission = false;
+        }
+      }
+      if (hasPermission) {
         replies.add(cur.getValue().toDirective());
         numReplies++;
       }
@@ -505,12 +512,13 @@ public final class CacheManager {
     String poolName = info.getPoolName();
     CachePool pool = cachePools.get(poolName);
     if (pool != null) {
-      throw new IOException("cache pool " + poolName + " already exists.");
+      throw new InvalidRequestException("Cache pool " + poolName
+          + " already exists.");
     }
     pool = CachePool.createFromInfoAndDefaults(info);
     cachePools.put(pool.getPoolName(), pool);
-    LOG.info("created new cache pool " + pool);
-    return pool.getInfo(true);
+    LOG.info("Created new cache pool " + pool);
+    return pool.getInfo(null);
   }
 
   /**
@@ -528,7 +536,8 @@ public final class CacheManager {
     String poolName = info.getPoolName();
     CachePool pool = cachePools.get(poolName);
     if (pool == null) {
-      throw new IOException("cache pool " + poolName + " does not exist.");
+      throw new InvalidRequestException("Cache pool " + poolName
+          + " does not exist.");
     }
     StringBuilder bld = new StringBuilder();
     String prefix = "";
@@ -575,7 +584,8 @@ public final class CacheManager {
     CachePoolInfo.validateName(poolName);
     CachePool pool = cachePools.remove(poolName);
     if (pool == null) {
-      throw new IOException("can't remove non-existent cache pool " + poolName);
+      throw new InvalidRequestException(
+          "Cannot remove non-existent cache pool " + poolName);
     }
     
     // Remove entries using this pool
@@ -607,11 +617,7 @@ public final class CacheManager {
       if (numListed++ >= maxListCachePoolsResponses) {
         return new BatchedListEntries<CachePoolInfo>(results, true);
       }
-      if (pc == null) {
-        results.add(cur.getValue().getInfo(true));
-      } else {
-        results.add(cur.getValue().getInfo(pc));
-      }
+      results.add(cur.getValue().getInfo(pc));
     }
     return new BatchedListEntries<CachePoolInfo>(results, false);
   }
@@ -755,7 +761,7 @@ public final class CacheManager {
     Counter counter = prog.getCounter(Phase.SAVING_CHECKPOINT, step);
     out.writeInt(cachePools.size());
     for (CachePool pool: cachePools.values()) {
-      pool.getInfo(true).writeTo(out);
+      pool.getInfo(null).writeTo(out);
       counter.increment();
     }
     prog.endStep(Phase.SAVING_CHECKPOINT, step);

+ 21 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/CachePool.java

@@ -27,6 +27,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
+import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
 
 import com.google.common.base.Preconditions;
@@ -162,7 +163,7 @@ public final class CachePool {
   }
   
   /**
-   * Get information about this cache pool.
+   * Get either full or partial information about this CachePool.
    *
    * @param fullInfo
    *          If true, only the name will be returned (i.e., what you 
@@ -170,7 +171,7 @@ public final class CachePool {
    * @return
    *          Cache pool information.
    */
-  public CachePoolInfo getInfo(boolean fullInfo) {
+  private CachePoolInfo getInfo(boolean fullInfo) {
     CachePoolInfo info = new CachePoolInfo(poolName);
     if (!fullInfo) {
       return info;
@@ -181,8 +182,25 @@ public final class CachePool {
         setWeight(weight);
   }
 
+  /**
+   * Returns a CachePoolInfo describing this CachePool based on the permissions
+   * of the calling user. Unprivileged users will see only minimal descriptive
+   * information about the pool.
+   * 
+   * @param pc Permission checker to be used to validate the user's permissions,
+   *          or null
+   * @return CachePoolInfo describing this CachePool
+   */
   public CachePoolInfo getInfo(FSPermissionChecker pc) {
-    return getInfo(pc.checkPermission(this, FsAction.READ)); 
+    boolean hasPermission = true;
+    if (pc != null) {
+      try {
+        pc.checkPermission(this, FsAction.READ);
+      } catch (AccessControlException e) {
+        hasPermission = false;
+      }
+    }
+    return getInfo(hasPermission);
   }
 
   public String toString() {

+ 10 - 7
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSPermissionChecker.java

@@ -261,24 +261,27 @@ class FSPermissionChecker {
    *
    * @param pool CachePool being accessed
    * @param access type of action being performed on the cache pool
-   * @return if the pool can be accessed
+   * @throws AccessControlException if pool cannot be accessed
    */
-  public boolean checkPermission(CachePool pool, FsAction access) {
+  public void checkPermission(CachePool pool, FsAction access)
+      throws AccessControlException {
     FsPermission mode = pool.getMode();
     if (isSuperUser()) {
-      return true;
+      return;
     }
     if (user.equals(pool.getOwnerName())
         && mode.getUserAction().implies(access)) {
-      return true;
+      return;
     }
     if (groups.contains(pool.getGroupName())
         && mode.getGroupAction().implies(access)) {
-      return true;
+      return;
     }
     if (mode.getOtherAction().implies(access)) {
-      return true;
+      return;
     }
-    return false;
+    throw new AccessControlException("Permission denied while accessing pool "
+        + pool.getPoolName() + ": user " + user + " does not have "
+        + access.toString() + " permissions.");
   }
 }

+ 8 - 12
hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto

@@ -406,23 +406,23 @@ message ListPathBasedCacheDirectivesResponseProto {
   required bool hasMore = 2;
 }
 
-message AddCachePoolRequestProto {
-  required string poolName = 1;
+message CachePoolInfoProto {
+  optional string poolName = 1;
   optional string ownerName = 2;
   optional string groupName = 3;
   optional int32 mode = 4;
   optional int32 weight = 5;
 }
 
+message AddCachePoolRequestProto {
+  required CachePoolInfoProto info = 1;
+}
+
 message AddCachePoolResponseProto { // void response
 }
 
 message ModifyCachePoolRequestProto {
-  required string poolName = 1;
-  optional string ownerName = 2;
-  optional string groupName = 3;
-  optional int32 mode = 4;
-  optional int32 weight = 5;
+  required CachePoolInfoProto info = 1;
 }
 
 message ModifyCachePoolResponseProto { // void response
@@ -445,11 +445,7 @@ message ListCachePoolsResponseProto {
 }
 
 message ListCachePoolsResponseElementProto {
-  required string poolName = 1;
-  required string ownerName = 2;
-  required string groupName = 3;
-  required int32 mode = 4;
-  required int32 weight = 5;
+  required CachePoolInfoProto info = 1;
 }
 
 message GetFileLinkInfoRequestProto {

+ 64 - 28
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestPathBasedCacheRequests.java

@@ -17,14 +17,16 @@
  */
 package org.apache.hadoop.hdfs.server.namenode;
 
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCK_SIZE_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CACHEREPORT_INTERVAL_MSEC_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_MAX_LOCKED_MEMORY_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCK_SIZE_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_CACHING_ENABLED_KEY;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CACHEREPORT_INTERVAL_MSEC_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_PATH_BASED_CACHE_REFRESH_INTERVAL_MS;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
@@ -40,11 +42,12 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystemTestHelper;
-import org.apache.hadoop.fs.IdNotFoundException;
+import org.apache.hadoop.fs.InvalidRequestException;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
@@ -58,7 +61,6 @@ import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.util.GSet;
 import org.junit.After;
-import org.junit.Assert;
 import org.junit.Assume;
 import org.junit.Before;
 import org.junit.Test;
@@ -186,15 +188,15 @@ public class TestPathBasedCacheRequests {
       fail("expected to get an exception when " +
           "removing a non-existent pool.");
     } catch (IOException ioe) {
-      GenericTestUtils.assertExceptionContains("can't remove " +
+      GenericTestUtils.assertExceptionContains("Cannot remove " +
           "non-existent cache pool", ioe);
     }
     try {
       dfs.removeCachePool(poolName);
-      Assert.fail("expected to get an exception when " +
+      fail("expected to get an exception when " +
           "removing a non-existent pool.");
     } catch (IOException ioe) {
-      GenericTestUtils.assertExceptionContains("can't remove " +
+      GenericTestUtils.assertExceptionContains("Cannot remove " +
           "non-existent cache pool", ioe);
     }
     try {
@@ -271,18 +273,18 @@ public class TestPathBasedCacheRequests {
 
     try {
       proto.removeCachePool("pool99");
-      Assert.fail("expected to get an exception when " +
+      fail("expected to get an exception when " +
           "removing a non-existent pool.");
     } catch (IOException ioe) {
-      GenericTestUtils.assertExceptionContains("can't remove non-existent",
+      GenericTestUtils.assertExceptionContains("Cannot remove non-existent",
           ioe);
     }
     try {
       proto.removeCachePool(poolName);
-      Assert.fail("expected to get an exception when " +
+      fail("expected to get an exception when " +
           "removing a non-existent pool.");
     } catch (IOException ioe) {
-      GenericTestUtils.assertExceptionContains("can't remove non-existent",
+      GenericTestUtils.assertExceptionContains("Cannot remove non-existent",
           ioe);
     }
 
@@ -350,8 +352,8 @@ public class TestPathBasedCacheRequests {
           setPool("no_such_pool").
           build());
       fail("expected an error when adding to a non-existent pool.");
-    } catch (IdNotFoundException ioe) {
-      GenericTestUtils.assertExceptionContains("no such pool as", ioe);
+    } catch (InvalidRequestException ioe) {
+      GenericTestUtils.assertExceptionContains("Unknown pool", ioe);
     }
 
     try {
@@ -363,7 +365,7 @@ public class TestPathBasedCacheRequests {
           "mode 0 (no permissions for anyone).");
     } catch (AccessControlException e) {
       GenericTestUtils.
-          assertExceptionContains("permission denied for pool", e);
+          assertExceptionContains("Permission denied while accessing pool", e);
     }
 
     try {
@@ -383,10 +385,10 @@ public class TestPathBasedCacheRequests {
           setReplication((short)1).
           setPool("").
           build());
-      Assert.fail("expected an error when adding a PathBasedCache " +
+      fail("expected an error when adding a PathBasedCache " +
           "directive with an empty pool name.");
-    } catch (IdNotFoundException e) {
-      GenericTestUtils.assertExceptionContains("pool name was empty", e);
+    } catch (InvalidRequestException e) {
+      GenericTestUtils.assertExceptionContains("Invalid empty pool name", e);
     }
 
     long deltaId = addAsUnprivileged(delta);
@@ -404,7 +406,7 @@ public class TestPathBasedCacheRequests {
     validateListAll(iter, alphaId, alphaId2, betaId, deltaId, relativeId );
     iter = dfs.listPathBasedCacheDirectives(
         new PathBasedCacheDirective.Builder().setPool("pool3").build());
-    Assert.assertFalse(iter.hasNext());
+    assertFalse(iter.hasNext());
     iter = dfs.listPathBasedCacheDirectives(
         new PathBasedCacheDirective.Builder().setPool("pool1").build());
     validateListAll(iter, alphaId, alphaId2, deltaId, relativeId );
@@ -415,27 +417,27 @@ public class TestPathBasedCacheRequests {
     dfs.removePathBasedCacheDirective(betaId);
     iter = dfs.listPathBasedCacheDirectives(
         new PathBasedCacheDirective.Builder().setPool("pool2").build());
-    Assert.assertFalse(iter.hasNext());
+    assertFalse(iter.hasNext());
 
     try {
       dfs.removePathBasedCacheDirective(betaId);
-      Assert.fail("expected an error when removing a non-existent ID");
-    } catch (IdNotFoundException e) {
-      GenericTestUtils.assertExceptionContains("id not found", e);
+      fail("expected an error when removing a non-existent ID");
+    } catch (InvalidRequestException e) {
+      GenericTestUtils.assertExceptionContains("No directive with ID", e);
     }
 
     try {
       proto.removePathBasedCacheDirective(-42l);
-      Assert.fail("expected an error when removing a negative ID");
-    } catch (IdNotFoundException e) {
+      fail("expected an error when removing a negative ID");
+    } catch (InvalidRequestException e) {
       GenericTestUtils.assertExceptionContains(
-          "invalid non-positive directive ID", e);
+          "Invalid negative ID", e);
     }
     try {
       proto.removePathBasedCacheDirective(43l);
-      Assert.fail("expected an error when removing a non-existent ID");
-    } catch (IdNotFoundException e) {
-      GenericTestUtils.assertExceptionContains("id not found", e);
+      fail("expected an error when removing a non-existent ID");
+    } catch (InvalidRequestException e) {
+      GenericTestUtils.assertExceptionContains("No directive with ID", e);
     }
 
     dfs.removePathBasedCacheDirective(alphaId);
@@ -744,4 +746,38 @@ public class TestPathBasedCacheRequests {
     }
   }
 
+  @Test(timeout=60000)
+  public void testListCachePoolPermissions() throws Exception {
+    final UserGroupInformation myUser = UserGroupInformation
+        .createRemoteUser("myuser");
+    final DistributedFileSystem myDfs = 
+        (DistributedFileSystem)DFSTestUtil.getFileSystemAs(myUser, conf);
+    final String poolName = "poolparty";
+    dfs.addCachePool(new CachePoolInfo(poolName)
+        .setMode(new FsPermission((short)0700)));
+    // Should only see partial info
+    RemoteIterator<CachePoolInfo> it = myDfs.listCachePools();
+    CachePoolInfo info = it.next();
+    assertFalse(it.hasNext());
+    assertEquals("Expected pool name", poolName, info.getPoolName());
+    assertNull("Unexpected owner name", info.getOwnerName());
+    assertNull("Unexpected group name", info.getGroupName());
+    assertNull("Unexpected mode", info.getMode());
+    assertNull("Unexpected weight", info.getWeight());
+    // Modify the pool so myuser is now the owner
+    dfs.modifyCachePool(new CachePoolInfo(poolName)
+        .setOwnerName(myUser.getShortUserName())
+        .setWeight(99));
+    // Should see full info
+    it = myDfs.listCachePools();
+    info = it.next();
+    assertFalse(it.hasNext());
+    assertEquals("Expected pool name", poolName, info.getPoolName());
+    assertEquals("Mismatched owner name", myUser.getShortUserName(),
+        info.getOwnerName());
+    assertNotNull("Expected group name", info.getGroupName());
+    assertEquals("Mismatched mode", (short) 0700,
+        info.getMode().toShort());
+    assertEquals("Mismatched weight", 99, (int)info.getWeight());
+  }
 }