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/04 20:23:52 UTC

svn commit: r1520090 [2/2] - in /hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src: main/java/org/apache/hadoop/hdfs/protocol/ main/java/org/apache/hadoop/hdfs/protocolPB/ main/java/org/apache/hadoop/hdfs/server/namenode/ main/proto/...

Modified: hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.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/NameNodeRpcServer.java?rev=1520090&r1=1520089&r2=1520090&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java (original)
+++ hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java Wed Sep  4 18:23:51 2013
@@ -31,13 +31,11 @@ import java.util.Arrays;
 import java.util.Collection;
 import java.util.HashMap;
 import java.util.List;
-import java.util.NoSuchElementException;
 
 import org.apache.commons.logging.Log;
 import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.BatchedRemoteIterator;
-import org.apache.hadoop.fs.BatchedRemoteIterator.BatchedEntries;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.CreateFlag;
@@ -62,9 +60,9 @@ import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.HDFSPolicyProvider;
 import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException;
 import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
+import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
 import org.apache.hadoop.hdfs.protocol.PathCacheDirective;
 import org.apache.hadoop.hdfs.protocol.PathCacheEntry;
-import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
 import org.apache.hadoop.hdfs.protocol.CorruptFileBlocks;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
@@ -1225,72 +1223,73 @@ class NameNodeRpcServer implements Namen
   private class ServerSidePathCacheEntriesIterator
       extends BatchedRemoteIterator<Long, PathCacheEntry> {
 
-    private final String pool;
+    private final Long poolId;
 
     public ServerSidePathCacheEntriesIterator(Long firstKey,
-        int maxRepliesPerRequest, String pool) {
+        int maxRepliesPerRequest, Long poolId) {
       super(firstKey, maxRepliesPerRequest);
-      this.pool = pool;
+      this.poolId = poolId;
     }
 
     @Override
     public BatchedEntries<PathCacheEntry> makeRequest(
-        Long nextKey, int maxRepliesPerRequest) throws IOException {
+        Long prevKey, int maxRepliesPerRequest) throws IOException {
       return new BatchedListEntries<PathCacheEntry>(
-          namesystem.listPathCacheEntries(nextKey, pool,
+          namesystem.listPathCacheEntries(prevKey, poolId,
               maxRepliesPerRequest));
     }
 
     @Override
-    public Long elementToNextKey(PathCacheEntry entry) {
+    public Long elementToPrevKey(PathCacheEntry entry) {
       return entry.getEntryId();
     }
   }
-  
+
   @Override
-  public RemoteIterator<PathCacheEntry> listPathCacheEntries(long prevId, String pool,
-      int maxReplies) throws IOException {
-    return new ServerSidePathCacheEntriesIterator(prevId, maxReplies, pool);
+  public RemoteIterator<PathCacheEntry> listPathCacheEntries(long prevId,
+      long poolId, int maxReplies) throws IOException {
+    return new ServerSidePathCacheEntriesIterator(prevId, maxReplies, poolId);
   }
 
   @Override
-  public void addCachePool(CachePoolInfo info) throws IOException {
-    namesystem.addCachePool(info);
+  public CachePool addCachePool(CachePoolInfo info) throws IOException {
+    return namesystem.addCachePool(info);
   }
 
   @Override
-  public void modifyCachePool(CachePoolInfo info) throws IOException {
-    namesystem.modifyCachePool(info);
+  public void modifyCachePool(long poolId, CachePoolInfo info)
+      throws IOException {
+    namesystem.modifyCachePool(poolId, info);
   }
 
   @Override
-  public void removeCachePool(String cachePoolName) throws IOException {
-    namesystem.removeCachePool(cachePoolName);
+  public void removeCachePool(long poolId) throws IOException {
+    namesystem.removeCachePool(poolId);
   }
 
   private class ServerSideCachePoolIterator 
-      extends BatchedRemoteIterator<String, CachePoolInfo> {
+      extends BatchedRemoteIterator<Long, CachePool> {
 
-    public ServerSideCachePoolIterator(String prevKey, int maxRepliesPerRequest) {
-      super(prevKey, maxRepliesPerRequest);
+    public ServerSideCachePoolIterator(long prevId, int maxRepliesPerRequest) {
+      super(prevId, maxRepliesPerRequest);
     }
 
     @Override
-    public BatchedEntries<CachePoolInfo> makeRequest(String prevKey,
+    public BatchedEntries<CachePool> makeRequest(Long prevId,
         int maxRepliesPerRequest) throws IOException {
-      return new BatchedListEntries<CachePoolInfo>(
-          namesystem.listCachePools(prevKey, maxRepliesPerRequest));
+      return new BatchedListEntries<CachePool>(
+          namesystem.listCachePools(prevId, maxRepliesPerRequest));
     }
 
     @Override
-    public String elementToNextKey(CachePoolInfo element) {
-      return element.getPoolName();
+    public Long elementToPrevKey(CachePool element) {
+      return element.getId();
     }
   }
 
   @Override
-  public RemoteIterator<CachePoolInfo> listCachePools(String prevKey,
+  public RemoteIterator<CachePool> listCachePools(long prevPoolId,
       int maxRepliesPerRequest) throws IOException {
-    return new ServerSideCachePoolIterator(prevKey, maxRepliesPerRequest);
+    return new ServerSideCachePoolIterator(prevPoolId, maxRepliesPerRequest);
   }
 }

Modified: hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto?rev=1520090&r1=1520089&r2=1520090&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto (original)
+++ hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto Wed Sep  4 18:23:51 2013
@@ -363,9 +363,27 @@ message IsFileClosedResponseProto {
   required bool result = 1;
 }
 
+message CachePoolInfoProto {
+  optional string poolName = 1;
+  optional string ownerName = 2;
+  optional string groupName = 3;
+  optional int32 mode = 4;
+  optional int32 weight = 5;
+}
+
+message CachePoolProto {
+  optional int64 id = 1;
+  optional CachePoolInfoProto info = 2;
+}
+
 message PathCacheDirectiveProto {
   required string path = 1;
-  required string pool = 2;
+  required CachePoolProto pool = 2;
+}
+
+message PathCacheEntryProto {
+  required int64 id = 1;
+  optional PathCacheDirectiveProto directive = 2;
 }
 
 message AddPathCacheDirectivesRequestProto {
@@ -399,53 +417,41 @@ enum RemovePathCacheEntryErrorProto {
 }
 
 message ListPathCacheEntriesRequestProto {
-  required int64 prevId = 1;
-  required string pool = 2;
+  required PathCacheEntryProto prevEntry = 1;
+  required CachePoolProto pool = 2;
   optional int32 maxReplies = 3;
 }
 
-message ListPathCacheEntriesElementProto {
-  required int64 id = 1;
-  required string path = 2;
-  required string pool = 3;
-}
-
 message ListPathCacheEntriesResponseProto {
-  repeated ListPathCacheEntriesElementProto elements = 1;
+  repeated PathCacheEntryProto entries = 1;
   required bool hasMore = 2;
 }
 
 message AddCachePoolRequestProto {
-  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 AddCachePoolResponseProto { // void response
+message AddCachePoolResponseProto {
+  required CachePoolProto pool = 1;
 }
 
 message ModifyCachePoolRequestProto {
-  required string poolName = 1;
-  optional string ownerName = 2;
-  optional string groupName = 3;
-  optional int32 mode = 4;
-  optional int32 weight = 5;
+  required CachePoolProto pool = 1;
+  required CachePoolInfoProto info = 2;
 }
 
 message ModifyCachePoolResponseProto { // void response
 }
 
 message RemoveCachePoolRequestProto {
-  required string poolName = 1;
+  required CachePoolProto pool = 1;
 }
 
 message RemoveCachePoolResponseProto { // void response
 }
 
 message ListCachePoolsRequestProto {
-  required string prevPoolName = 1;
+  required CachePoolProto prevPool = 1;
   required int32 maxReplies = 2;
 }
 
@@ -455,11 +461,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 CachePoolProto pool = 1;
 }
 
 message GetFileLinkInfoRequestProto {

Modified: hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestFsDatasetCache.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/datanode/TestFsDatasetCache.java?rev=1520090&r1=1520089&r2=1520090&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestFsDatasetCache.java (original)
+++ hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestFsDatasetCache.java Wed Sep  4 18:23:51 2013
@@ -92,6 +92,9 @@ public class TestFsDatasetCache {
 
   @After
   public void tearDown() throws Exception {
+    if (fs != null) {
+      fs.close();
+    }
     if (cluster != null) {
       cluster.shutdown();
     }
@@ -159,13 +162,11 @@ public class TestFsDatasetCache {
   }
 
   /**
-   * Blocks until cache usage changes from the current value, then verifies
-   * against the expected new value.
+   * Blocks until cache usage hits the expected new value.
    */
-  private long verifyExpectedCacheUsage(final long current,
-      final long expected) throws Exception {
+  private long verifyExpectedCacheUsage(final long expected) throws Exception {
     long cacheUsed = fsd.getCacheUsed();
-    while (cacheUsed == current) {
+    while (cacheUsed != expected) {
       cacheUsed = fsd.getCacheUsed();
       Thread.sleep(100);
     }
@@ -202,13 +203,13 @@ public class TestFsDatasetCache {
     // Cache each block in succession, checking each time
     for (int i=0; i<NUM_BLOCKS; i++) {
       setHeartbeatResponse(cacheBlock(locs[i]));
-      current = verifyExpectedCacheUsage(current, current + blockSizes[i]);
+      current = verifyExpectedCacheUsage(current + blockSizes[i]);
     }
 
     // Uncache each block in succession, again checking each time
     for (int i=0; i<NUM_BLOCKS; i++) {
       setHeartbeatResponse(uncacheBlock(locs[i]));
-      current = verifyExpectedCacheUsage(current, current - blockSizes[i]);
+      current = verifyExpectedCacheUsage(current - blockSizes[i]);
     }
   }
 
@@ -237,7 +238,7 @@ public class TestFsDatasetCache {
     long current = 0;
     for (int i=0; i<numFiles-1; i++) {
       setHeartbeatResponse(cacheBlocks(fileLocs[i]));
-      current = verifyExpectedCacheUsage(current, current + fileSizes[i]);
+      current = verifyExpectedCacheUsage(current + fileSizes[i]);
     }
     final long oldCurrent = current;
 
@@ -262,7 +263,7 @@ public class TestFsDatasetCache {
     // Uncache the n-1 files
     for (int i=0; i<numFiles-1; i++) {
       setHeartbeatResponse(uncacheBlocks(fileLocs[i]));
-      current = verifyExpectedCacheUsage(current, current - fileSizes[i]);
+      current = verifyExpectedCacheUsage(current - fileSizes[i]);
     }
   }
 }

Modified: hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestPathCacheRequests.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/TestPathCacheRequests.java?rev=1520090&r1=1520089&r2=1520090&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestPathCacheRequests.java (original)
+++ hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestPathCacheRequests.java Wed Sep  4 18:23:51 2013
@@ -17,9 +17,10 @@
  */
 package org.apache.hadoop.hdfs.server.namenode;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
 
 import java.io.IOException;
+import java.security.PrivilegedExceptionAction;
 import java.util.Arrays;
 import java.util.List;
 
@@ -29,53 +30,65 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.AddPathCacheDirectiveException.EmptyPathError;
-import org.apache.hadoop.hdfs.protocol.AddPathCacheDirectiveException.InvalidPoolNameError;
 import org.apache.hadoop.hdfs.protocol.AddPathCacheDirectiveException.InvalidPathNameError;
+import org.apache.hadoop.hdfs.protocol.AddPathCacheDirectiveException.InvalidPoolError;
 import org.apache.hadoop.hdfs.protocol.AddPathCacheDirectiveException.PoolWritePermissionDeniedError;
 import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
-import org.apache.hadoop.hdfs.protocol.RemovePathCacheEntryException.InvalidIdException;
 import org.apache.hadoop.hdfs.protocol.PathCacheDirective;
 import org.apache.hadoop.hdfs.protocol.PathCacheEntry;
+import org.apache.hadoop.hdfs.protocol.RemovePathCacheEntryException.InvalidIdException;
 import org.apache.hadoop.hdfs.protocol.RemovePathCacheEntryException.NoSuchIdException;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
-import org.apache.hadoop.test.GenericTestUtils;
+import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.Fallible;
+import org.junit.After;
+import org.junit.Before;
 import org.junit.Test;
 
 public class TestPathCacheRequests {
   static final Log LOG = LogFactory.getLog(TestPathCacheRequests.class);
 
-  @Test
-  public void testCreateAndRemovePools() throws Exception {
-    Configuration conf = new HdfsConfiguration();
-    MiniDFSCluster cluster = null;
+  private static Configuration conf = new HdfsConfiguration();
+  private static MiniDFSCluster cluster = null;
+  private static NamenodeProtocols proto = null;
 
+  @Before
+  public void setUp() throws Exception {
     cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
     cluster.waitActive();
-    NamenodeProtocols proto = cluster.getNameNodeRpc();
-    CachePoolInfo req = new CachePoolInfo("pool1").
-        setOwnerName("bob").setGroupName("bobgroup").
-        setMode(0755).setWeight(150);
-    proto.addCachePool(req);
+    proto = cluster.getNameNodeRpc();
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
+
+  @Test
+  public void testCreateAndRemovePools() throws Exception {
+    CachePoolInfo req =
+        CachePoolInfo.newBuilder().setPoolName("pool1").setOwnerName("bob")
+            .setGroupName("bobgroup").setMode(new FsPermission((short) 0755))
+            .setWeight(150).build();
+    CachePool pool = proto.addCachePool(req);
     try {
-      proto.removeCachePool("pool99");
+      proto.removeCachePool(909);
       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);
     }
-    proto.removeCachePool("pool1");
+    proto.removeCachePool(pool.getId());
     try {
-      proto.removeCachePool("pool1");
+      proto.removeCachePool(pool.getId());
       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);
     }
     req = new CachePoolInfo("pool2");
     proto.addCachePool(req);
@@ -83,34 +96,42 @@ public class TestPathCacheRequests {
 
   @Test
   public void testCreateAndModifyPools() throws Exception {
-    Configuration conf = new HdfsConfiguration();
-    MiniDFSCluster cluster = null;
-
-    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
-    cluster.waitActive();
-    NamenodeProtocols proto = cluster.getNameNodeRpc();
-    proto.addCachePool(new CachePoolInfo("pool1").
-        setOwnerName("abc").setGroupName("123").
-        setMode(0755).setWeight(150));
-    proto.modifyCachePool(new CachePoolInfo("pool1").
-        setOwnerName("def").setGroupName("456"));
-    RemoteIterator<CachePoolInfo> iter = proto.listCachePools("", 1);
-    CachePoolInfo info = iter.next();
-    assertEquals("pool1", info.getPoolName());
-    assertEquals("def", info.getOwnerName());
-    assertEquals("456", info.getGroupName());
-    assertEquals(Integer.valueOf(0755), info.getMode());
-    assertEquals(Integer.valueOf(150), info.getWeight());
+    // Create a new pool
+    CachePoolInfo info = CachePoolInfo.newBuilder().
+        setPoolName("pool1").
+        setOwnerName("abc").
+        setGroupName("123").
+        setMode(new FsPermission((short)0755)).
+        setWeight(150).
+        build();
+    CachePool pool = proto.addCachePool(info);
+    CachePoolInfo actualInfo = pool.getInfo();
+    assertEquals("Expected info to match create time settings",
+        info, actualInfo);
+    // Modify the pool
+    info = CachePoolInfo.newBuilder().
+        setPoolName("pool2").
+        setOwnerName("def").
+        setGroupName("456").
+        setMode(new FsPermission((short)0644)).
+        setWeight(200).
+        build();
+    proto.modifyCachePool(pool.getId(), info);
+    // Check via listing this time
+    RemoteIterator<CachePool> iter = proto.listCachePools(0, 1);
+    CachePool listedPool = iter.next();
+    actualInfo = listedPool.getInfo();
+    assertEquals("Expected info to match modified settings", info, actualInfo);
 
     try {
-      proto.removeCachePool("pool99");
+      proto.removeCachePool(808);
       Assert.fail("expected to get an exception when " +
           "removing a non-existent pool.");
     } catch (IOException ioe) {
     }
-    proto.removeCachePool("pool1");
+    proto.removeCachePool(pool.getId());
     try {
-      proto.removeCachePool("pool1");
+      proto.removeCachePool(pool.getId());
       Assert.fail("expected to get an exception when " +
           "removing a non-existent pool.");
     } catch (IOException ioe) {
@@ -121,13 +142,13 @@ public class TestPathCacheRequests {
       RemoteIterator<PathCacheEntry> iter,
       long id0, long id1, long id2) throws Exception {
     Assert.assertEquals(new PathCacheEntry(id0,
-        new PathCacheDirective("/alpha", "pool1")),
+        new PathCacheDirective("/alpha", 1)),
         iter.next());
     Assert.assertEquals(new PathCacheEntry(id1,
-        new PathCacheDirective("/beta", "pool2")),
+        new PathCacheDirective("/beta", 2)),
         iter.next());
     Assert.assertEquals(new PathCacheEntry(id2,
-        new PathCacheDirective("/gamma", "pool1")),
+        new PathCacheDirective("/gamma", 1)),
         iter.next());
     Assert.assertFalse(iter.hasNext());
   }
@@ -140,23 +161,36 @@ public class TestPathCacheRequests {
     try {
       cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
       cluster.waitActive();
-      NamenodeProtocols proto = cluster.getNameNodeRpc();
-      proto.addCachePool(new CachePoolInfo("pool1"));
-      proto.addCachePool(new CachePoolInfo("pool2"));
-      proto.addCachePool(new CachePoolInfo("pool3"));
-      proto.addCachePool(new CachePoolInfo("pool4").setMode(0));
-      List<Fallible<PathCacheEntry>> addResults1 = 
-          proto.addPathCacheDirectives(Arrays.asList(
-            new PathCacheDirective[] {
-        new PathCacheDirective("/alpha", "pool1"),
-        new PathCacheDirective("/beta", "pool2"),
-        new PathCacheDirective("", "pool3"),
-        new PathCacheDirective("/zeta", "nonexistent_pool"),
-        new PathCacheDirective("/zeta", "pool4")
-      }));
+      final CachePool pool1 = proto.addCachePool(new CachePoolInfo("pool1"));
+      final CachePool pool2 = proto.addCachePool(new CachePoolInfo("pool2"));
+      final CachePool pool3 = proto.addCachePool(new CachePoolInfo("pool3"));
+      final CachePool pool4 = proto.addCachePool(CachePoolInfo.newBuilder()
+          .setPoolName("pool4")
+          .setMode(new FsPermission((short)0)).build());
+      UserGroupInformation testUgi = UserGroupInformation
+          .createUserForTesting("myuser", new String[]{"mygroup"});
+      List<Fallible<PathCacheEntry>> addResults1 = testUgi.doAs(
+          new PrivilegedExceptionAction<List<Fallible<PathCacheEntry>>>() {
+            @Override
+            public List<Fallible<PathCacheEntry>> run() throws IOException {
+              List<Fallible<PathCacheEntry>> entries;
+              entries = proto.addPathCacheDirectives(
+                  Arrays.asList(new PathCacheDirective[] {
+                      new PathCacheDirective("/alpha", pool1.getId()),
+                      new PathCacheDirective("/beta", pool2.getId()),
+                      new PathCacheDirective("", pool3.getId()),
+                      new PathCacheDirective("/zeta", 404),
+                      new PathCacheDirective("/zeta", pool4.getId())
+                  }));
+              return entries;
+            }
+      });
+      // Save the successful additions
       long ids1[] = new long[2];
-      ids1[0] = addResults1.get(0).get().getEntryId();
-      ids1[1] = addResults1.get(1).get().getEntryId();
+      for (int i=0; i<2; i++) {
+        ids1[i] = addResults1.get(i).get().getEntryId();
+      }
+      // Verify that the unsuccessful additions failed properly
       try {
         addResults1.get(2).get();
         Assert.fail("expected an error when adding an empty path");
@@ -167,7 +201,7 @@ public class TestPathCacheRequests {
         addResults1.get(3).get();
         Assert.fail("expected an error when adding to a nonexistent pool.");
       } catch (IOException ioe) {
-        Assert.assertTrue(ioe.getCause() instanceof InvalidPoolNameError);
+        Assert.assertTrue(ioe.getCause() instanceof InvalidPoolError);
       }
       try {
         addResults1.get(4).get();
@@ -181,10 +215,10 @@ public class TestPathCacheRequests {
       List<Fallible<PathCacheEntry>> addResults2 = 
           proto.addPathCacheDirectives(Arrays.asList(
             new PathCacheDirective[] {
-        new PathCacheDirective("/alpha", "pool1"),
-        new PathCacheDirective("/theta", ""),
-        new PathCacheDirective("bogus", "pool1"),
-        new PathCacheDirective("/gamma", "pool1")
+        new PathCacheDirective("/alpha", pool1.getId()),
+        new PathCacheDirective("/theta", 404),
+        new PathCacheDirective("bogus", pool1.getId()),
+        new PathCacheDirective("/gamma", pool1.getId())
       }));
       long id = addResults2.get(0).get().getEntryId();
       Assert.assertEquals("expected to get back the same ID as last time " +
@@ -194,7 +228,7 @@ public class TestPathCacheRequests {
         Assert.fail("expected an error when adding a path cache " +
             "directive with an empty pool name.");
       } catch (IOException ioe) {
-        Assert.assertTrue(ioe.getCause() instanceof InvalidPoolNameError);
+        Assert.assertTrue(ioe.getCause() instanceof InvalidPoolError);
       }
       try {
         addResults2.get(2).get();
@@ -206,14 +240,16 @@ public class TestPathCacheRequests {
       long ids2[] = new long[1];
       ids2[0] = addResults2.get(3).get().getEntryId();
 
+      // Validate listing all entries
       RemoteIterator<PathCacheEntry> iter =
-          proto.listPathCacheEntries(0, "", 100);
+          proto.listPathCacheEntries(-1l, -1l, 100);
       validateListAll(iter, ids1[0], ids1[1], ids2[0]);
-      iter = proto.listPathCacheEntries(0, "", 1);
+      iter = proto.listPathCacheEntries(-1l, -1l, 1);
       validateListAll(iter, ids1[0], ids1[1], ids2[0]);
-      iter = proto.listPathCacheEntries(0, "pool3", 1);
+      // Validate listing certain pools
+      iter = proto.listPathCacheEntries(0, pool3.getId(), 1);
       Assert.assertFalse(iter.hasNext());
-      iter = proto.listPathCacheEntries(0, "pool2", 4444);
+      iter = proto.listPathCacheEntries(0, pool2.getId(), 4444);
       Assert.assertEquals(addResults1.get(1).get(),
           iter.next());
       Assert.assertFalse(iter.hasNext());
@@ -235,7 +271,7 @@ public class TestPathCacheRequests {
       } catch (IOException ioe) {
         Assert.assertTrue(ioe.getCause() instanceof NoSuchIdException);
       }
-      iter = proto.listPathCacheEntries(0, "pool2", 4444);
+      iter = proto.listPathCacheEntries(0, pool2.getId(), 4444);
       Assert.assertFalse(iter.hasNext());
     } finally {
       if (cluster != null) { cluster.shutdown(); }