|
@@ -19,14 +19,13 @@ package org.apache.hadoop.hdfs.server.namenode;
|
|
|
|
|
|
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_LIST_CACHE_POOLS_NUM_RESPONSES;
|
|
|
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_LIST_CACHE_POOLS_NUM_RESPONSES_DEFAULT;
|
|
|
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_LIST_CACHE_DESCRIPTORS_NUM_RESPONSES;
|
|
|
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_LIST_CACHE_DIRECTIVES_NUM_RESPONSES;
|
|
|
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_CACHING_ENABLED_KEY;
|
|
|
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_CACHING_ENABLED_DEFAULT;
|
|
|
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_LIST_CACHE_DESCRIPTORS_NUM_RESPONSES_DEFAULT;
|
|
|
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_LIST_CACHE_DIRECTIVES_NUM_RESPONSES_DEFAULT;
|
|
|
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_PATH_BASED_CACHE_REFRESH_INTERVAL_MS;
|
|
|
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_PATH_BASED_CACHE_REFRESH_INTERVAL_MS_DEFAULT;
|
|
|
|
|
|
-import java.io.Closeable;
|
|
|
import java.io.DataInput;
|
|
|
import java.io.DataOutput;
|
|
|
import java.io.IOException;
|
|
@@ -44,33 +43,22 @@ import org.apache.commons.logging.Log;
|
|
|
import org.apache.commons.logging.LogFactory;
|
|
|
import org.apache.hadoop.classification.InterfaceAudience;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
+import org.apache.hadoop.fs.IdNotFoundException;
|
|
|
import org.apache.hadoop.fs.BatchedRemoteIterator.BatchedListEntries;
|
|
|
-import org.apache.hadoop.fs.Path;
|
|
|
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.BlockListAsLongs;
|
|
|
-import org.apache.hadoop.hdfs.protocol.BlockListAsLongs.BlockReportIterator;
|
|
|
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.PathBasedCacheDescriptor;
|
|
|
-import org.apache.hadoop.hdfs.protocol.AddPathBasedCacheDirectiveException.InvalidPoolNameError;
|
|
|
-import org.apache.hadoop.hdfs.protocol.AddPathBasedCacheDirectiveException.UnexpectedAddPathBasedCacheDirectiveException;
|
|
|
-import org.apache.hadoop.hdfs.protocol.AddPathBasedCacheDirectiveException.PoolWritePermissionDeniedError;
|
|
|
import org.apache.hadoop.hdfs.protocol.PathBasedCacheEntry;
|
|
|
-import org.apache.hadoop.hdfs.protocol.RemovePathBasedCacheDescriptorException.InvalidIdException;
|
|
|
-import org.apache.hadoop.hdfs.protocol.RemovePathBasedCacheDescriptorException.NoSuchIdException;
|
|
|
-import org.apache.hadoop.hdfs.protocol.RemovePathBasedCacheDescriptorException.UnexpectedRemovePathBasedCacheDescriptorException;
|
|
|
-import org.apache.hadoop.hdfs.protocol.RemovePathBasedCacheDescriptorException.RemovePermissionDeniedException;
|
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
|
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
|
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.CacheReplicationMonitor;
|
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
|
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor.CachedBlocksList;
|
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor.CachedBlocksList.Type;
|
|
|
-import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.metrics.NameNodeMetrics;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.startupprogress.Phase;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.startupprogress.StartupProgress;
|
|
@@ -78,6 +66,7 @@ import org.apache.hadoop.hdfs.server.namenode.startupprogress.StartupProgress.Co
|
|
|
import org.apache.hadoop.hdfs.server.namenode.startupprogress.Step;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.startupprogress.StepType;
|
|
|
import org.apache.hadoop.io.Text;
|
|
|
+import org.apache.hadoop.security.AccessControlException;
|
|
|
import org.apache.hadoop.util.GSet;
|
|
|
import org.apache.hadoop.util.LightWeightGSet;
|
|
|
import org.apache.hadoop.util.Time;
|
|
@@ -111,7 +100,7 @@ public final class CacheManager {
|
|
|
/**
|
|
|
* Cache entries, sorted by ID.
|
|
|
*
|
|
|
- * listPathBasedCacheDescriptors relies on the ordering of elements in this map
|
|
|
+ * listPathBasedCacheDirectives 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 =
|
|
@@ -143,7 +132,7 @@ public final class CacheManager {
|
|
|
/**
|
|
|
* Maximum number of cache pool directives to list in one operation.
|
|
|
*/
|
|
|
- private final int maxListCacheDescriptorsResponses;
|
|
|
+ private final int maxListCacheDirectivesNumResponses;
|
|
|
|
|
|
/**
|
|
|
* Interval between scans in milliseconds.
|
|
@@ -191,9 +180,9 @@ public final class CacheManager {
|
|
|
this.maxListCachePoolsResponses = conf.getInt(
|
|
|
DFS_NAMENODE_LIST_CACHE_POOLS_NUM_RESPONSES,
|
|
|
DFS_NAMENODE_LIST_CACHE_POOLS_NUM_RESPONSES_DEFAULT);
|
|
|
- this.maxListCacheDescriptorsResponses = conf.getInt(
|
|
|
- DFS_NAMENODE_LIST_CACHE_DESCRIPTORS_NUM_RESPONSES,
|
|
|
- DFS_NAMENODE_LIST_CACHE_DESCRIPTORS_NUM_RESPONSES_DEFAULT);
|
|
|
+ this.maxListCacheDirectivesNumResponses = conf.getInt(
|
|
|
+ DFS_NAMENODE_LIST_CACHE_DIRECTIVES_NUM_RESPONSES,
|
|
|
+ DFS_NAMENODE_LIST_CACHE_DIRECTIVES_NUM_RESPONSES_DEFAULT);
|
|
|
scanIntervalMs = conf.getLong(
|
|
|
DFS_NAMENODE_PATH_BASED_CACHE_REFRESH_INTERVAL_MS,
|
|
|
DFS_NAMENODE_PATH_BASED_CACHE_REFRESH_INTERVAL_MS_DEFAULT);
|
|
@@ -266,132 +255,239 @@ public final class CacheManager {
|
|
|
return nextEntryId++;
|
|
|
}
|
|
|
|
|
|
- public PathBasedCacheDescriptor addDirective(
|
|
|
- PathBasedCacheDirective directive, FSPermissionChecker pc)
|
|
|
- throws IOException {
|
|
|
- assert namesystem.hasWriteLock();
|
|
|
- CachePool pool = cachePools.get(directive.getPool());
|
|
|
- if (pool == null) {
|
|
|
- LOG.info("addDirective " + directive + ": pool not found.");
|
|
|
- throw new InvalidPoolNameError(directive);
|
|
|
- }
|
|
|
- if ((pc != null) && (!pc.checkPermission(pool, FsAction.WRITE))) {
|
|
|
- LOG.info("addDirective " + directive + ": write permission denied.");
|
|
|
- throw new PoolWritePermissionDeniedError(directive);
|
|
|
- }
|
|
|
- try {
|
|
|
- directive.validate();
|
|
|
- } catch (IOException ioe) {
|
|
|
- LOG.info("addDirective " + directive + ": validation failed: "
|
|
|
- + ioe.getClass().getName() + ": " + ioe.getMessage());
|
|
|
- throw ioe;
|
|
|
- }
|
|
|
-
|
|
|
- // Add a new entry with the next available ID.
|
|
|
- PathBasedCacheEntry entry;
|
|
|
- try {
|
|
|
- entry = new PathBasedCacheEntry(getNextEntryId(),
|
|
|
- directive.getPath().toUri().getPath(),
|
|
|
- directive.getReplication(), pool);
|
|
|
- } catch (IOException ioe) {
|
|
|
- throw new UnexpectedAddPathBasedCacheDirectiveException(directive);
|
|
|
- }
|
|
|
- LOG.info("addDirective " + directive + ": added cache directive "
|
|
|
- + directive);
|
|
|
-
|
|
|
- // Success!
|
|
|
- // First, add it to the various maps
|
|
|
+ private void addInternal(PathBasedCacheEntry entry) {
|
|
|
entriesById.put(entry.getEntryId(), entry);
|
|
|
- String path = directive.getPath().toUri().getPath();
|
|
|
+ String path = entry.getPath();
|
|
|
List<PathBasedCacheEntry> entryList = entriesByPath.get(path);
|
|
|
if (entryList == null) {
|
|
|
entryList = new ArrayList<PathBasedCacheEntry>(1);
|
|
|
entriesByPath.put(path, entryList);
|
|
|
}
|
|
|
entryList.add(entry);
|
|
|
+ }
|
|
|
+
|
|
|
+ public PathBasedCacheDirective addDirective(
|
|
|
+ PathBasedCacheDirective directive, FSPermissionChecker pc)
|
|
|
+ throws IOException {
|
|
|
+ assert namesystem.hasWriteLock();
|
|
|
+ PathBasedCacheEntry entry;
|
|
|
+ try {
|
|
|
+ if (directive.getPool() == null) {
|
|
|
+ throw new IdNotFoundException("addDirective: no pool was specified.");
|
|
|
+ }
|
|
|
+ if (directive.getPool().isEmpty()) {
|
|
|
+ throw new IdNotFoundException("addDirective: pool name was empty.");
|
|
|
+ }
|
|
|
+ CachePool pool = cachePools.get(directive.getPool());
|
|
|
+ if (pool == null) {
|
|
|
+ throw new IdNotFoundException("addDirective: no such pool as " +
|
|
|
+ directive.getPool());
|
|
|
+ }
|
|
|
+ if ((pc != null) && (!pc.checkPermission(pool, FsAction.WRITE))) {
|
|
|
+ throw new AccessControlException("addDirective: write " +
|
|
|
+ "permission denied for pool " + directive.getPool());
|
|
|
+ }
|
|
|
+ if (directive.getPath() == null) {
|
|
|
+ throw new IOException("addDirective: no path was specified.");
|
|
|
+ }
|
|
|
+ String path = directive.getPath().toUri().getPath();
|
|
|
+ if (!DFSUtil.isValidName(path)) {
|
|
|
+ throw new IOException("addDirective: path '" + path + "' is invalid.");
|
|
|
+ }
|
|
|
+ short replication = directive.getReplication() == null ?
|
|
|
+ (short)1 : directive.getReplication();
|
|
|
+ if (replication <= 0) {
|
|
|
+ throw new IOException("addDirective: replication " + replication +
|
|
|
+ " is invalid.");
|
|
|
+ }
|
|
|
+ long id;
|
|
|
+ if (directive.getId() != null) {
|
|
|
+ // We are loading an entry from the edit log.
|
|
|
+ // Use the ID from the edit log.
|
|
|
+ id = directive.getId();
|
|
|
+ } else {
|
|
|
+ // Add a new entry with the next available ID.
|
|
|
+ id = getNextEntryId();
|
|
|
+ }
|
|
|
+ entry = new PathBasedCacheEntry(id, path, replication, pool);
|
|
|
+ addInternal(entry);
|
|
|
+ } catch (IOException e) {
|
|
|
+ LOG.warn("addDirective " + directive + ": failed.", e);
|
|
|
+ throw e;
|
|
|
+ }
|
|
|
+ LOG.info("addDirective " + directive + ": succeeded.");
|
|
|
if (monitor != null) {
|
|
|
monitor.kick();
|
|
|
}
|
|
|
- return entry.getDescriptor();
|
|
|
+ return entry.toDirective();
|
|
|
+ }
|
|
|
+
|
|
|
+ public void modifyDirective(PathBasedCacheDirective directive,
|
|
|
+ FSPermissionChecker pc) throws IOException {
|
|
|
+ assert namesystem.hasWriteLock();
|
|
|
+ String idString =
|
|
|
+ (directive.getId() == null) ?
|
|
|
+ "(null)" : directive.getId().toString();
|
|
|
+ try {
|
|
|
+ // Check for invalid IDs.
|
|
|
+ Long id = directive.getId();
|
|
|
+ if (id == null) {
|
|
|
+ throw new IdNotFoundException("modifyDirective: " +
|
|
|
+ "no ID to modify was supplied.");
|
|
|
+ }
|
|
|
+ if (id <= 0) {
|
|
|
+ throw new IdNotFoundException("modifyDirective " + id +
|
|
|
+ ": invalid non-positive directive ID.");
|
|
|
+ }
|
|
|
+ // Find the entry.
|
|
|
+ PathBasedCacheEntry prevEntry = entriesById.get(id);
|
|
|
+ if (prevEntry == null) {
|
|
|
+ throw new IdNotFoundException("modifyDirective " + id +
|
|
|
+ ": id not found.");
|
|
|
+ }
|
|
|
+ if ((pc != null) &&
|
|
|
+ (!pc.checkPermission(prevEntry.getPool(), FsAction.WRITE))) {
|
|
|
+ throw new AccessControlException("modifyDirective " + id +
|
|
|
+ ": permission denied for initial pool " + prevEntry.getPool());
|
|
|
+ }
|
|
|
+ String path = prevEntry.getPath();
|
|
|
+ if (directive.getPath() != null) {
|
|
|
+ path = directive.getPath().toUri().getPath();
|
|
|
+ if (!DFSUtil.isValidName(path)) {
|
|
|
+ throw new IOException("modifyDirective " + id + ": new path " +
|
|
|
+ path + " is not valid.");
|
|
|
+ }
|
|
|
+ }
|
|
|
+ short replication = (directive.getReplication() != null) ?
|
|
|
+ directive.getReplication() : prevEntry.getReplication();
|
|
|
+ if (replication <= 0) {
|
|
|
+ throw new IOException("modifyDirective: replication " + replication +
|
|
|
+ " is invalid.");
|
|
|
+ }
|
|
|
+ CachePool pool = prevEntry.getPool();
|
|
|
+ if (directive.getPool() != null) {
|
|
|
+ pool = cachePools.get(directive.getPool());
|
|
|
+ if (pool == null) {
|
|
|
+ throw new IdNotFoundException("modifyDirective " + id +
|
|
|
+ ": pool " + directive.getPool() + " not found.");
|
|
|
+ }
|
|
|
+ if (directive.getPool().isEmpty()) {
|
|
|
+ throw new IdNotFoundException("modifyDirective: pool name was " +
|
|
|
+ "empty.");
|
|
|
+ }
|
|
|
+ if ((pc != null) &&
|
|
|
+ (!pc.checkPermission(pool, FsAction.WRITE))) {
|
|
|
+ throw new AccessControlException("modifyDirective " + id +
|
|
|
+ ": permission denied for target pool " + pool);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ removeInternal(prevEntry);
|
|
|
+ PathBasedCacheEntry newEntry =
|
|
|
+ new PathBasedCacheEntry(id, path, replication, pool);
|
|
|
+ addInternal(newEntry);
|
|
|
+ } catch (IOException e) {
|
|
|
+ LOG.warn("modifyDirective " + idString + ": failed.", e);
|
|
|
+ throw e;
|
|
|
+ }
|
|
|
+ LOG.info("modifyDirective " + idString + ": successfully applied " +
|
|
|
+ directive);
|
|
|
}
|
|
|
|
|
|
- public void removeDescriptor(long id, FSPermissionChecker pc)
|
|
|
+ public void removeInternal(PathBasedCacheEntry existing)
|
|
|
throws IOException {
|
|
|
assert namesystem.hasWriteLock();
|
|
|
- // Check for invalid IDs.
|
|
|
- if (id <= 0) {
|
|
|
- LOG.info("removeDescriptor " + id + ": invalid non-positive " +
|
|
|
- "descriptor ID.");
|
|
|
- throw new InvalidIdException(id);
|
|
|
- }
|
|
|
- // Find the entry.
|
|
|
- PathBasedCacheEntry existing = entriesById.get(id);
|
|
|
- if (existing == null) {
|
|
|
- LOG.info("removeDescriptor " + id + ": entry not found.");
|
|
|
- throw new NoSuchIdException(id);
|
|
|
- }
|
|
|
- CachePool pool = cachePools.get(existing.getDescriptor().getPool());
|
|
|
- if (pool == null) {
|
|
|
- LOG.info("removeDescriptor " + id + ": pool not found for directive " +
|
|
|
- existing.getDescriptor());
|
|
|
- throw new UnexpectedRemovePathBasedCacheDescriptorException(id);
|
|
|
- }
|
|
|
- if ((pc != null) && (!pc.checkPermission(pool, FsAction.WRITE))) {
|
|
|
- LOG.info("removeDescriptor " + id + ": write permission denied to " +
|
|
|
- "pool " + pool + " for entry " + existing);
|
|
|
- throw new RemovePermissionDeniedException(id);
|
|
|
- }
|
|
|
-
|
|
|
// Remove the corresponding entry in entriesByPath.
|
|
|
- String path = existing.getDescriptor().getPath().toUri().getPath();
|
|
|
+ String path = existing.getPath();
|
|
|
List<PathBasedCacheEntry> entries = entriesByPath.get(path);
|
|
|
if (entries == null || !entries.remove(existing)) {
|
|
|
- throw new UnexpectedRemovePathBasedCacheDescriptorException(id);
|
|
|
+ throw new IdNotFoundException("removeInternal: failed to locate entry " +
|
|
|
+ existing.getEntryId() + " by path " + existing.getPath());
|
|
|
}
|
|
|
if (entries.size() == 0) {
|
|
|
entriesByPath.remove(path);
|
|
|
}
|
|
|
- entriesById.remove(id);
|
|
|
+ entriesById.remove(existing.getEntryId());
|
|
|
+ }
|
|
|
+
|
|
|
+ public void removeDirective(long id, FSPermissionChecker pc)
|
|
|
+ throws IOException {
|
|
|
+ assert namesystem.hasWriteLock();
|
|
|
+ try {
|
|
|
+ // Check for invalid IDs.
|
|
|
+ if (id <= 0) {
|
|
|
+ throw new IdNotFoundException("removeDirective " + id + ": invalid " +
|
|
|
+ "non-positive directive ID.");
|
|
|
+ }
|
|
|
+ // Find the entry.
|
|
|
+ PathBasedCacheEntry existing = entriesById.get(id);
|
|
|
+ if (existing == null) {
|
|
|
+ throw new IdNotFoundException("removeDirective " + id +
|
|
|
+ ": id not found.");
|
|
|
+ }
|
|
|
+ if ((pc != null) &&
|
|
|
+ (!pc.checkPermission(existing.getPool(), FsAction.WRITE))) {
|
|
|
+ throw new AccessControlException("removeDirective " + id +
|
|
|
+ ": write permission denied on pool " +
|
|
|
+ existing.getPool().getPoolName());
|
|
|
+ }
|
|
|
+ removeInternal(existing);
|
|
|
+ } catch (IOException e) {
|
|
|
+ LOG.warn("removeDirective " + id + " failed.", e);
|
|
|
+ throw e;
|
|
|
+ }
|
|
|
if (monitor != null) {
|
|
|
monitor.kick();
|
|
|
}
|
|
|
- LOG.info("removeDescriptor successful for PathCacheEntry id " + id);
|
|
|
+ LOG.info("removeDirective " + id + ": succeeded.");
|
|
|
}
|
|
|
|
|
|
- public BatchedListEntries<PathBasedCacheDescriptor>
|
|
|
- listPathBasedCacheDescriptors(long prevId, String filterPool,
|
|
|
- String filterPath, FSPermissionChecker pc) throws IOException {
|
|
|
+ public BatchedListEntries<PathBasedCacheDirective>
|
|
|
+ listPathBasedCacheDirectives(long prevId,
|
|
|
+ PathBasedCacheDirective filter,
|
|
|
+ FSPermissionChecker pc) throws IOException {
|
|
|
assert namesystem.hasReadOrWriteLock();
|
|
|
final int NUM_PRE_ALLOCATED_ENTRIES = 16;
|
|
|
- if (filterPath != null) {
|
|
|
+ String filterPath = null;
|
|
|
+ if (filter.getId() != null) {
|
|
|
+ throw new IOException("we currently don't support filtering by ID");
|
|
|
+ }
|
|
|
+ if (filter.getPath() != null) {
|
|
|
+ filterPath = filter.getPath().toUri().getPath();
|
|
|
if (!DFSUtil.isValidName(filterPath)) {
|
|
|
- throw new IOException("invalid path name '" + filterPath + "'");
|
|
|
+ throw new IOException("listPathBasedCacheDirectives: invalid " +
|
|
|
+ "path name '" + filterPath + "'");
|
|
|
}
|
|
|
}
|
|
|
- ArrayList<PathBasedCacheDescriptor> replies =
|
|
|
- new ArrayList<PathBasedCacheDescriptor>(NUM_PRE_ALLOCATED_ENTRIES);
|
|
|
+ if (filter.getReplication() != null) {
|
|
|
+ throw new IOException("we currently don't support filtering " +
|
|
|
+ "by replication");
|
|
|
+ }
|
|
|
+ ArrayList<PathBasedCacheDirective> replies =
|
|
|
+ new ArrayList<PathBasedCacheDirective>(NUM_PRE_ALLOCATED_ENTRIES);
|
|
|
int numReplies = 0;
|
|
|
- SortedMap<Long, PathBasedCacheEntry> tailMap = entriesById.tailMap(prevId + 1);
|
|
|
+ SortedMap<Long, PathBasedCacheEntry> tailMap =
|
|
|
+ entriesById.tailMap(prevId + 1);
|
|
|
for (Entry<Long, PathBasedCacheEntry> cur : tailMap.entrySet()) {
|
|
|
- if (numReplies >= maxListCacheDescriptorsResponses) {
|
|
|
- return new BatchedListEntries<PathBasedCacheDescriptor>(replies, true);
|
|
|
+ if (numReplies >= maxListCacheDirectivesNumResponses) {
|
|
|
+ return new BatchedListEntries<PathBasedCacheDirective>(replies, true);
|
|
|
}
|
|
|
PathBasedCacheEntry curEntry = cur.getValue();
|
|
|
- PathBasedCacheDirective directive = cur.getValue().getDescriptor();
|
|
|
- if (filterPool != null &&
|
|
|
- !directive.getPool().equals(filterPool)) {
|
|
|
+ PathBasedCacheDirective directive = cur.getValue().toDirective();
|
|
|
+ if (filter.getPool() != null &&
|
|
|
+ !directive.getPool().equals(filter.getPool())) {
|
|
|
continue;
|
|
|
}
|
|
|
if (filterPath != null &&
|
|
|
!directive.getPath().toUri().getPath().equals(filterPath)) {
|
|
|
continue;
|
|
|
}
|
|
|
- if (pc.checkPermission(curEntry.getPool(), FsAction.READ)) {
|
|
|
- replies.add(cur.getValue().getDescriptor());
|
|
|
+ if ((pc == null) ||
|
|
|
+ (pc.checkPermission(curEntry.getPool(), FsAction.READ))) {
|
|
|
+ replies.add(cur.getValue().toDirective());
|
|
|
numReplies++;
|
|
|
}
|
|
|
}
|
|
|
- return new BatchedListEntries<PathBasedCacheDescriptor>(replies, false);
|
|
|
+ return new BatchedListEntries<PathBasedCacheDirective>(replies, false);
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -553,7 +649,8 @@ public final class CacheManager {
|
|
|
blockManager.getDatanodeManager().getDatanode(datanodeID);
|
|
|
if (datanode == null || !datanode.isAlive) {
|
|
|
throw new IOException(
|
|
|
- "processCacheReport from dead or unregistered datanode: " + datanode);
|
|
|
+ "processCacheReport from dead or unregistered datanode: " +
|
|
|
+ datanode);
|
|
|
}
|
|
|
processCacheReportImpl(datanode, blockIds);
|
|
|
} finally {
|