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/11/21 18:12:59 UTC

svn commit: r1544252 [1/3] - in /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs: ./ src/main/java/org/apache/hadoop/hdfs/ src/main/java/org/apache/hadoop/hdfs/client/ src/main/java/org/apache/hadoop/hdfs/protocol/ src/main/java/org/apache/hadoop/h...

Author: cmccabe
Date: Thu Nov 21 17:12:58 2013
New Revision: 1544252

URL: http://svn.apache.org/r1544252
Log:
HDFS-5473. Consistent naming of user-visible caching classes and methods (cmccabe)

Added:
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CacheDirective.java
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CacheDirectiveEntry.java
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CacheDirectiveInfo.java
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CacheDirectiveStats.java
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCacheDirectives.java
Removed:
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/PathBasedCacheDirective.java
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/PathBasedCacheEntry.java
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestPathBasedCacheRequests.java
Modified:
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/HdfsAdmin.java
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CachePoolInfo.java
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/CacheReplicationMonitor.java
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/CacheManager.java
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/CacheAdmin.java
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/site/apt/CentralizedCacheManagement.apt.vm
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/OfflineEditsViewerHelper.java
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestRetryCacheWithHA.java

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt?rev=1544252&r1=1544251&r2=1544252&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt Thu Nov 21 17:12:58 2013
@@ -210,6 +210,9 @@ Trunk (Unreleased)
     HDFS-5531. Combine the getNsQuota() and getDsQuota() methods in INode.
     (szetszwo)
 
+    HDFS-5473. Consistent naming of user-visible caching classes and methods
+    (cmccabe)
+
   OPTIMIZATIONS
     HDFS-5349. DNA_CACHE and DNA_UNCACHE should be by blockId only. (cmccabe)
 

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java?rev=1544252&r1=1544251&r2=1544252&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java Thu Nov 21 17:12:58 2013
@@ -108,6 +108,7 @@ import org.apache.hadoop.fs.permission.F
 import org.apache.hadoop.hdfs.client.ClientMmapManager;
 import org.apache.hadoop.hdfs.client.HdfsDataInputStream;
 import org.apache.hadoop.hdfs.client.HdfsDataOutputStream;
+import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry;
 import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.CorruptFileBlocks;
@@ -117,7 +118,7 @@ import org.apache.hadoop.hdfs.protocol.D
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsBlocksMetadata;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
-import org.apache.hadoop.hdfs.protocol.PathBasedCacheDirective;
+import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
@@ -2290,41 +2291,41 @@ public class DFSClient implements java.i
     }
   }
 
-  public long addPathBasedCacheDirective(
-      PathBasedCacheDirective directive) throws IOException {
+  public long addCacheDirective(
+      CacheDirectiveInfo info) throws IOException {
     checkOpen();
     try {
-      return namenode.addPathBasedCacheDirective(directive);
+      return namenode.addCacheDirective(info);
     } catch (RemoteException re) {
       throw re.unwrapRemoteException();
     }
   }
   
-  public void modifyPathBasedCacheDirective(
-      PathBasedCacheDirective directive) throws IOException {
+  public void modifyCacheDirective(
+      CacheDirectiveInfo info) throws IOException {
     checkOpen();
     try {
-      namenode.modifyPathBasedCacheDirective(directive);
+      namenode.modifyCacheDirective(info);
     } catch (RemoteException re) {
       throw re.unwrapRemoteException();
     }
   }
 
-  public void removePathBasedCacheDirective(long id)
+  public void removeCacheDirective(long id)
       throws IOException {
     checkOpen();
     try {
-      namenode.removePathBasedCacheDirective(id);
+      namenode.removeCacheDirective(id);
     } catch (RemoteException re) {
       throw re.unwrapRemoteException();
     }
   }
   
-  public RemoteIterator<PathBasedCacheDirective> listPathBasedCacheDirectives(
-      PathBasedCacheDirective filter) throws IOException {
+  public RemoteIterator<CacheDirectiveEntry> listCacheDirectives(
+      CacheDirectiveInfo filter) throws IOException {
     checkOpen();
     try {
-      return namenode.listPathBasedCacheDirectives(0, filter);
+      return namenode.listCacheDirectives(0, filter);
     } catch (RemoteException re) {
       throw re.unwrapRemoteException();
     }

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java?rev=1544252&r1=1544251&r2=1544252&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java Thu Nov 21 17:12:58 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.CacheDirectiveEntry;
 import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
@@ -67,7 +68,7 @@ import org.apache.hadoop.hdfs.protocol.H
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
-import org.apache.hadoop.hdfs.protocol.PathBasedCacheDirective;
+import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
 import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
 import org.apache.hadoop.hdfs.security.token.block.InvalidBlockTokenException;
@@ -1584,78 +1585,88 @@ public class DistributedFileSystem exten
   }
 
   /**
-   * Add a new PathBasedCacheDirective.
+   * Add a new CacheDirective.
    * 
-   * @param directive A directive to add.
+   * @param info Information about a directive to add.
    * @return the ID of the directive that was created.
    * @throws IOException if the directive could not be added
    */
-  public long addPathBasedCacheDirective(
-      PathBasedCacheDirective directive) throws IOException {
-    Preconditions.checkNotNull(directive.getPath());
-    Path path = new Path(getPathName(fixRelativePart(directive.getPath()))).
+  public long addCacheDirective(
+      CacheDirectiveInfo info) throws IOException {
+    Preconditions.checkNotNull(info.getPath());
+    Path path = new Path(getPathName(fixRelativePart(info.getPath()))).
         makeQualified(getUri(), getWorkingDirectory());
-    return dfs.addPathBasedCacheDirective(
-        new PathBasedCacheDirective.Builder(directive).
+    return dfs.addCacheDirective(
+        new CacheDirectiveInfo.Builder(info).
             setPath(path).
             build());
   }
   
-  public void modifyPathBasedCacheDirective(
-      PathBasedCacheDirective directive) throws IOException {
-    if (directive.getPath() != null) {
-      directive = new PathBasedCacheDirective.Builder(directive).
-          setPath(new Path(getPathName(fixRelativePart(directive.getPath()))).
+  /**
+   * Modify a CacheDirective.
+   * 
+   * @param info Information about the directive to modify.
+   *             You must set the ID to indicate which CacheDirective you want
+   *             to modify.
+   * @throws IOException if the directive could not be modified
+   */
+  public void modifyCacheDirective(
+      CacheDirectiveInfo info) throws IOException {
+    if (info.getPath() != null) {
+      info = new CacheDirectiveInfo.Builder(info).
+          setPath(new Path(getPathName(fixRelativePart(info.getPath()))).
               makeQualified(getUri(), getWorkingDirectory())).build();
     }
-    dfs.modifyPathBasedCacheDirective(directive);
+    dfs.modifyCacheDirective(info);
   }
 
   /**
-   * Remove a PathBasedCacheDirective.
+   * Remove a CacheDirectiveInfo.
    * 
-   * @param id identifier of the PathBasedCacheDirective to remove
+   * @param id identifier of the CacheDirectiveInfo to remove
    * @throws IOException if the directive could not be removed
    */
-  public void removePathBasedCacheDirective(long id)
+  public void removeCacheDirective(long id)
       throws IOException {
-    dfs.removePathBasedCacheDirective(id);
+    dfs.removeCacheDirective(id);
   }
   
   /**
-   * List the set of cached paths of a cache pool. Incrementally fetches results
-   * from the server.
+   * List cache directives.  Incrementally fetches results from the server.
    * 
    * @param filter Filter parameters to use when listing the directives, null to
    *               list all directives visible to us.
-   * @return A RemoteIterator which returns PathBasedCacheDirective objects.
+   * @return A RemoteIterator which returns CacheDirectiveInfo objects.
    */
-  public RemoteIterator<PathBasedCacheDirective> listPathBasedCacheDirectives(
-      PathBasedCacheDirective filter) throws IOException {
+  public RemoteIterator<CacheDirectiveEntry> listCacheDirectives(
+      CacheDirectiveInfo filter) throws IOException {
     if (filter == null) {
-      filter = new PathBasedCacheDirective.Builder().build();
+      filter = new CacheDirectiveInfo.Builder().build();
     }
     if (filter.getPath() != null) {
-      filter = new PathBasedCacheDirective.Builder(filter).
+      filter = new CacheDirectiveInfo.Builder(filter).
           setPath(new Path(getPathName(fixRelativePart(filter.getPath())))).
           build();
     }
-    final RemoteIterator<PathBasedCacheDirective> iter =
-        dfs.listPathBasedCacheDirectives(filter);
-    return new RemoteIterator<PathBasedCacheDirective>() {
+    final RemoteIterator<CacheDirectiveEntry> iter =
+        dfs.listCacheDirectives(filter);
+    return new RemoteIterator<CacheDirectiveEntry>() {
       @Override
       public boolean hasNext() throws IOException {
         return iter.hasNext();
       }
 
       @Override
-      public PathBasedCacheDirective next() throws IOException {
+      public CacheDirectiveEntry next() throws IOException {
         // Although the paths we get back from the NameNode should always be
         // absolute, we call makeQualified to add the scheme and authority of
         // this DistributedFilesystem.
-        PathBasedCacheDirective desc = iter.next();
-        Path p = desc.getPath().makeQualified(getUri(), getWorkingDirectory());
-        return new PathBasedCacheDirective.Builder(desc).setPath(p).build();
+        CacheDirectiveEntry desc = iter.next();
+        CacheDirectiveInfo info = desc.getInfo();
+        Path p = info.getPath().makeQualified(getUri(), getWorkingDirectory());
+        return new CacheDirectiveEntry(
+            new CacheDirectiveInfo.Builder(info).setPath(p).build(),
+            desc.getStats());
       }
     };
   }

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/HdfsAdmin.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/HdfsAdmin.java?rev=1544252&r1=1544251&r2=1544252&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/HdfsAdmin.java (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/HdfsAdmin.java Thu Nov 21 17:12:58 2013
@@ -25,7 +25,11 @@ import org.apache.hadoop.classification.
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry;
+import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
+import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.tools.DFSAdmin;
 
@@ -121,4 +125,100 @@ public class HdfsAdmin {
   public void disallowSnapshot(Path path) throws IOException {
     dfs.disallowSnapshot(path);
   }
+
+  /**
+   * Add a new CacheDirectiveInfo.
+   * 
+   * @param info Information about a directive to add.
+   * @return the ID of the directive that was created.
+   * @throws IOException if the directive could not be added
+   */
+  public long addCacheDirective(CacheDirectiveInfo info)
+      throws IOException {
+    return dfs.addCacheDirective(info);
+  }
+  
+  /**
+   * Modify a CacheDirective.
+   * 
+   * @param info Information about the directive to modify.
+   *             You must set the ID to indicate which CacheDirective you want
+   *             to modify.
+   * @throws IOException if the directive could not be modified
+   */
+  public void modifyCacheDirective(CacheDirectiveInfo info)
+      throws IOException {
+    dfs.modifyCacheDirective(info);
+  }
+
+  /**
+   * Remove a CacheDirective.
+   * 
+   * @param id identifier of the CacheDirectiveInfo to remove
+   * @throws IOException if the directive could not be removed
+   */
+  public void removeCacheDirective(long id)
+      throws IOException {
+    dfs.removeCacheDirective(id);
+  }
+
+  /**
+   * List cache directives. Incrementally fetches results from the server.
+   * 
+   * @param filter Filter parameters to use when listing the directives, null to
+   *               list all directives visible to us.
+   * @return A RemoteIterator which returns CacheDirectiveInfo objects.
+   */
+  public RemoteIterator<CacheDirectiveEntry> listCacheDirectives(
+      CacheDirectiveInfo filter) throws IOException {
+    return dfs.listCacheDirectives(filter);
+  }
+
+  /**
+   * Add a cache pool.
+   *
+   * @param info
+   *          The request to add a cache pool.
+   * @throws IOException 
+   *          If the request could not be completed.
+   */
+  public void addCachePool(CachePoolInfo info) throws IOException {
+    dfs.addCachePool(info);
+  }
+
+  /**
+   * Modify an existing cache pool.
+   *
+   * @param info
+   *          The request to modify a cache pool.
+   * @throws IOException 
+   *          If the request could not be completed.
+   */
+  public void modifyCachePool(CachePoolInfo info) throws IOException {
+    dfs.modifyCachePool(info);
+  }
+    
+  /**
+   * Remove a cache pool.
+   *
+   * @param poolName
+   *          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 poolName) throws IOException {
+    dfs.removeCachePool(poolName);
+  }
+
+  /**
+   * 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.listCachePools();
+  }
 }

Added: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CacheDirective.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CacheDirective.java?rev=1544252&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CacheDirective.java (added)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CacheDirective.java Thu Nov 21 17:12:58 2013
@@ -0,0 +1,159 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.protocol;
+
+import org.apache.commons.lang.builder.HashCodeBuilder;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.server.namenode.CachePool;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * Represents an entry in the PathBasedCache on the NameNode.
+ *
+ * This is an implementation class, not part of the public API.
+ */
+@InterfaceAudience.Private
+public final class CacheDirective {
+  private final long entryId;
+  private final String path;
+  private final short replication;
+  private final CachePool pool;
+  private long bytesNeeded;
+  private long bytesCached;
+  private long filesAffected;
+
+  public CacheDirective(long entryId, String path,
+      short replication, CachePool pool) {
+    Preconditions.checkArgument(entryId > 0);
+    this.entryId = entryId;
+    Preconditions.checkArgument(replication > 0);
+    this.path = path;
+    Preconditions.checkNotNull(pool);
+    this.replication = replication;
+    Preconditions.checkNotNull(path);
+    this.pool = pool;
+    this.bytesNeeded = 0;
+    this.bytesCached = 0;
+    this.filesAffected = 0;
+  }
+
+  public long getEntryId() {
+    return entryId;
+  }
+
+  public String getPath() {
+    return path;
+  }
+
+  public CachePool getPool() {
+    return pool;
+  }
+
+  public short getReplication() {
+    return replication;
+  }
+
+  public CacheDirectiveInfo toDirective() {
+    return new CacheDirectiveInfo.Builder().
+        setId(entryId).
+        setPath(new Path(path)).
+        setReplication(replication).
+        setPool(pool.getPoolName()).
+        build();
+  }
+
+  public CacheDirectiveStats toStats() {
+    return new CacheDirectiveStats.Builder().
+        setBytesNeeded(bytesNeeded).
+        setBytesCached(bytesCached).
+        setFilesAffected(filesAffected).
+        build();
+  }
+
+  public CacheDirectiveEntry toEntry() {
+    return new CacheDirectiveEntry(toDirective(), toStats());
+  }
+  
+  @Override
+  public String toString() {
+    StringBuilder builder = new StringBuilder();
+    builder.append("{ entryId:").append(entryId).
+      append(", path:").append(path).
+      append(", replication:").append(replication).
+      append(", pool:").append(pool).
+      append(", bytesNeeded:").append(bytesNeeded).
+      append(", bytesCached:").append(bytesCached).
+      append(", filesAffected:").append(filesAffected).
+      append(" }");
+    return builder.toString();
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (o == null) { return false; }
+    if (o == this) { return true; }
+    if (o.getClass() != this.getClass()) {
+      return false;
+    }
+    CacheDirective other = (CacheDirective)o;
+    return entryId == other.entryId;
+  }
+
+  @Override
+  public int hashCode() {
+    return new HashCodeBuilder().append(entryId).toHashCode();
+  }
+
+  public long getBytesNeeded() {
+    return bytesNeeded;
+  }
+
+  public void clearBytesNeeded() {
+    this.bytesNeeded = 0;
+  }
+
+  public void addBytesNeeded(long toAdd) {
+    this.bytesNeeded += toAdd;
+  }
+
+  public long getBytesCached() {
+    return bytesCached;
+  }
+
+  public void clearBytesCached() {
+    this.bytesCached = 0;
+  }
+
+  public void addBytesCached(long toAdd) {
+    this.bytesCached += toAdd;
+  }
+
+  public long getFilesAffected() {
+    return filesAffected;
+  }
+
+  public void clearFilesAffected() {
+    this.filesAffected = 0;
+  }
+
+  public void incrementFilesAffected() {
+    this.filesAffected++;
+  }
+};

Added: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CacheDirectiveEntry.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CacheDirectiveEntry.java?rev=1544252&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CacheDirectiveEntry.java (added)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CacheDirectiveEntry.java Thu Nov 21 17:12:58 2013
@@ -0,0 +1,45 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.protocol;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Describes a path-based cache directive entry.
+ */
+@InterfaceStability.Evolving
+@InterfaceAudience.Public
+public class CacheDirectiveEntry {
+  private final CacheDirectiveInfo info;
+  private final CacheDirectiveStats stats;
+
+  public CacheDirectiveEntry(CacheDirectiveInfo info,
+      CacheDirectiveStats stats) {
+    this.info = info;
+    this.stats = stats;
+  }
+
+  public CacheDirectiveInfo getInfo() {
+    return info;
+  }
+
+  public CacheDirectiveStats getStats() {
+    return stats;
+  }
+};

Added: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CacheDirectiveInfo.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CacheDirectiveInfo.java?rev=1544252&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CacheDirectiveInfo.java (added)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CacheDirectiveInfo.java Thu Nov 21 17:12:58 2013
@@ -0,0 +1,201 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.protocol;
+
+import org.apache.commons.lang.builder.EqualsBuilder;
+import org.apache.commons.lang.builder.HashCodeBuilder;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.Path;
+
+/**
+ * Describes a path-based cache directive.
+ */
+@InterfaceStability.Evolving
+@InterfaceAudience.Public
+public class CacheDirectiveInfo {
+  /**
+   * A builder for creating new CacheDirectiveInfo instances.
+   */
+  public static class Builder {
+    private Long id;
+    private Path path;
+    private Short replication;
+    private String pool;
+
+    /**
+     * Builds a new CacheDirectiveInfo populated with the set properties.
+     * 
+     * @return New CacheDirectiveInfo.
+     */
+    public CacheDirectiveInfo build() {
+      return new CacheDirectiveInfo(id, path, replication, pool);
+    }
+
+    /**
+     * Creates an empty builder.
+     */
+    public Builder() {
+    }
+
+    /**
+     * Creates a builder with all elements set to the same values as the
+     * given CacheDirectiveInfo.
+     */
+    public Builder(CacheDirectiveInfo directive) {
+      this.id = directive.getId();
+      this.path = directive.getPath();
+      this.replication = directive.getReplication();
+      this.pool = directive.getPool();
+    }
+
+    /**
+     * Sets the id used in this request.
+     * 
+     * @param id The id used in this request.
+     * @return This builder, for call chaining.
+     */
+    public Builder setId(Long id) {
+      this.id = id;
+      return this;
+    }
+
+    /**
+     * Sets the path used in this request.
+     * 
+     * @param path The path used in this request.
+     * @return This builder, for call chaining.
+     */
+    public Builder setPath(Path path) {
+      this.path = path;
+      return this;
+    }
+
+    /**
+     * Sets the replication used in this request.
+     * 
+     * @param replication The replication used in this request.
+     * @return This builder, for call chaining.
+     */
+    public Builder setReplication(Short replication) {
+      this.replication = replication;
+      return this;
+    }
+
+    /**
+     * Sets the pool used in this request.
+     * 
+     * @param pool The pool used in this request.
+     * @return This builder, for call chaining.
+     */
+    public Builder setPool(String pool) {
+      this.pool = pool;
+      return this;
+    }
+  }
+
+  private final Long id;
+  private final Path path;
+  private final Short replication;
+  private final String pool;
+
+  CacheDirectiveInfo(Long id, Path path, Short replication, String pool) {
+    this.id = id;
+    this.path = path;
+    this.replication = replication;
+    this.pool = pool;
+  }
+
+  /**
+   * @return The ID of this directive.
+   */
+  public Long getId() {
+    return id;
+  }
+
+  /**
+   * @return The path used in this request.
+   */
+  public Path getPath() {
+    return path;
+  }
+
+  /**
+   * @return The number of times the block should be cached.
+   */
+  public Short getReplication() {
+    return replication;
+  }
+
+  /**
+   * @return The pool used in this request.
+   */
+  public String getPool() {
+    return pool;
+  }
+  
+  @Override
+  public boolean equals(Object o) {
+    if (o == null) {
+      return false;
+    }
+    if (getClass() != o.getClass()) {
+      return false;
+    }
+    CacheDirectiveInfo other = (CacheDirectiveInfo)o;
+    return new EqualsBuilder().append(getId(), other.getId()).
+        append(getPath(), other.getPath()).
+        append(getReplication(), other.getReplication()).
+        append(getPool(), other.getPool()).
+        isEquals();
+  }
+
+  @Override
+  public int hashCode() {
+    return new HashCodeBuilder().append(id).
+        append(path).
+        append(replication).
+        append(pool).
+        hashCode();
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder builder = new StringBuilder();
+    builder.append("{");
+    String prefix = "";
+    if (id != null) {
+      builder.append(prefix).append("id: ").append(id);
+      prefix = ",";
+    }
+    if (path != null) {
+      builder.append(prefix).append("path: ").append(path);
+      prefix = ",";
+    }
+    if (replication != null) {
+      builder.append(prefix).append("replication: ").append(replication);
+      prefix = ",";
+    }
+    if (pool != null) {
+      builder.append(prefix).append("pool: ").append(pool);
+      prefix = ",";
+    }
+    builder.append("}");
+    return builder.toString();
+  }
+};

Added: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CacheDirectiveStats.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CacheDirectiveStats.java?rev=1544252&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CacheDirectiveStats.java (added)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CacheDirectiveStats.java Thu Nov 21 17:12:58 2013
@@ -0,0 +1,125 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.protocol;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Describes a path-based cache directive.
+ */
+@InterfaceStability.Evolving
+@InterfaceAudience.Public
+public class CacheDirectiveStats {
+  public static class Builder {
+    private long bytesNeeded;
+    private long bytesCached;
+    private long filesAffected;
+
+    /**
+     * Builds a new CacheDirectiveStats populated with the set properties.
+     * 
+     * @return New CacheDirectiveStats.
+     */
+    public CacheDirectiveStats build() {
+      return new CacheDirectiveStats(bytesNeeded, bytesCached, filesAffected);
+    }
+
+    /**
+     * Creates an empty builder.
+     */
+    public Builder() {
+    }
+
+    /**
+     * Sets the bytes needed by this directive.
+     * 
+     * @param bytesNeeded The bytes needed.
+     * @return This builder, for call chaining.
+     */
+    public Builder setBytesNeeded(Long bytesNeeded) {
+      this.bytesNeeded = bytesNeeded;
+      return this;
+    }
+
+    /**
+     * Sets the bytes cached by this directive.
+     * 
+     * @param bytesCached The bytes cached.
+     * @return This builder, for call chaining.
+     */
+    public Builder setBytesCached(Long bytesCached) {
+      this.bytesCached = bytesCached;
+      return this;
+    }
+
+    /**
+     * Sets the files affected by this directive.
+     * 
+     * @param filesAffected The files affected.
+     * @return This builder, for call chaining.
+     */
+    public Builder setFilesAffected(Long filesAffected) {
+      this.filesAffected = filesAffected;
+      return this;
+    }
+  }
+
+  private final long bytesNeeded;
+  private final long bytesCached;
+  private final long filesAffected;
+
+  private CacheDirectiveStats(long bytesNeeded, long bytesCached,
+      long filesAffected) {
+    this.bytesNeeded = bytesNeeded;
+    this.bytesCached = bytesCached;
+    this.filesAffected = filesAffected;
+  }
+
+  /**
+   * @return The bytes needed.
+   */
+  public Long getBytesNeeded() {
+    return bytesNeeded;
+  }
+
+  /**
+   * @return The bytes cached.
+   */
+  public Long getBytesCached() {
+    return bytesCached;
+  }
+
+  /**
+   * @return The files affected.
+   */
+  public Long getFilesAffected() {
+    return filesAffected;
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder builder = new StringBuilder();
+    builder.append("{");
+    builder.append("bytesNeeded: ").append(bytesNeeded);
+    builder.append(", ").append("bytesCached: ").append(bytesCached);
+    builder.append(", ").append("filesAffected: ").append(filesAffected);
+    builder.append("}");
+    return builder.toString();
+  }
+};

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CachePoolInfo.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CachePoolInfo.java?rev=1544252&r1=1544251&r2=1544252&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CachePoolInfo.java (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CachePoolInfo.java Thu Nov 21 17:12:58 2013
@@ -46,7 +46,7 @@ import org.xml.sax.SAXException;
  * This class is used in RPCs to create and modify cache pools.
  * It is serializable and can be stored in the edit log.
  */
-@InterfaceAudience.Private
+@InterfaceAudience.Public
 @InterfaceStability.Evolving
 public class CachePoolInfo {
   public static final Log LOG = LogFactory.getLog(CachePoolInfo.class);
@@ -225,4 +225,4 @@ public class CachePoolInfo {
         setMode(perm.getPermission()).
         setWeight(weight);
   }
-}
\ No newline at end of file
+}

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java?rev=1544252&r1=1544251&r2=1544252&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java Thu Nov 21 17:12:58 2013
@@ -1096,49 +1096,49 @@ public interface ClientProtocol {
       String fromSnapshot, String toSnapshot) throws IOException;
 
   /**
-   * Add a PathBasedCache entry to the CacheManager.
+   * Add a CacheDirective to the CacheManager.
    * 
-   * @param directive A PathBasedCacheDirective to be added
-   * @return A PathBasedCacheDirective associated with the added directive
+   * @param directive A CacheDirectiveInfo to be added
+   * @return A CacheDirectiveInfo associated with the added directive
    * @throws IOException if the directive could not be added
    */
   @AtMostOnce
-  public long addPathBasedCacheDirective(
-      PathBasedCacheDirective directive) throws IOException;
+  public long addCacheDirective(
+      CacheDirectiveInfo directive) throws IOException;
 
   /**
-   * Modify a PathBasedCache entry in the CacheManager.
+   * Modify a CacheDirective in the CacheManager.
    * 
    * @return directive The directive to modify.  Must contain 
    *                   a directive ID.
    * @throws IOException if the directive could not be modified
    */
   @AtMostOnce
-  public void modifyPathBasedCacheDirective(
-      PathBasedCacheDirective directive) throws IOException;
+  public void modifyCacheDirective(
+      CacheDirectiveInfo directive) throws IOException;
 
   /**
-   * Remove a PathBasedCacheDirective from the CacheManager.
+   * Remove a CacheDirectiveInfo from the CacheManager.
    * 
-   * @param id of a PathBasedCacheDirective
+   * @param id of a CacheDirectiveInfo
    * @throws IOException if the cache directive could not be removed
    */
   @AtMostOnce
-  public void removePathBasedCacheDirective(long id) throws IOException;
+  public void removeCacheDirective(long id) throws IOException;
 
   /**
    * List the set of cached paths of a cache pool. Incrementally fetches results
    * from the server.
    * 
    * @param prevId The last listed entry ID, or -1 if this is the first call to
-   *               listPathBasedCacheDirectives.
+   *               listCacheDirectives.
    * @param filter Parameters to use to filter the list results, 
    *               or null to display all directives visible to us.
-   * @return A RemoteIterator which returns PathBasedCacheDirective objects.
+   * @return A RemoteIterator which returns CacheDirectiveInfo objects.
    */
   @Idempotent
-  public RemoteIterator<PathBasedCacheDirective> listPathBasedCacheDirectives(
-      long prevId, PathBasedCacheDirective filter) throws IOException;
+  public RemoteIterator<CacheDirectiveEntry> listCacheDirectives(
+      long prevId, CacheDirectiveInfo filter) throws IOException;
 
   /**
    * Add a new cache pool.

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java?rev=1544252&r1=1544251&r2=1544252&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java Thu Nov 21 17:12:58 2013
@@ -28,6 +28,7 @@ import org.apache.hadoop.fs.Options.Rena
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry;
 import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.CorruptFileBlocks;
@@ -35,7 +36,7 @@ import org.apache.hadoop.hdfs.protocol.D
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
-import org.apache.hadoop.hdfs.protocol.PathBasedCacheDirective;
+import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
 import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AbandonBlockRequestProto;
@@ -44,8 +45,8 @@ import org.apache.hadoop.hdfs.protocol.p
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AddBlockResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AddCachePoolRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AddCachePoolResponseProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AddPathBasedCacheDirectiveRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AddPathBasedCacheDirectiveResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AddCacheDirectiveRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AddCacheDirectiveResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AllowSnapshotRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AllowSnapshotResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AppendRequestProto;
@@ -106,25 +107,25 @@ import org.apache.hadoop.hdfs.protocol.p
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCachePoolsResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCorruptFileBlocksRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCorruptFileBlocksResponseProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListPathBasedCacheDirectivesElementProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListPathBasedCacheDirectivesRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListPathBasedCacheDirectivesResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CacheDirectiveEntryProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCacheDirectivesRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCacheDirectivesResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.MetaSaveRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.MetaSaveResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.MkdirsRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.MkdirsResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ModifyCachePoolRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ModifyCachePoolResponseProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ModifyPathBasedCacheDirectiveRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ModifyPathBasedCacheDirectiveResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ModifyCacheDirectiveRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ModifyCacheDirectiveResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RecoverLeaseRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RecoverLeaseResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RefreshNodesRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RefreshNodesResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RemoveCachePoolRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RemoveCachePoolResponseProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RemovePathBasedCacheDirectiveRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RemovePathBasedCacheDirectiveResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RemoveCacheDirectiveRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RemoveCacheDirectiveResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.Rename2RequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.Rename2ResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RenameRequestProto;
@@ -1030,12 +1031,12 @@ public class ClientNamenodeProtocolServe
   }
 
   @Override
-  public AddPathBasedCacheDirectiveResponseProto addPathBasedCacheDirective(
-      RpcController controller, AddPathBasedCacheDirectiveRequestProto request)
+  public AddCacheDirectiveResponseProto addCacheDirective(
+      RpcController controller, AddCacheDirectiveRequestProto request)
       throws ServiceException {
     try {
-      return AddPathBasedCacheDirectiveResponseProto.newBuilder().
-              setId(server.addPathBasedCacheDirective(
+      return AddCacheDirectiveResponseProto.newBuilder().
+              setId(server.addCacheDirective(
                   PBHelper.convert(request.getInfo()))).build();
     } catch (IOException e) {
       throw new ServiceException(e);
@@ -1043,26 +1044,26 @@ public class ClientNamenodeProtocolServe
   }
 
   @Override
-  public ModifyPathBasedCacheDirectiveResponseProto modifyPathBasedCacheDirective(
-      RpcController controller, ModifyPathBasedCacheDirectiveRequestProto request)
+  public ModifyCacheDirectiveResponseProto modifyCacheDirective(
+      RpcController controller, ModifyCacheDirectiveRequestProto request)
       throws ServiceException {
     try {
-      server.modifyPathBasedCacheDirective(
+      server.modifyCacheDirective(
           PBHelper.convert(request.getInfo()));
-      return ModifyPathBasedCacheDirectiveResponseProto.newBuilder().build();
+      return ModifyCacheDirectiveResponseProto.newBuilder().build();
     } catch (IOException e) {
       throw new ServiceException(e);
     }
   }
 
   @Override
-  public RemovePathBasedCacheDirectiveResponseProto
-      removePathBasedCacheDirective(RpcController controller,
-          RemovePathBasedCacheDirectiveRequestProto request)
+  public RemoveCacheDirectiveResponseProto
+      removeCacheDirective(RpcController controller,
+          RemoveCacheDirectiveRequestProto request)
               throws ServiceException {
     try {
-      server.removePathBasedCacheDirective(request.getId());
-      return RemovePathBasedCacheDirectiveResponseProto.
+      server.removeCacheDirective(request.getId());
+      return RemoveCacheDirectiveResponseProto.
           newBuilder().build();
     } catch (IOException e) {
       throw new ServiceException(e);
@@ -1070,28 +1071,26 @@ public class ClientNamenodeProtocolServe
   }
 
   @Override
-  public ListPathBasedCacheDirectivesResponseProto listPathBasedCacheDirectives(
-      RpcController controller, ListPathBasedCacheDirectivesRequestProto request)
+  public ListCacheDirectivesResponseProto listCacheDirectives(
+      RpcController controller, ListCacheDirectivesRequestProto request)
           throws ServiceException {
     try {
-      PathBasedCacheDirective filter =
+      CacheDirectiveInfo filter =
           PBHelper.convert(request.getFilter());
-      RemoteIterator<PathBasedCacheDirective> iter =
-         server.listPathBasedCacheDirectives(request.getPrevId(), filter);
-      ListPathBasedCacheDirectivesResponseProto.Builder builder =
-          ListPathBasedCacheDirectivesResponseProto.newBuilder();
+      RemoteIterator<CacheDirectiveEntry> iter =
+         server.listCacheDirectives(request.getPrevId(), filter);
+      ListCacheDirectivesResponseProto.Builder builder =
+          ListCacheDirectivesResponseProto.newBuilder();
       long prevId = 0;
       while (iter.hasNext()) {
-        PathBasedCacheDirective directive = iter.next();
-        builder.addElements(
-            ListPathBasedCacheDirectivesElementProto.newBuilder().
-                setInfo(PBHelper.convert(directive)));
-        prevId = directive.getId();
+        CacheDirectiveEntry entry = iter.next();
+        builder.addElements(PBHelper.convert(entry));
+        prevId = entry.getInfo().getId();
       }
       if (prevId == 0) {
         builder.setHasMore(false);
       } else {
-        iter = server.listPathBasedCacheDirectives(prevId, filter);
+        iter = server.listCacheDirectives(prevId, filter);
         builder.setHasMore(iter.hasNext());
       }
       return builder.build();

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java?rev=1544252&r1=1544251&r2=1544252&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java Thu Nov 21 17:12:58 2013
@@ -32,11 +32,11 @@ import org.apache.hadoop.fs.FileAlreadyE
 import org.apache.hadoop.fs.FsServerDefaults;
 import org.apache.hadoop.fs.Options.Rename;
 import org.apache.hadoop.fs.ParentNotDirectoryException;
-import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hadoop.fs.UnresolvedLinkException;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException;
+import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry;
 import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.CorruptFileBlocks;
@@ -51,14 +51,13 @@ import org.apache.hadoop.hdfs.protocol.H
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException;
-import org.apache.hadoop.hdfs.protocol.PathBasedCacheDirective;
+import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
 import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AbandonBlockRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AddBlockRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AddCachePoolRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AddPathBasedCacheDirectiveRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AddPathBasedCacheDirectiveResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AddCacheDirectiveRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AllowSnapshotRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AppendRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AppendResponseProto;
@@ -100,16 +99,16 @@ import org.apache.hadoop.hdfs.protocol.p
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCachePoolsResponseElementProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCachePoolsResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCorruptFileBlocksRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListPathBasedCacheDirectivesRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListPathBasedCacheDirectivesResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCacheDirectivesRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCacheDirectivesResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.MetaSaveRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.MkdirsRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ModifyCachePoolRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ModifyPathBasedCacheDirectiveRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ModifyCacheDirectiveRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RecoverLeaseRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RefreshNodesRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RemoveCachePoolRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RemovePathBasedCacheDirectiveRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RemoveCacheDirectiveRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.Rename2RequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RenameRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RenameSnapshotRequestProto;
@@ -146,7 +145,6 @@ import org.apache.hadoop.security.proto.
 import org.apache.hadoop.security.proto.SecurityProtos.RenewDelegationTokenRequestProto;
 import org.apache.hadoop.security.token.Token;
 
-import com.google.common.primitives.Shorts;
 import com.google.protobuf.ByteString;
 import com.google.protobuf.ServiceException;
 
@@ -1003,11 +1001,11 @@ public class ClientNamenodeProtocolTrans
   }
 
   @Override
-  public long addPathBasedCacheDirective(
-      PathBasedCacheDirective directive) throws IOException {
+  public long addCacheDirective(
+      CacheDirectiveInfo directive) throws IOException {
     try {
-      return rpcProxy.addPathBasedCacheDirective(null, 
-              AddPathBasedCacheDirectiveRequestProto.newBuilder().
+      return rpcProxy.addCacheDirective(null, 
+              AddCacheDirectiveRequestProto.newBuilder().
                   setInfo(PBHelper.convert(directive)).build()).getId();
     } catch (ServiceException e) {
       throw ProtobufHelper.getRemoteException(e);
@@ -1015,11 +1013,11 @@ public class ClientNamenodeProtocolTrans
   }
 
   @Override
-  public void modifyPathBasedCacheDirective(
-      PathBasedCacheDirective directive) throws IOException {
+  public void modifyCacheDirective(
+      CacheDirectiveInfo directive) throws IOException {
     try {
-      rpcProxy.modifyPathBasedCacheDirective(null,
-          ModifyPathBasedCacheDirectiveRequestProto.newBuilder().
+      rpcProxy.modifyCacheDirective(null,
+          ModifyCacheDirectiveRequestProto.newBuilder().
               setInfo(PBHelper.convert(directive)).build());
     } catch (ServiceException e) {
       throw ProtobufHelper.getRemoteException(e);
@@ -1027,29 +1025,29 @@ public class ClientNamenodeProtocolTrans
   }
 
   @Override
-  public void removePathBasedCacheDirective(long id)
+  public void removeCacheDirective(long id)
       throws IOException {
     try {
-      rpcProxy.removePathBasedCacheDirective(null,
-          RemovePathBasedCacheDirectiveRequestProto.newBuilder().
+      rpcProxy.removeCacheDirective(null,
+          RemoveCacheDirectiveRequestProto.newBuilder().
               setId(id).build());
     } catch (ServiceException e) {
       throw ProtobufHelper.getRemoteException(e);
     }
   }
 
-  private static class BatchedPathBasedCacheEntries
-      implements BatchedEntries<PathBasedCacheDirective> {
-    private ListPathBasedCacheDirectivesResponseProto response;
+  private static class BatchedCacheEntries
+      implements BatchedEntries<CacheDirectiveEntry> {
+    private ListCacheDirectivesResponseProto response;
 
-    BatchedPathBasedCacheEntries(
-        ListPathBasedCacheDirectivesResponseProto response) {
+    BatchedCacheEntries(
+        ListCacheDirectivesResponseProto response) {
       this.response = response;
     }
 
     @Override
-    public PathBasedCacheDirective get(int i) {
-      return PBHelper.convert(response.getElements(i).getInfo());
+    public CacheDirectiveEntry get(int i) {
+      return PBHelper.convert(response.getElements(i));
     }
 
     @Override
@@ -1063,46 +1061,46 @@ public class ClientNamenodeProtocolTrans
     }
   }
 
-  private class PathBasedCacheEntriesIterator
-      extends BatchedRemoteIterator<Long, PathBasedCacheDirective> {
-    private final PathBasedCacheDirective filter;
+  private class CacheEntriesIterator
+    extends BatchedRemoteIterator<Long, CacheDirectiveEntry> {
+      private final CacheDirectiveInfo filter;
 
-    public PathBasedCacheEntriesIterator(long prevKey,
-        PathBasedCacheDirective filter) {
+    public CacheEntriesIterator(long prevKey,
+        CacheDirectiveInfo filter) {
       super(prevKey);
       this.filter = filter;
     }
 
     @Override
-    public BatchedEntries<PathBasedCacheDirective> makeRequest(
+    public BatchedEntries<CacheDirectiveEntry> makeRequest(
         Long nextKey) throws IOException {
-      ListPathBasedCacheDirectivesResponseProto response;
+      ListCacheDirectivesResponseProto response;
       try {
-        response = rpcProxy.listPathBasedCacheDirectives(null,
-            ListPathBasedCacheDirectivesRequestProto.newBuilder().
+        response = rpcProxy.listCacheDirectives(null,
+            ListCacheDirectivesRequestProto.newBuilder().
                 setPrevId(nextKey).
                 setFilter(PBHelper.convert(filter)).
                 build());
       } catch (ServiceException e) {
         throw ProtobufHelper.getRemoteException(e);
       }
-      return new BatchedPathBasedCacheEntries(response);
+      return new BatchedCacheEntries(response);
     }
 
     @Override
-    public Long elementToPrevKey(PathBasedCacheDirective element) {
-      return element.getId();
+    public Long elementToPrevKey(CacheDirectiveEntry element) {
+      return element.getInfo().getId();
     }
   }
 
   @Override
-  public RemoteIterator<PathBasedCacheDirective>
-      listPathBasedCacheDirectives(long prevId,
-          PathBasedCacheDirective filter) throws IOException {
+  public RemoteIterator<CacheDirectiveEntry>
+      listCacheDirectives(long prevId,
+          CacheDirectiveInfo filter) throws IOException {
     if (filter == null) {
-      filter = new PathBasedCacheDirective.Builder().build();
+      filter = new CacheDirectiveInfo.Builder().build();
     }
-    return new PathBasedCacheEntriesIterator(prevId, filter);
+    return new CacheEntriesIterator(prevId, filter);
   }
 
   @Override
@@ -1140,11 +1138,11 @@ public class ClientNamenodeProtocolTrans
     }
   }
 
-  private static class BatchedPathDirectiveEntries
-      implements BatchedEntries<CachePoolInfo> {
-    private final ListCachePoolsResponseProto proto;
+  private static class BatchedCachePoolInfo
+    implements BatchedEntries<CachePoolInfo> {
+      private final ListCachePoolsResponseProto proto;
     
-    public BatchedPathDirectiveEntries(ListCachePoolsResponseProto proto) {
+    public BatchedCachePoolInfo(ListCachePoolsResponseProto proto) {
       this.proto = proto;
     }
       
@@ -1176,7 +1174,7 @@ public class ClientNamenodeProtocolTrans
     public BatchedEntries<CachePoolInfo> makeRequest(String prevKey)
         throws IOException {
       try {
-        return new BatchedPathDirectiveEntries(
+        return new BatchedCachePoolInfo(
             rpcProxy.listCachePools(null, 
               ListCachePoolsRequestProto.newBuilder().
                 setPrevPoolName(prevKey).build()));

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java?rev=1544252&r1=1544251&r2=1544252&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java Thu Nov 21 17:12:58 2013
@@ -36,12 +36,14 @@ import org.apache.hadoop.ha.HAServicePro
 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.CacheDirectiveEntry;
+import org.apache.hadoop.hdfs.protocol.CacheDirectiveStats;
 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;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
-import org.apache.hadoop.hdfs.protocol.PathBasedCacheDirective;
+import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo.AdminStates;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
@@ -56,12 +58,14 @@ import org.apache.hadoop.hdfs.protocol.L
 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.CacheDirectiveEntryProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CacheDirectiveStatsProto;
 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;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SafeModeActionProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.PathBasedCacheDirectiveInfoProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CacheDirectiveInfoProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BalancerBandwidthCommandProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockCommandProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockIdCommandProto;
@@ -1567,38 +1571,29 @@ public class PBHelper {
     return DataChecksum.Type.valueOf(type.getNumber());
   }
 
-  public static PathBasedCacheDirectiveInfoProto convert
-      (PathBasedCacheDirective directive) {
-    PathBasedCacheDirectiveInfoProto.Builder builder = 
-        PathBasedCacheDirectiveInfoProto.newBuilder();
-    if (directive.getId() != null) {
-      builder.setId(directive.getId());
-    }
-    if (directive.getPath() != null) {
-      builder.setPath(directive.getPath().toUri().getPath());
-    }
-    if (directive.getReplication() != null) {
-      builder.setReplication(directive.getReplication());
-    }
-    if (directive.getPool() != null) {
-      builder.setPool(directive.getPool());
+  public static CacheDirectiveInfoProto convert
+      (CacheDirectiveInfo info) {
+    CacheDirectiveInfoProto.Builder builder = 
+        CacheDirectiveInfoProto.newBuilder();
+    if (info.getId() != null) {
+      builder.setId(info.getId());
     }
-    if (directive.getBytesNeeded() != null) {
-      builder.setBytesNeeded(directive.getBytesNeeded());
+    if (info.getPath() != null) {
+      builder.setPath(info.getPath().toUri().getPath());
     }
-    if (directive.getBytesCached() != null) {
-      builder.setBytesCached(directive.getBytesCached());
+    if (info.getReplication() != null) {
+      builder.setReplication(info.getReplication());
     }
-    if (directive.getFilesAffected() != null) {
-      builder.setFilesAffected(directive.getFilesAffected());
+    if (info.getPool() != null) {
+      builder.setPool(info.getPool());
     }
     return builder.build();
   }
 
-  public static PathBasedCacheDirective convert
-      (PathBasedCacheDirectiveInfoProto proto) {
-    PathBasedCacheDirective.Builder builder =
-        new PathBasedCacheDirective.Builder();
+  public static CacheDirectiveInfo convert
+      (CacheDirectiveInfoProto proto) {
+    CacheDirectiveInfo.Builder builder =
+        new CacheDirectiveInfo.Builder();
     if (proto.hasId()) {
       builder.setId(proto.getId());
     }
@@ -1612,18 +1607,40 @@ public class PBHelper {
     if (proto.hasPool()) {
       builder.setPool(proto.getPool());
     }
-    if (proto.hasBytesNeeded()) {
-      builder.setBytesNeeded(proto.getBytesNeeded());
-    }
-    if (proto.hasBytesCached()) {
-      builder.setBytesCached(proto.getBytesCached());
-    }
-    if (proto.hasFilesAffected()) {
-      builder.setFilesAffected(proto.getFilesAffected());
-    }
     return builder.build();
   }
   
+  public static CacheDirectiveStatsProto convert(CacheDirectiveStats stats) {
+    CacheDirectiveStatsProto.Builder builder = 
+        CacheDirectiveStatsProto.newBuilder();
+    builder.setBytesNeeded(stats.getBytesNeeded());
+    builder.setBytesCached(stats.getBytesCached());
+    builder.setFilesAffected(stats.getFilesAffected());
+    return builder.build();
+  }
+  
+  public static CacheDirectiveStats convert(CacheDirectiveStatsProto proto) {
+    CacheDirectiveStats.Builder builder = new CacheDirectiveStats.Builder();
+    builder.setBytesNeeded(proto.getBytesNeeded());
+    builder.setBytesCached(proto.getBytesCached());
+    builder.setFilesAffected(proto.getFilesAffected());
+    return builder.build();
+  }
+
+  public static CacheDirectiveEntryProto convert(CacheDirectiveEntry entry) {
+    CacheDirectiveEntryProto.Builder builder = 
+        CacheDirectiveEntryProto.newBuilder();
+    builder.setInfo(PBHelper.convert(entry.getInfo()));
+    builder.setStats(PBHelper.convert(entry.getStats()));
+    return builder.build();
+  }
+  
+  public static CacheDirectiveEntry convert(CacheDirectiveEntryProto proto) {
+    CacheDirectiveInfo info = PBHelper.convert(proto.getInfo());
+    CacheDirectiveStats stats = PBHelper.convert(proto.getStats());
+    return new CacheDirectiveEntry(info, stats);
+  }
+
   public static CachePoolInfoProto convert(CachePoolInfo info) {
     CachePoolInfoProto.Builder builder = CachePoolInfoProto.newBuilder();
     builder.setPoolName(info.getPoolName());

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/CacheReplicationMonitor.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/CacheReplicationMonitor.java?rev=1544252&r1=1544251&r2=1544252&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/CacheReplicationMonitor.java (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/CacheReplicationMonitor.java Thu Nov 21 17:12:58 2013
@@ -32,7 +32,7 @@ import org.apache.commons.logging.LogFac
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.fs.UnresolvedLinkException;
 import org.apache.hadoop.hdfs.protocol.Block;
-import org.apache.hadoop.hdfs.protocol.PathBasedCacheEntry;
+import org.apache.hadoop.hdfs.protocol.CacheDirective;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor.CachedBlocksList.Type;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.namenode.CacheManager;
@@ -197,7 +197,7 @@ public class CacheReplicationMonitor ext
     scannedBlocks = 0;
     namesystem.writeLock();
     try {
-      rescanPathBasedCacheEntries();
+      rescanCacheDirectives();
       rescanCachedBlockMap();
       blockManager.getDatanodeManager().resetLastCachingDirectiveSentTime();
     } finally {
@@ -206,14 +206,14 @@ public class CacheReplicationMonitor ext
   }
 
   /**
-   * Scan all PathBasedCacheEntries.  Use the information to figure out
+   * Scan all CacheDirectives.  Use the information to figure out
    * what cache replication factor each block should have.
    *
    * @param mark       Whether the current scan is setting or clearing the mark
    */
-  private void rescanPathBasedCacheEntries() {
+  private void rescanCacheDirectives() {
     FSDirectory fsDir = namesystem.getFSDirectory();
-    for (PathBasedCacheEntry pce : cacheManager.getEntriesById().values()) {
+    for (CacheDirective pce : cacheManager.getEntriesById().values()) {
       scannedDirectives++;
       pce.clearBytesNeeded();
       pce.clearBytesCached();
@@ -250,12 +250,12 @@ public class CacheReplicationMonitor ext
   }
   
   /**
-   * Apply a PathBasedCacheEntry to a file.
+   * Apply a CacheDirective to a file.
    *
-   * @param pce       The PathBasedCacheEntry to apply.
+   * @param pce       The CacheDirective to apply.
    * @param file      The file.
    */
-  private void rescanFile(PathBasedCacheEntry pce, INodeFile file) {
+  private void rescanFile(CacheDirective pce, INodeFile file) {
     pce.incrementFilesAffected();
     BlockInfo[] blockInfos = file.getBlocks();
     long cachedTotal = 0;
@@ -292,7 +292,7 @@ public class CacheReplicationMonitor ext
           ocblock.setReplicationAndMark(pce.getReplication(), mark);
         } else {
           // Mark already set in this scan.  Set replication to highest value in
-          // any PathBasedCacheEntry that covers this file.
+          // any CacheDirective that covers this file.
           ocblock.setReplicationAndMark((short)Math.max(
               pce.getReplication(), ocblock.getReplication()), mark);
         }

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/CacheManager.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/CacheManager.java?rev=1544252&r1=1544251&r2=1544252&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/CacheManager.java (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/CacheManager.java Thu Nov 21 17:12:58 2013
@@ -48,11 +48,12 @@ import org.apache.hadoop.fs.BatchedRemot
 import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry;
 import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
-import org.apache.hadoop.hdfs.protocol.PathBasedCacheDirective;
-import org.apache.hadoop.hdfs.protocol.PathBasedCacheEntry;
+import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
+import org.apache.hadoop.hdfs.protocol.CacheDirective;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.CacheReplicationMonitor;
@@ -100,11 +101,11 @@ public final class CacheManager {
   /**
    * Cache entries, sorted by ID.
    *
-   * listPathBasedCacheDirectives relies on the ordering of elements in this map
+   * listCacheDirectives relies on the ordering of elements in this map
    * to track what has already been listed by the client.
    */
-  private final TreeMap<Long, PathBasedCacheEntry> entriesById =
-      new TreeMap<Long, PathBasedCacheEntry>();
+  private final TreeMap<Long, CacheDirective> entriesById =
+      new TreeMap<Long, CacheDirective>();
 
   /**
    * The entry ID to use for a new entry.  Entry IDs always increase, and are
@@ -115,8 +116,8 @@ public final class CacheManager {
   /**
    * Cache entries, sorted by path
    */
-  private final TreeMap<String, List<PathBasedCacheEntry>> entriesByPath =
-      new TreeMap<String, List<PathBasedCacheEntry>>();
+  private final TreeMap<String, List<CacheDirective>> entriesByPath =
+      new TreeMap<String, List<CacheDirective>>();
 
   /**
    * Cache pools, sorted by name.
@@ -236,7 +237,7 @@ public final class CacheManager {
     return active;
   }
 
-  public TreeMap<Long, PathBasedCacheEntry> getEntriesById() {
+  public TreeMap<Long, CacheDirective> getEntriesById() {
     assert namesystem.hasReadLock();
     return entriesById;
   }
@@ -264,7 +265,7 @@ public final class CacheManager {
     }
   }
 
-  private static String validatePoolName(PathBasedCacheDirective directive)
+  private static String validatePoolName(CacheDirectiveInfo directive)
       throws InvalidRequestException {
     String pool = directive.getPool();
     if (pool == null) {
@@ -276,7 +277,7 @@ public final class CacheManager {
     return pool;
   }
 
-  private static String validatePath(PathBasedCacheDirective directive)
+  private static String validatePath(CacheDirectiveInfo directive)
       throws InvalidRequestException {
     if (directive.getPath() == null) {
       throw new InvalidRequestException("No path specified.");
@@ -288,7 +289,7 @@ public final class CacheManager {
     return path;
   }
 
-  private static short validateReplication(PathBasedCacheDirective directive,
+  private static short validateReplication(CacheDirectiveInfo directive,
       short defaultValue) throws InvalidRequestException {
     short repl = (directive.getReplication() != null)
         ? directive.getReplication() : defaultValue;
@@ -300,16 +301,16 @@ public final class CacheManager {
   }
 
   /**
-   * Get a PathBasedCacheEntry by ID, validating the ID and that the entry
+   * Get a CacheDirective by ID, validating the ID and that the entry
    * exists.
    */
-  private PathBasedCacheEntry getById(long id) throws InvalidRequestException {
+  private CacheDirective 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);
+    CacheDirective entry = entriesById.get(id);
     if (entry == null) {
       throw new InvalidRequestException("No directive with ID " + id
           + " found.");
@@ -331,22 +332,22 @@ public final class CacheManager {
 
   // RPC handlers
 
-  private void addInternal(PathBasedCacheEntry entry) {
+  private void addInternal(CacheDirective entry) {
     entriesById.put(entry.getEntryId(), entry);
     String path = entry.getPath();
-    List<PathBasedCacheEntry> entryList = entriesByPath.get(path);
+    List<CacheDirective> entryList = entriesByPath.get(path);
     if (entryList == null) {
-      entryList = new ArrayList<PathBasedCacheEntry>(1);
+      entryList = new ArrayList<CacheDirective>(1);
       entriesByPath.put(path, entryList);
     }
     entryList.add(entry);
   }
 
-  public PathBasedCacheDirective addDirective(
-      PathBasedCacheDirective directive, FSPermissionChecker pc)
+  public CacheDirectiveInfo addDirective(
+      CacheDirectiveInfo directive, FSPermissionChecker pc)
       throws IOException {
     assert namesystem.hasWriteLock();
-    PathBasedCacheEntry entry;
+    CacheDirective entry;
     try {
       CachePool pool = getCachePool(validatePoolName(directive));
       checkWritePermission(pc, pool);
@@ -372,7 +373,7 @@ public final class CacheManager {
         // Add a new entry with the next available ID.
         id = getNextEntryId();
       }
-      entry = new PathBasedCacheEntry(id, path, replication, pool);
+      entry = new CacheDirective(id, path, replication, pool);
       addInternal(entry);
     } catch (IOException e) {
       LOG.warn("addDirective of " + directive + " failed: ", e);
@@ -385,7 +386,7 @@ public final class CacheManager {
     return entry.toDirective();
   }
 
-  public void modifyDirective(PathBasedCacheDirective directive,
+  public void modifyDirective(CacheDirectiveInfo directive,
       FSPermissionChecker pc) throws IOException {
     assert namesystem.hasWriteLock();
     String idString =
@@ -397,7 +398,7 @@ public final class CacheManager {
       if (id == null) {
         throw new InvalidRequestException("Must supply an ID.");
       }
-      PathBasedCacheEntry prevEntry = getById(id);
+      CacheDirective prevEntry = getById(id);
       checkWritePermission(pc, prevEntry.getPool());
       String path = prevEntry.getPath();
       if (directive.getPath() != null) {
@@ -413,8 +414,8 @@ public final class CacheManager {
         checkWritePermission(pc, pool);
       }
       removeInternal(prevEntry);
-      PathBasedCacheEntry newEntry =
-          new PathBasedCacheEntry(id, path, replication, pool);
+      CacheDirective newEntry =
+          new CacheDirective(id, path, replication, pool);
       addInternal(newEntry);
     } catch (IOException e) {
       LOG.warn("modifyDirective of " + idString + " failed: ", e);
@@ -424,12 +425,12 @@ public final class CacheManager {
         directive + ".");
   }
 
-  public void removeInternal(PathBasedCacheEntry existing)
+  public void removeInternal(CacheDirective existing)
       throws InvalidRequestException {
     assert namesystem.hasWriteLock();
     // Remove the corresponding entry in entriesByPath.
     String path = existing.getPath();
-    List<PathBasedCacheEntry> entries = entriesByPath.get(path);
+    List<CacheDirective> entries = entriesByPath.get(path);
     if (entries == null || !entries.remove(existing)) {
       throw new InvalidRequestException("Failed to locate entry " +
           existing.getEntryId() + " by path " + existing.getPath());
@@ -444,7 +445,7 @@ public final class CacheManager {
       throws IOException {
     assert namesystem.hasWriteLock();
     try {
-      PathBasedCacheEntry existing = getById(id);
+      CacheDirective existing = getById(id);
       checkWritePermission(pc, existing.getPool());
       removeInternal(existing);
     } catch (IOException e) {
@@ -457,9 +458,9 @@ public final class CacheManager {
     LOG.info("removeDirective of " + id + " successful.");
   }
 
-  public BatchedListEntries<PathBasedCacheDirective> 
-        listPathBasedCacheDirectives(long prevId,
-            PathBasedCacheDirective filter,
+  public BatchedListEntries<CacheDirectiveEntry> 
+        listCacheDirectives(long prevId,
+            CacheDirectiveInfo filter,
             FSPermissionChecker pc) throws IOException {
     assert namesystem.hasReadLock();
     final int NUM_PRE_ALLOCATED_ENTRIES = 16;
@@ -473,23 +474,23 @@ public final class CacheManager {
     if (filter.getReplication() != null) {
       throw new IOException("Filtering by replication is unsupported.");
     }
-    ArrayList<PathBasedCacheDirective> replies =
-        new ArrayList<PathBasedCacheDirective>(NUM_PRE_ALLOCATED_ENTRIES);
+    ArrayList<CacheDirectiveEntry> replies =
+        new ArrayList<CacheDirectiveEntry>(NUM_PRE_ALLOCATED_ENTRIES);
     int numReplies = 0;
-    SortedMap<Long, PathBasedCacheEntry> tailMap =
+    SortedMap<Long, CacheDirective> tailMap =
       entriesById.tailMap(prevId + 1);
-    for (Entry<Long, PathBasedCacheEntry> cur : tailMap.entrySet()) {
+    for (Entry<Long, CacheDirective> cur : tailMap.entrySet()) {
       if (numReplies >= maxListCacheDirectivesNumResponses) {
-        return new BatchedListEntries<PathBasedCacheDirective>(replies, true);
+        return new BatchedListEntries<CacheDirectiveEntry>(replies, true);
       }
-      PathBasedCacheEntry curEntry = cur.getValue();
-      PathBasedCacheDirective directive = cur.getValue().toDirective();
+      CacheDirective curEntry = cur.getValue();
+      CacheDirectiveInfo info = cur.getValue().toDirective();
       if (filter.getPool() != null && 
-          !directive.getPool().equals(filter.getPool())) {
+          !info.getPool().equals(filter.getPool())) {
         continue;
       }
       if (filterPath != null &&
-          !directive.getPath().toUri().getPath().equals(filterPath)) {
+          !info.getPath().toUri().getPath().equals(filterPath)) {
         continue;
       }
       boolean hasPermission = true;
@@ -501,11 +502,11 @@ public final class CacheManager {
         }
       }
       if (hasPermission) {
-        replies.add(cur.getValue().toDirective());
+        replies.add(new CacheDirectiveEntry(info, cur.getValue().toStats()));
         numReplies++;
       }
     }
-    return new BatchedListEntries<PathBasedCacheDirective>(replies, false);
+    return new BatchedListEntries<CacheDirectiveEntry>(replies, false);
   }
 
   /**
@@ -602,10 +603,10 @@ public final class CacheManager {
     // Remove entries using this pool
     // TODO: could optimize this somewhat to avoid the need to iterate
     // over all entries in entriesById
-    Iterator<Entry<Long, PathBasedCacheEntry>> iter = 
+    Iterator<Entry<Long, CacheDirective>> iter = 
         entriesById.entrySet().iterator();
     while (iter.hasNext()) {
-      Entry<Long, PathBasedCacheEntry> entry = iter.next();
+      Entry<Long, CacheDirective> entry = iter.next();
       if (entry.getValue().getPool() == pool) {
         entriesByPath.remove(entry.getValue().getPath());
         iter.remove();
@@ -789,7 +790,7 @@ public final class CacheManager {
     prog.setTotal(Phase.SAVING_CHECKPOINT, step, entriesById.size());
     Counter counter = prog.getCounter(Phase.SAVING_CHECKPOINT, step);
     out.writeInt(entriesById.size());
-    for (PathBasedCacheEntry entry: entriesById.values()) {
+    for (CacheDirective entry: entriesById.values()) {
       out.writeLong(entry.getEntryId());
       Text.writeString(out, entry.getPath());
       out.writeShort(entry.getReplication());
@@ -838,15 +839,15 @@ public final class CacheManager {
         throw new IOException("Entry refers to pool " + poolName +
             ", which does not exist.");
       }
-      PathBasedCacheEntry entry =
-          new PathBasedCacheEntry(entryId, path, replication, pool);
+      CacheDirective entry =
+          new CacheDirective(entryId, path, replication, pool);
       if (entriesById.put(entry.getEntryId(), entry) != null) {
         throw new IOException("An entry with ID " + entry.getEntryId() +
             " already exists");
       }
-      List<PathBasedCacheEntry> entries = entriesByPath.get(entry.getPath());
+      List<CacheDirective> entries = entriesByPath.get(entry.getPath());
       if (entries == null) {
-        entries = new LinkedList<PathBasedCacheEntry>();
+        entries = new LinkedList<CacheDirective>();
         entriesByPath.put(entry.getPath(), entries);
       }
       entries.add(entry);