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 wa...@apache.org on 2013/09/21 02:20:37 UTC

svn commit: r1525183 [2/2] - 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/protocolPB/ src/main/java/or...

Modified: hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestPathBasedCacheRequests.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestPathBasedCacheRequests.java?rev=1525183&r1=1525182&r2=1525183&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestPathBasedCacheRequests.java (original)
+++ hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestPathBasedCacheRequests.java Sat Sep 21 00:20:36 2013
@@ -17,36 +17,40 @@
  */
 package org.apache.hadoop.hdfs.server.namenode;
 
-import static org.junit.Assert.*;
+import static junit.framework.Assert.assertTrue;
+import static junit.framework.Assert.fail;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
 
 import java.io.IOException;
 import java.security.PrivilegedExceptionAction;
-import java.util.Arrays;
-import java.util.List;
 
 import junit.framework.Assert;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.AddPathBasedCacheDirectiveException.EmptyPathError;
-import org.apache.hadoop.hdfs.protocol.AddPathBasedCacheDirectiveException.InvalidPoolNameError;
 import org.apache.hadoop.hdfs.protocol.AddPathBasedCacheDirectiveException.InvalidPathNameError;
+import org.apache.hadoop.hdfs.protocol.AddPathBasedCacheDirectiveException.InvalidPoolNameError;
 import org.apache.hadoop.hdfs.protocol.AddPathBasedCacheDirectiveException.PoolWritePermissionDeniedError;
 import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
-import org.apache.hadoop.hdfs.protocol.RemovePathBasedCacheDescriptorException.InvalidIdException;
-import org.apache.hadoop.hdfs.protocol.PathBasedCacheDirective;
 import org.apache.hadoop.hdfs.protocol.PathBasedCacheDescriptor;
+import org.apache.hadoop.hdfs.protocol.PathBasedCacheDirective;
+import org.apache.hadoop.hdfs.protocol.RemovePathBasedCacheDescriptorException.InvalidIdException;
 import org.apache.hadoop.hdfs.protocol.RemovePathBasedCacheDescriptorException.NoSuchIdException;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.test.GenericTestUtils;
-import org.apache.hadoop.util.Fallible;
+import org.junit.After;
+import org.junit.Before;
 import org.junit.Test;
 
 public class TestPathBasedCacheRequests {
@@ -55,221 +59,357 @@ public class TestPathBasedCacheRequests 
   private static final UserGroupInformation unprivilegedUser =
       UserGroupInformation.createRemoteUser("unprivilegedUser");
 
-  @Test
-  public void testCreateAndRemovePools() throws Exception {
-    Configuration conf = new HdfsConfiguration();
-    MiniDFSCluster cluster = null;
-
+  static private Configuration conf;
+  static private MiniDFSCluster cluster;
+  static private DistributedFileSystem dfs;
+  static private NamenodeProtocols proto;
+
+  @Before
+  public void setup() throws Exception {
+    conf = new HdfsConfiguration();
+    // set low limits here for testing purposes
+    conf.setInt(DFSConfigKeys.DFS_NAMENODE_LIST_CACHE_POOLS_NUM_RESPONSES, 2);
+    conf.setInt(DFSConfigKeys.DFS_NAMENODE_LIST_CACHE_DESCRIPTORS_NUM_RESPONSES, 2);
     cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
     cluster.waitActive();
-    NamenodeProtocols proto = cluster.getNameNodeRpc();
-    CachePoolInfo req = new CachePoolInfo("pool1").
+    dfs = cluster.getFileSystem();
+    proto = cluster.getNameNodeRpc();
+  }
+
+  @After
+  public void teardown() throws Exception {
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
+
+  @Test
+  public void testBasicPoolOperations() throws Exception {
+    final String poolName = "pool1";
+    CachePoolInfo info = new CachePoolInfo(poolName).
         setOwnerName("bob").setGroupName("bobgroup").
         setMode(new FsPermission((short)0755)).setWeight(150);
-    proto.addCachePool(req);
+
+    // Add a pool
+    dfs.addCachePool(info);
+
+    // Do some bad addCachePools
     try {
-      proto.removeCachePool("pool99");
-      Assert.fail("expected to get an exception when " +
+      dfs.addCachePool(info);
+      fail("added the pool with the same name twice");
+    } catch (IOException ioe) {
+      GenericTestUtils.assertExceptionContains("pool1 already exists", ioe);
+    }
+    try {
+      dfs.addCachePool(new CachePoolInfo(""));
+      fail("added empty pool");
+    } catch (IOException ioe) {
+      GenericTestUtils.assertExceptionContains("invalid empty cache pool name",
+          ioe);
+    }
+    try {
+      dfs.addCachePool(null);
+      fail("added null pool");
+    } catch (IOException ioe) {
+      GenericTestUtils.assertExceptionContains("CachePoolInfo is null", ioe);
+    }
+    try {
+      proto.addCachePool(new CachePoolInfo(""));
+      fail("added empty pool");
+    } catch (IOException ioe) {
+      GenericTestUtils.assertExceptionContains("invalid empty cache pool name",
+          ioe);
+    }
+    try {
+      proto.addCachePool(null);
+      fail("added null pool");
+    } catch (IOException ioe) {
+      GenericTestUtils.assertExceptionContains("CachePoolInfo is null", ioe);
+    }
+
+    // Modify the pool
+    info.setOwnerName("jane").setGroupName("janegroup")
+        .setMode(new FsPermission((short)0700)).setWeight(314);
+    dfs.modifyCachePool(info);
+
+    // Do some invalid modify pools
+    try {
+      dfs.modifyCachePool(new CachePoolInfo("fool"));
+      fail("modified non-existent cache pool");
+    } catch (IOException ioe) {
+      GenericTestUtils.assertExceptionContains("fool does not exist", ioe);
+    }
+    try {
+      dfs.modifyCachePool(new CachePoolInfo(""));
+      fail("modified empty pool");
+    } catch (IOException ioe) {
+      GenericTestUtils.assertExceptionContains("invalid empty cache pool name",
+          ioe);
+    }
+    try {
+      dfs.modifyCachePool(null);
+      fail("modified null pool");
+    } catch (IOException ioe) {
+      GenericTestUtils.assertExceptionContains("CachePoolInfo is null", ioe);
+    }
+    try {
+      proto.modifyCachePool(new CachePoolInfo(""));
+      fail("modified empty pool");
+    } catch (IOException ioe) {
+      GenericTestUtils.assertExceptionContains("invalid empty cache pool name",
+          ioe);
+    }
+    try {
+      proto.modifyCachePool(null);
+      fail("modified null pool");
+    } catch (IOException ioe) {
+      GenericTestUtils.assertExceptionContains("CachePoolInfo is null", ioe);
+    }
+
+    // Remove the pool
+    dfs.removeCachePool(poolName);
+    // Do some bad removePools
+    try {
+      dfs.removeCachePool("pool99");
+      fail("expected to get an exception when " +
           "removing a non-existent pool.");
     } catch (IOException ioe) {
       GenericTestUtils.assertExceptionContains("can't remove " +
-          "nonexistent cache pool", ioe);
+          "non-existent cache pool", ioe);
     }
-    proto.removeCachePool("pool1");
     try {
-      proto.removeCachePool("pool1");
+      dfs.removeCachePool(poolName);
       Assert.fail("expected to get an exception when " +
           "removing a non-existent pool.");
     } catch (IOException ioe) {
       GenericTestUtils.assertExceptionContains("can't remove " +
-          "nonexistent cache pool", ioe);
+          "non-existent cache pool", ioe);
     }
-    req = new CachePoolInfo("pool2");
-    proto.addCachePool(req);
+    try {
+      dfs.removeCachePool("");
+      fail("removed empty pool");
+    } catch (IOException ioe) {
+      GenericTestUtils.assertExceptionContains("invalid empty cache pool name",
+          ioe);
+    }
+    try {
+      dfs.removeCachePool(null);
+      fail("removed null pool");
+    } catch (IOException ioe) {
+      GenericTestUtils.assertExceptionContains("invalid empty cache pool name",
+          ioe);
+    }
+    try {
+      proto.removeCachePool("");
+      fail("removed empty pool");
+    } catch (IOException ioe) {
+      GenericTestUtils.assertExceptionContains("invalid empty cache pool name",
+          ioe);
+    }
+    try {
+      proto.removeCachePool(null);
+      fail("removed null pool");
+    } catch (IOException ioe) {
+      GenericTestUtils.assertExceptionContains("invalid empty cache pool name",
+          ioe);
+    }
+
+    info = new CachePoolInfo("pool2");
+    dfs.addCachePool(info);
   }
 
   @Test
   public void testCreateAndModifyPools() throws Exception {
-    Configuration conf = new HdfsConfiguration();
-    MiniDFSCluster cluster = null;
-    // set low limits here for testing purposes
-    conf.setInt(DFSConfigKeys.DFS_NAMENODE_LIST_CACHE_POOLS_NUM_RESPONSES, 2);
-    conf.setInt(DFSConfigKeys.DFS_NAMENODE_LIST_CACHE_DESCRIPTORS_NUM_RESPONSES, 2);
-    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
-    cluster.waitActive();
-    NamenodeProtocols proto = cluster.getNameNodeRpc();
-    proto.addCachePool(new CachePoolInfo("pool1").
-        setOwnerName("abc").setGroupName("123").
-        setMode(new FsPermission((short)0755)).setWeight(150));
-    RemoteIterator<CachePoolInfo> iter = proto.listCachePools("");
+    String poolName = "pool1";
+    String ownerName = "abc";
+    String groupName = "123";
+    FsPermission mode = new FsPermission((short)0755);
+    int weight = 150;
+    dfs.addCachePool(new CachePoolInfo(poolName).
+        setOwnerName(ownerName).setGroupName(groupName).
+        setMode(mode).setWeight(weight));
+    
+    RemoteIterator<CachePoolInfo> iter = dfs.listCachePools();
     CachePoolInfo info = iter.next();
-    assertEquals("pool1", info.getPoolName());
-    assertEquals("abc", info.getOwnerName());
-    assertEquals("123", info.getGroupName());
-    proto.modifyCachePool(new CachePoolInfo("pool1").
-        setOwnerName("def").setGroupName("456"));
-    iter = proto.listCachePools("");
+    assertEquals(poolName, info.getPoolName());
+    assertEquals(ownerName, info.getOwnerName());
+    assertEquals(groupName, info.getGroupName());
+
+    ownerName = "def";
+    groupName = "456";
+    mode = new FsPermission((short)0700);
+    weight = 151;
+    dfs.modifyCachePool(new CachePoolInfo(poolName).
+        setOwnerName(ownerName).setGroupName(groupName).
+        setMode(mode).setWeight(weight));
+
+    iter = dfs.listCachePools();
     info = iter.next();
-    assertEquals("pool1", info.getPoolName());
-    assertEquals("def", info.getOwnerName());
-    assertEquals("456", info.getGroupName());
-    assertEquals(new FsPermission((short)0755), info.getMode());
-    assertEquals(Integer.valueOf(150), info.getWeight());
+    assertEquals(poolName, info.getPoolName());
+    assertEquals(ownerName, info.getOwnerName());
+    assertEquals(groupName, info.getGroupName());
+    assertEquals(mode, info.getMode());
+    assertEquals(Integer.valueOf(weight), info.getWeight());
+
+    dfs.removeCachePool(poolName);
+    iter = dfs.listCachePools();
+    assertFalse("expected no cache pools after deleting pool", iter.hasNext());
+
+    proto.listCachePools(null);
 
     try {
       proto.removeCachePool("pool99");
       Assert.fail("expected to get an exception when " +
           "removing a non-existent pool.");
     } catch (IOException ioe) {
+      GenericTestUtils.assertExceptionContains("can't remove non-existent",
+          ioe);
     }
-    proto.removeCachePool("pool1");
     try {
-      proto.removeCachePool("pool1");
+      proto.removeCachePool(poolName);
       Assert.fail("expected to get an exception when " +
           "removing a non-existent pool.");
     } catch (IOException ioe) {
+      GenericTestUtils.assertExceptionContains("can't remove non-existent",
+          ioe);
     }
+
+    iter = dfs.listCachePools();
+    assertFalse("expected no cache pools after deleting pool", iter.hasNext());
   }
 
   private static void validateListAll(
       RemoteIterator<PathBasedCacheDescriptor> iter,
-      long id0, long id1, long id2) throws Exception {
-    Assert.assertEquals(new PathBasedCacheDescriptor(id0,
-        "/alpha", "pool1"), iter.next());
-    Assert.assertEquals(new PathBasedCacheDescriptor(id1,
-        "/beta", "pool2"), iter.next());
-    Assert.assertEquals(new PathBasedCacheDescriptor(id2,
-        "/gamma", "pool1"), iter.next());
-    Assert.assertFalse(iter.hasNext());
+      PathBasedCacheDescriptor... descriptors) throws Exception {
+    for (PathBasedCacheDescriptor descriptor: descriptors) {
+      assertTrue("Unexpectedly few elements", iter.hasNext());
+      assertEquals("Unexpected descriptor", descriptor, iter.next());
+    }
+    assertFalse("Unexpectedly many list elements", iter.hasNext());
   }
 
-  @Test
-  public void testSetAndGet() throws Exception {
-    Configuration conf = new HdfsConfiguration();
-    MiniDFSCluster cluster = null;
-
-    try {
-      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
-      cluster.waitActive();
-      final NamenodeProtocols proto = cluster.getNameNodeRpc();
-      proto.addCachePool(new CachePoolInfo("pool1").
-          setMode(new FsPermission((short)0777)));
-      proto.addCachePool(new CachePoolInfo("pool2").
-          setMode(new FsPermission((short)0777)));
-      proto.addCachePool(new CachePoolInfo("pool3").
-          setMode(new FsPermission((short)0777)));
-      proto.addCachePool(new CachePoolInfo("pool4").
-          setMode(new FsPermission((short)0)));
-
-      List<Fallible<PathBasedCacheDescriptor>> addResults1 = 
-        unprivilegedUser.doAs(new PrivilegedExceptionAction<
-            List<Fallible<PathBasedCacheDescriptor>>>() {
+  private static PathBasedCacheDescriptor addAsUnprivileged(
+      final PathBasedCacheDirective directive) throws Exception {
+    return unprivilegedUser
+        .doAs(new PrivilegedExceptionAction<PathBasedCacheDescriptor>() {
           @Override
-          public List<Fallible<PathBasedCacheDescriptor>> run() throws IOException {
-            return proto.addPathBasedCacheDirectives(Arrays.asList(
-              new PathBasedCacheDirective[] {
-                new PathBasedCacheDirective("/alpha", "pool1"),
-                new PathBasedCacheDirective("/beta", "pool2"),
-                new PathBasedCacheDirective("", "pool3"),
-                new PathBasedCacheDirective("/zeta", "nonexistent_pool"),
-                new PathBasedCacheDirective("/zeta", "pool4"),
-                new PathBasedCacheDirective("//illegal/path/", "pool1")
-              }));
-            }
-          });
-      long ids1[] = new long[2];
-      ids1[0] = addResults1.get(0).get().getEntryId();
-      ids1[1] = addResults1.get(1).get().getEntryId();
-      try {
-        addResults1.get(2).get();
-        Assert.fail("expected an error when adding an empty path");
-      } catch (IOException ioe) {
-        Assert.assertTrue(ioe.getCause() instanceof EmptyPathError);
-      }
-      try {
-        addResults1.get(3).get();
-        Assert.fail("expected an error when adding to a nonexistent pool.");
-      } catch (IOException ioe) {
-        Assert.assertTrue(ioe.getCause() instanceof InvalidPoolNameError);
-      }
-      try {
-        addResults1.get(4).get();
-        Assert.fail("expected an error when adding to a pool with " +
-            "mode 0 (no permissions for anyone).");
-      } catch (IOException ioe) {
-        Assert.assertTrue(ioe.getCause()
-            instanceof PoolWritePermissionDeniedError);
-      }
-      try {
-        addResults1.get(5).get();
-        Assert.fail("expected an error when adding a malformed path " +
-            "to the cache directives.");
-      } catch (IOException ioe) {
-        //Assert.assertTrue(ioe.getCause()
-            //instanceof PoolWritePermissionDeniedError);
-      }
-
-      List<Fallible<PathBasedCacheDescriptor>> addResults2 = 
-          proto.addPathBasedCacheDirectives(Arrays.asList(
-            new PathBasedCacheDirective[] {
-        new PathBasedCacheDirective("/alpha", "pool1"),
-        new PathBasedCacheDirective("/theta", ""),
-        new PathBasedCacheDirective("bogus", "pool1"),
-        new PathBasedCacheDirective("/gamma", "pool1")
-      }));
-      long id = addResults2.get(0).get().getEntryId();
-      Assert.assertEquals("expected to get back the same ID as last time " +
-          "when re-adding an existing PathBasedCache directive.", ids1[0], id);
-      try {
-        addResults2.get(1).get();
-        Assert.fail("expected an error when adding a PathBasedCache " +
-            "directive with an empty pool name.");
-      } catch (IOException ioe) {
-        Assert.assertTrue(ioe.getCause() instanceof InvalidPoolNameError);
-      }
-      try {
-        addResults2.get(2).get();
-        Assert.fail("expected an error when adding a PathBasedCache " +
-            "directive with a non-absolute path name.");
-      } catch (IOException ioe) {
-        Assert.assertTrue(ioe.getCause() instanceof InvalidPathNameError);
-      }
-      long ids2[] = new long[1];
-      ids2[0] = addResults2.get(3).get().getEntryId();
-
-      RemoteIterator<PathBasedCacheDescriptor> iter =
-          proto.listPathBasedCacheDescriptors(0, null, null);
-      validateListAll(iter, ids1[0], ids1[1], ids2[0]);
-      iter = proto.listPathBasedCacheDescriptors(0, null, null);
-      validateListAll(iter, ids1[0], ids1[1], ids2[0]);
-      iter = proto.listPathBasedCacheDescriptors(0, "pool3", null);
-      Assert.assertFalse(iter.hasNext());
-      iter = proto.listPathBasedCacheDescriptors(0, "pool2", null);
-      Assert.assertEquals(addResults1.get(1).get(),
-          iter.next());
-      Assert.assertFalse(iter.hasNext());
-
-      List<Fallible<Long>> removeResults1 = 
-          proto.removePathBasedCacheDescriptors(Arrays.asList(
-            new Long[] { ids1[1], -42L, 999999L }));
-      Assert.assertEquals(Long.valueOf(ids1[1]),
-          removeResults1.get(0).get());
-      try {
-        removeResults1.get(1).get();
-        Assert.fail("expected an error when removing a negative ID");
-      } catch (IOException ioe) {
-        Assert.assertTrue(ioe.getCause() instanceof InvalidIdException);
-      }
-      try {
-        removeResults1.get(2).get();
-        Assert.fail("expected an error when removing a nonexistent ID");
-      } catch (IOException ioe) {
-        Assert.assertTrue(ioe.getCause() instanceof NoSuchIdException);
-      }
-      iter = proto.listPathBasedCacheDescriptors(0, "pool2", null);
-      Assert.assertFalse(iter.hasNext());
-    } finally {
-      if (cluster != null) { cluster.shutdown(); }
+          public PathBasedCacheDescriptor run() throws IOException {
+            DistributedFileSystem myDfs =
+                (DistributedFileSystem) FileSystem.get(conf);
+            return myDfs.addPathBasedCacheDirective(directive);
+          }
+        });
+  }
+
+  @Test
+  public void testAddRemoveDirectives() throws Exception {
+    proto.addCachePool(new CachePoolInfo("pool1").
+        setMode(new FsPermission((short)0777)));
+    proto.addCachePool(new CachePoolInfo("pool2").
+        setMode(new FsPermission((short)0777)));
+    proto.addCachePool(new CachePoolInfo("pool3").
+        setMode(new FsPermission((short)0777)));
+    proto.addCachePool(new CachePoolInfo("pool4").
+        setMode(new FsPermission((short)0)));
+
+    PathBasedCacheDirective alpha =
+        new PathBasedCacheDirective("/alpha", "pool1");
+    PathBasedCacheDirective beta =
+        new PathBasedCacheDirective("/beta", "pool2");
+    PathBasedCacheDirective delta =
+        new PathBasedCacheDirective("/delta", "pool1");
+
+    PathBasedCacheDescriptor alphaD = addAsUnprivileged(alpha);
+    PathBasedCacheDescriptor alphaD2 = addAsUnprivileged(alpha);
+    assertEquals("Expected to get the same descriptor when re-adding"
+        + "an existing PathBasedCacheDirective", alphaD, alphaD2);
+    PathBasedCacheDescriptor betaD = addAsUnprivileged(beta);
+
+    try {
+      addAsUnprivileged(new PathBasedCacheDirective("", "pool3"));
+      fail("expected an error when adding an empty path");
+    } catch (IOException ioe) {
+      assertTrue(ioe instanceof EmptyPathError);
     }
+
+    try {
+      addAsUnprivileged(new PathBasedCacheDirective("/unicorn", "no_such_pool"));
+      fail("expected an error when adding to a non-existent pool.");
+    } catch (IOException ioe) {
+      assertTrue(ioe instanceof InvalidPoolNameError);
+    }
+
+    try {
+      addAsUnprivileged(new PathBasedCacheDirective("/blackhole", "pool4"));
+      fail("expected an error when adding to a pool with " +
+          "mode 0 (no permissions for anyone).");
+    } catch (IOException ioe) {
+      assertTrue(ioe instanceof PoolWritePermissionDeniedError);
+    }
+
+    try {
+      addAsUnprivileged(new PathBasedCacheDirective("//illegal/path/", "pool1"));
+      fail("expected an error when adding a malformed path " +
+          "to the cache directives.");
+    } catch (IOException ioe) {
+      assertTrue(ioe instanceof InvalidPathNameError);
+    }
+
+    try {
+      addAsUnprivileged(new PathBasedCacheDirective("/emptypoolname", ""));
+      Assert.fail("expected an error when adding a PathBasedCache " +
+          "directive with an empty pool name.");
+    } catch (IOException ioe) {
+      Assert.assertTrue(ioe instanceof InvalidPoolNameError);
+    }
+
+    try {
+      addAsUnprivileged(new PathBasedCacheDirective("bogus", "pool1"));
+      Assert.fail("expected an error when adding a PathBasedCache " +
+          "directive with a non-absolute path name.");
+    } catch (IOException ioe) {
+      Assert.assertTrue(ioe instanceof InvalidPathNameError);
+    }
+
+    PathBasedCacheDescriptor deltaD = addAsUnprivileged(delta);
+
+    RemoteIterator<PathBasedCacheDescriptor> iter;
+    iter = proto.listPathBasedCacheDescriptors(0, null, null);
+    validateListAll(iter, alphaD, betaD, deltaD);
+    iter = proto.listPathBasedCacheDescriptors(0, "pool3", null);
+    Assert.assertFalse(iter.hasNext());
+    iter = proto.listPathBasedCacheDescriptors(0, "pool1", null);
+    validateListAll(iter, alphaD, deltaD);
+    iter = proto.listPathBasedCacheDescriptors(0, "pool2", null);
+    validateListAll(iter, betaD);
+
+    dfs.removePathBasedCacheDescriptor(betaD);
+    iter = proto.listPathBasedCacheDescriptors(0, "pool2", null);
+    Assert.assertFalse(iter.hasNext());
+
+    try {
+      dfs.removePathBasedCacheDescriptor(betaD);
+      Assert.fail("expected an error when removing a non-existent ID");
+    } catch (IOException ioe) {
+      Assert.assertTrue(ioe instanceof NoSuchIdException);
+    }
+
+    try {
+      proto.removePathBasedCacheDescriptor(-42l);
+      Assert.fail("expected an error when removing a negative ID");
+    } catch (IOException ioe) {
+      Assert.assertTrue(ioe instanceof InvalidIdException);
+    }
+    try {
+      proto.removePathBasedCacheDescriptor(43l);
+      Assert.fail("expected an error when removing a non-existent ID");
+    } catch (IOException ioe) {
+      Assert.assertTrue(ioe instanceof NoSuchIdException);
+    }
+
+    dfs.removePathBasedCacheDescriptor(alphaD);
+    dfs.removePathBasedCacheDescriptor(deltaD);
+    iter = proto.listPathBasedCacheDescriptors(0, null, null);
+    assertFalse(iter.hasNext());
   }
 }