|
@@ -49,6 +49,7 @@ 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.CachePoolEntry;
|
|
|
import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
|
|
|
import org.apache.hadoop.hdfs.protocol.DatanodeID;
|
|
|
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
|
|
@@ -99,24 +100,24 @@ public final class CacheManager {
|
|
|
private final BlockManager blockManager;
|
|
|
|
|
|
/**
|
|
|
- * Cache entries, sorted by ID.
|
|
|
+ * Cache directives, sorted by ID.
|
|
|
*
|
|
|
* listCacheDirectives relies on the ordering of elements in this map
|
|
|
* to track what has already been listed by the client.
|
|
|
*/
|
|
|
- private final TreeMap<Long, CacheDirective> entriesById =
|
|
|
+ private final TreeMap<Long, CacheDirective> directivesById =
|
|
|
new TreeMap<Long, CacheDirective>();
|
|
|
|
|
|
/**
|
|
|
- * The entry ID to use for a new entry. Entry IDs always increase, and are
|
|
|
+ * The directive ID to use for a new directive. IDs always increase, and are
|
|
|
* never reused.
|
|
|
*/
|
|
|
- private long nextEntryId;
|
|
|
+ private long nextDirectiveId;
|
|
|
|
|
|
/**
|
|
|
- * Cache entries, sorted by path
|
|
|
+ * Cache directives, sorted by path
|
|
|
*/
|
|
|
- private final TreeMap<String, List<CacheDirective>> entriesByPath =
|
|
|
+ private final TreeMap<String, List<CacheDirective>> directivesByPath =
|
|
|
new TreeMap<String, List<CacheDirective>>();
|
|
|
|
|
|
/**
|
|
@@ -177,7 +178,7 @@ public final class CacheManager {
|
|
|
BlockManager blockManager) {
|
|
|
this.namesystem = namesystem;
|
|
|
this.blockManager = blockManager;
|
|
|
- this.nextEntryId = 1;
|
|
|
+ this.nextDirectiveId = 1;
|
|
|
this.maxListCachePoolsResponses = conf.getInt(
|
|
|
DFS_NAMENODE_LIST_CACHE_POOLS_NUM_RESPONSES,
|
|
|
DFS_NAMENODE_LIST_CACHE_POOLS_NUM_RESPONSES_DEFAULT);
|
|
@@ -239,7 +240,7 @@ public final class CacheManager {
|
|
|
|
|
|
public TreeMap<Long, CacheDirective> getEntriesById() {
|
|
|
assert namesystem.hasReadLock();
|
|
|
- return entriesById;
|
|
|
+ return directivesById;
|
|
|
}
|
|
|
|
|
|
@VisibleForTesting
|
|
@@ -250,10 +251,10 @@ public final class CacheManager {
|
|
|
|
|
|
private long getNextEntryId() throws IOException {
|
|
|
assert namesystem.hasWriteLock();
|
|
|
- if (nextEntryId >= Long.MAX_VALUE - 1) {
|
|
|
+ if (nextDirectiveId >= Long.MAX_VALUE - 1) {
|
|
|
throw new IOException("No more available IDs.");
|
|
|
}
|
|
|
- return nextEntryId++;
|
|
|
+ return nextDirectiveId++;
|
|
|
}
|
|
|
|
|
|
// Helper getter / validation methods
|
|
@@ -301,7 +302,7 @@ public final class CacheManager {
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
- * Get a CacheDirective by ID, validating the ID and that the entry
|
|
|
+ * Get a CacheDirective by ID, validating the ID and that the directive
|
|
|
* exists.
|
|
|
*/
|
|
|
private CacheDirective getById(long id) throws InvalidRequestException {
|
|
@@ -309,13 +310,13 @@ public final class CacheManager {
|
|
|
if (id <= 0) {
|
|
|
throw new InvalidRequestException("Invalid negative ID.");
|
|
|
}
|
|
|
- // Find the entry.
|
|
|
- CacheDirective entry = entriesById.get(id);
|
|
|
- if (entry == null) {
|
|
|
+ // Find the directive.
|
|
|
+ CacheDirective directive = directivesById.get(id);
|
|
|
+ if (directive == null) {
|
|
|
throw new InvalidRequestException("No directive with ID " + id
|
|
|
+ " found.");
|
|
|
}
|
|
|
- return entry;
|
|
|
+ return directive;
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -332,32 +333,34 @@ public final class CacheManager {
|
|
|
|
|
|
// RPC handlers
|
|
|
|
|
|
- private void addInternal(CacheDirective entry) {
|
|
|
- entriesById.put(entry.getEntryId(), entry);
|
|
|
- String path = entry.getPath();
|
|
|
- List<CacheDirective> entryList = entriesByPath.get(path);
|
|
|
- if (entryList == null) {
|
|
|
- entryList = new ArrayList<CacheDirective>(1);
|
|
|
- entriesByPath.put(path, entryList);
|
|
|
+ private void addInternal(CacheDirective directive, CachePool pool) {
|
|
|
+ boolean addedDirective = pool.getDirectiveList().add(directive);
|
|
|
+ assert addedDirective;
|
|
|
+ directivesById.put(directive.getId(), directive);
|
|
|
+ String path = directive.getPath();
|
|
|
+ List<CacheDirective> directives = directivesByPath.get(path);
|
|
|
+ if (directives == null) {
|
|
|
+ directives = new ArrayList<CacheDirective>(1);
|
|
|
+ directivesByPath.put(path, directives);
|
|
|
}
|
|
|
- entryList.add(entry);
|
|
|
+ directives.add(directive);
|
|
|
}
|
|
|
|
|
|
public CacheDirectiveInfo addDirective(
|
|
|
- CacheDirectiveInfo directive, FSPermissionChecker pc)
|
|
|
+ CacheDirectiveInfo info, FSPermissionChecker pc)
|
|
|
throws IOException {
|
|
|
assert namesystem.hasWriteLock();
|
|
|
- CacheDirective entry;
|
|
|
+ CacheDirective directive;
|
|
|
try {
|
|
|
- CachePool pool = getCachePool(validatePoolName(directive));
|
|
|
+ CachePool pool = getCachePool(validatePoolName(info));
|
|
|
checkWritePermission(pc, pool);
|
|
|
- String path = validatePath(directive);
|
|
|
- short replication = validateReplication(directive, (short)1);
|
|
|
+ String path = validatePath(info);
|
|
|
+ short replication = validateReplication(info, (short)1);
|
|
|
long id;
|
|
|
- if (directive.getId() != null) {
|
|
|
- // We are loading an entry from the edit log.
|
|
|
+ if (info.getId() != null) {
|
|
|
+ // We are loading a directive from the edit log.
|
|
|
// Use the ID from the edit log.
|
|
|
- id = directive.getId();
|
|
|
+ id = info.getId();
|
|
|
if (id <= 0) {
|
|
|
throw new InvalidRequestException("can't add an ID " +
|
|
|
"of " + id + ": it is not positive.");
|
|
@@ -366,88 +369,90 @@ public final class CacheManager {
|
|
|
throw new InvalidRequestException("can't add an ID " +
|
|
|
"of " + id + ": it is too big.");
|
|
|
}
|
|
|
- if (nextEntryId <= id) {
|
|
|
- nextEntryId = id + 1;
|
|
|
+ if (nextDirectiveId <= id) {
|
|
|
+ nextDirectiveId = id + 1;
|
|
|
}
|
|
|
} else {
|
|
|
- // Add a new entry with the next available ID.
|
|
|
+ // Add a new directive with the next available ID.
|
|
|
id = getNextEntryId();
|
|
|
}
|
|
|
- entry = new CacheDirective(id, path, replication, pool);
|
|
|
- addInternal(entry);
|
|
|
+ directive = new CacheDirective(id, path, replication);
|
|
|
+ addInternal(directive, pool);
|
|
|
} catch (IOException e) {
|
|
|
- LOG.warn("addDirective of " + directive + " failed: ", e);
|
|
|
+ LOG.warn("addDirective of " + info + " failed: ", e);
|
|
|
throw e;
|
|
|
}
|
|
|
- LOG.info("addDirective of " + directive + " successful.");
|
|
|
+ LOG.info("addDirective of " + info + " successful.");
|
|
|
if (monitor != null) {
|
|
|
monitor.kick();
|
|
|
}
|
|
|
- return entry.toDirective();
|
|
|
+ return directive.toInfo();
|
|
|
}
|
|
|
|
|
|
- public void modifyDirective(CacheDirectiveInfo directive,
|
|
|
+ public void modifyDirective(CacheDirectiveInfo info,
|
|
|
FSPermissionChecker pc) throws IOException {
|
|
|
assert namesystem.hasWriteLock();
|
|
|
String idString =
|
|
|
- (directive.getId() == null) ?
|
|
|
- "(null)" : directive.getId().toString();
|
|
|
+ (info.getId() == null) ?
|
|
|
+ "(null)" : info.getId().toString();
|
|
|
try {
|
|
|
// Check for invalid IDs.
|
|
|
- Long id = directive.getId();
|
|
|
+ Long id = info.getId();
|
|
|
if (id == null) {
|
|
|
throw new InvalidRequestException("Must supply an ID.");
|
|
|
}
|
|
|
CacheDirective prevEntry = getById(id);
|
|
|
checkWritePermission(pc, prevEntry.getPool());
|
|
|
String path = prevEntry.getPath();
|
|
|
- if (directive.getPath() != null) {
|
|
|
- path = validatePath(directive);
|
|
|
+ if (info.getPath() != null) {
|
|
|
+ path = validatePath(info);
|
|
|
}
|
|
|
short replication = prevEntry.getReplication();
|
|
|
- if (directive.getReplication() != null) {
|
|
|
- replication = validateReplication(directive, replication);
|
|
|
+ if (info.getReplication() != null) {
|
|
|
+ replication = validateReplication(info, replication);
|
|
|
}
|
|
|
CachePool pool = prevEntry.getPool();
|
|
|
- if (directive.getPool() != null) {
|
|
|
- pool = getCachePool(validatePoolName(directive));
|
|
|
+ if (info.getPool() != null) {
|
|
|
+ pool = getCachePool(validatePoolName(info));
|
|
|
checkWritePermission(pc, pool);
|
|
|
}
|
|
|
removeInternal(prevEntry);
|
|
|
CacheDirective newEntry =
|
|
|
- new CacheDirective(id, path, replication, pool);
|
|
|
- addInternal(newEntry);
|
|
|
+ new CacheDirective(id, path, replication);
|
|
|
+ addInternal(newEntry, pool);
|
|
|
} catch (IOException e) {
|
|
|
LOG.warn("modifyDirective of " + idString + " failed: ", e);
|
|
|
throw e;
|
|
|
}
|
|
|
LOG.info("modifyDirective of " + idString + " successfully applied " +
|
|
|
- directive + ".");
|
|
|
+ info+ ".");
|
|
|
}
|
|
|
|
|
|
- public void removeInternal(CacheDirective existing)
|
|
|
+ public void removeInternal(CacheDirective directive)
|
|
|
throws InvalidRequestException {
|
|
|
assert namesystem.hasWriteLock();
|
|
|
- // Remove the corresponding entry in entriesByPath.
|
|
|
- String path = existing.getPath();
|
|
|
- List<CacheDirective> entries = entriesByPath.get(path);
|
|
|
- if (entries == null || !entries.remove(existing)) {
|
|
|
+ // Remove the corresponding entry in directivesByPath.
|
|
|
+ String path = directive.getPath();
|
|
|
+ List<CacheDirective> directives = directivesByPath.get(path);
|
|
|
+ if (directives == null || !directives.remove(directive)) {
|
|
|
throw new InvalidRequestException("Failed to locate entry " +
|
|
|
- existing.getEntryId() + " by path " + existing.getPath());
|
|
|
+ directive.getId() + " by path " + directive.getPath());
|
|
|
}
|
|
|
- if (entries.size() == 0) {
|
|
|
- entriesByPath.remove(path);
|
|
|
+ if (directives.size() == 0) {
|
|
|
+ directivesByPath.remove(path);
|
|
|
}
|
|
|
- entriesById.remove(existing.getEntryId());
|
|
|
+ directivesById.remove(directive.getId());
|
|
|
+ directive.getPool().getDirectiveList().remove(directive);
|
|
|
+ assert directive.getPool() == null;
|
|
|
}
|
|
|
|
|
|
public void removeDirective(long id, FSPermissionChecker pc)
|
|
|
throws IOException {
|
|
|
assert namesystem.hasWriteLock();
|
|
|
try {
|
|
|
- CacheDirective existing = getById(id);
|
|
|
- checkWritePermission(pc, existing.getPool());
|
|
|
- removeInternal(existing);
|
|
|
+ CacheDirective directive = getById(id);
|
|
|
+ checkWritePermission(pc, directive.getPool());
|
|
|
+ removeInternal(directive);
|
|
|
} catch (IOException e) {
|
|
|
LOG.warn("removeDirective of " + id + " failed: ", e);
|
|
|
throw e;
|
|
@@ -478,13 +483,13 @@ public final class CacheManager {
|
|
|
new ArrayList<CacheDirectiveEntry>(NUM_PRE_ALLOCATED_ENTRIES);
|
|
|
int numReplies = 0;
|
|
|
SortedMap<Long, CacheDirective> tailMap =
|
|
|
- entriesById.tailMap(prevId + 1);
|
|
|
+ directivesById.tailMap(prevId + 1);
|
|
|
for (Entry<Long, CacheDirective> cur : tailMap.entrySet()) {
|
|
|
if (numReplies >= maxListCacheDirectivesNumResponses) {
|
|
|
return new BatchedListEntries<CacheDirectiveEntry>(replies, true);
|
|
|
}
|
|
|
- CacheDirective curEntry = cur.getValue();
|
|
|
- CacheDirectiveInfo info = cur.getValue().toDirective();
|
|
|
+ CacheDirective curDirective = cur.getValue();
|
|
|
+ CacheDirectiveInfo info = cur.getValue().toInfo();
|
|
|
if (filter.getPool() != null &&
|
|
|
!info.getPool().equals(filter.getPool())) {
|
|
|
continue;
|
|
@@ -496,7 +501,7 @@ public final class CacheManager {
|
|
|
boolean hasPermission = true;
|
|
|
if (pc != null) {
|
|
|
try {
|
|
|
- pc.checkPermission(curEntry.getPool(), FsAction.READ);
|
|
|
+ pc.checkPermission(curDirective.getPool(), FsAction.READ);
|
|
|
} catch (AccessControlException e) {
|
|
|
hasPermission = false;
|
|
|
}
|
|
@@ -530,7 +535,7 @@ public final class CacheManager {
|
|
|
pool = CachePool.createFromInfoAndDefaults(info);
|
|
|
cachePools.put(pool.getPoolName(), pool);
|
|
|
LOG.info("Created new cache pool " + pool);
|
|
|
- return pool.getInfo(null);
|
|
|
+ return pool.getInfo(true);
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -599,39 +604,34 @@ public final class CacheManager {
|
|
|
throw new InvalidRequestException(
|
|
|
"Cannot remove non-existent cache pool " + poolName);
|
|
|
}
|
|
|
-
|
|
|
- // Remove entries using this pool
|
|
|
- // TODO: could optimize this somewhat to avoid the need to iterate
|
|
|
- // over all entries in entriesById
|
|
|
- Iterator<Entry<Long, CacheDirective>> iter =
|
|
|
- entriesById.entrySet().iterator();
|
|
|
+ // Remove all directives in this pool.
|
|
|
+ Iterator<CacheDirective> iter = pool.getDirectiveList().iterator();
|
|
|
while (iter.hasNext()) {
|
|
|
- Entry<Long, CacheDirective> entry = iter.next();
|
|
|
- if (entry.getValue().getPool() == pool) {
|
|
|
- entriesByPath.remove(entry.getValue().getPath());
|
|
|
- iter.remove();
|
|
|
- }
|
|
|
+ CacheDirective directive = iter.next();
|
|
|
+ directivesByPath.remove(directive.getPath());
|
|
|
+ directivesById.remove(directive.getId());
|
|
|
+ iter.remove();
|
|
|
}
|
|
|
if (monitor != null) {
|
|
|
monitor.kick();
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- public BatchedListEntries<CachePoolInfo>
|
|
|
+ public BatchedListEntries<CachePoolEntry>
|
|
|
listCachePools(FSPermissionChecker pc, String prevKey) {
|
|
|
assert namesystem.hasReadLock();
|
|
|
final int NUM_PRE_ALLOCATED_ENTRIES = 16;
|
|
|
- ArrayList<CachePoolInfo> results =
|
|
|
- new ArrayList<CachePoolInfo>(NUM_PRE_ALLOCATED_ENTRIES);
|
|
|
+ ArrayList<CachePoolEntry> results =
|
|
|
+ new ArrayList<CachePoolEntry>(NUM_PRE_ALLOCATED_ENTRIES);
|
|
|
SortedMap<String, CachePool> tailMap = cachePools.tailMap(prevKey, false);
|
|
|
int numListed = 0;
|
|
|
for (Entry<String, CachePool> cur : tailMap.entrySet()) {
|
|
|
if (numListed++ >= maxListCachePoolsResponses) {
|
|
|
- return new BatchedListEntries<CachePoolInfo>(results, true);
|
|
|
+ return new BatchedListEntries<CachePoolEntry>(results, true);
|
|
|
}
|
|
|
- results.add(cur.getValue().getInfo(pc));
|
|
|
+ results.add(cur.getValue().getEntry(pc));
|
|
|
}
|
|
|
- return new BatchedListEntries<CachePoolInfo>(results, false);
|
|
|
+ return new BatchedListEntries<CachePoolEntry>(results, false);
|
|
|
}
|
|
|
|
|
|
public void setCachedLocations(LocatedBlock block) {
|
|
@@ -693,13 +693,6 @@ public final class CacheManager {
|
|
|
for (Iterator<Long> iter = blockIds.iterator(); iter.hasNext(); ) {
|
|
|
Block block = new Block(iter.next());
|
|
|
BlockInfo blockInfo = blockManager.getStoredBlock(block);
|
|
|
- if (blockInfo.getGenerationStamp() < block.getGenerationStamp()) {
|
|
|
- // The NameNode will eventually remove or update the out-of-date block.
|
|
|
- // Until then, we pretend that it isn't cached.
|
|
|
- LOG.warn("Genstamp in cache report disagrees with our genstamp for " +
|
|
|
- block + ": expected genstamp " + blockInfo.getGenerationStamp());
|
|
|
- continue;
|
|
|
- }
|
|
|
if (!blockInfo.isComplete()) {
|
|
|
LOG.warn("Ignoring block id " + block.getBlockId() + ", because " +
|
|
|
"it is in not complete yet. It is in state " +
|
|
@@ -743,9 +736,9 @@ public final class CacheManager {
|
|
|
*/
|
|
|
public void saveState(DataOutput out, String sdPath)
|
|
|
throws IOException {
|
|
|
- out.writeLong(nextEntryId);
|
|
|
+ out.writeLong(nextDirectiveId);
|
|
|
savePools(out, sdPath);
|
|
|
- saveEntries(out, sdPath);
|
|
|
+ saveDirectives(out, sdPath);
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -755,10 +748,10 @@ public final class CacheManager {
|
|
|
* @throws IOException
|
|
|
*/
|
|
|
public void loadState(DataInput in) throws IOException {
|
|
|
- nextEntryId = in.readLong();
|
|
|
- // pools need to be loaded first since entries point to their parent pool
|
|
|
+ nextDirectiveId = in.readLong();
|
|
|
+ // pools need to be loaded first since directives point to their parent pool
|
|
|
loadPools(in);
|
|
|
- loadEntries(in);
|
|
|
+ loadDirectives(in);
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -773,7 +766,7 @@ public final class CacheManager {
|
|
|
Counter counter = prog.getCounter(Phase.SAVING_CHECKPOINT, step);
|
|
|
out.writeInt(cachePools.size());
|
|
|
for (CachePool pool: cachePools.values()) {
|
|
|
- pool.getInfo(null).writeTo(out);
|
|
|
+ pool.getInfo(true).writeTo(out);
|
|
|
counter.increment();
|
|
|
}
|
|
|
prog.endStep(Phase.SAVING_CHECKPOINT, step);
|
|
@@ -782,19 +775,19 @@ public final class CacheManager {
|
|
|
/*
|
|
|
* Save cache entries to fsimage
|
|
|
*/
|
|
|
- private void saveEntries(DataOutput out, String sdPath)
|
|
|
+ private void saveDirectives(DataOutput out, String sdPath)
|
|
|
throws IOException {
|
|
|
StartupProgress prog = NameNode.getStartupProgress();
|
|
|
Step step = new Step(StepType.CACHE_ENTRIES, sdPath);
|
|
|
prog.beginStep(Phase.SAVING_CHECKPOINT, step);
|
|
|
- prog.setTotal(Phase.SAVING_CHECKPOINT, step, entriesById.size());
|
|
|
+ prog.setTotal(Phase.SAVING_CHECKPOINT, step, directivesById.size());
|
|
|
Counter counter = prog.getCounter(Phase.SAVING_CHECKPOINT, step);
|
|
|
- out.writeInt(entriesById.size());
|
|
|
- for (CacheDirective entry: entriesById.values()) {
|
|
|
- out.writeLong(entry.getEntryId());
|
|
|
- Text.writeString(out, entry.getPath());
|
|
|
- out.writeShort(entry.getReplication());
|
|
|
- Text.writeString(out, entry.getPool().getPoolName());
|
|
|
+ out.writeInt(directivesById.size());
|
|
|
+ for (CacheDirective directive : directivesById.values()) {
|
|
|
+ out.writeLong(directive.getId());
|
|
|
+ Text.writeString(out, directive.getPath());
|
|
|
+ out.writeShort(directive.getReplication());
|
|
|
+ Text.writeString(out, directive.getPool().getPoolName());
|
|
|
counter.increment();
|
|
|
}
|
|
|
prog.endStep(Phase.SAVING_CHECKPOINT, step);
|
|
@@ -819,38 +812,41 @@ public final class CacheManager {
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
- * Load cache entries from the fsimage
|
|
|
+ * Load cache directives from the fsimage
|
|
|
*/
|
|
|
- private void loadEntries(DataInput in) throws IOException {
|
|
|
+ private void loadDirectives(DataInput in) throws IOException {
|
|
|
StartupProgress prog = NameNode.getStartupProgress();
|
|
|
Step step = new Step(StepType.CACHE_ENTRIES);
|
|
|
prog.beginStep(Phase.LOADING_FSIMAGE, step);
|
|
|
- int numberOfEntries = in.readInt();
|
|
|
- prog.setTotal(Phase.LOADING_FSIMAGE, step, numberOfEntries);
|
|
|
+ int numDirectives = in.readInt();
|
|
|
+ prog.setTotal(Phase.LOADING_FSIMAGE, step, numDirectives);
|
|
|
Counter counter = prog.getCounter(Phase.LOADING_FSIMAGE, step);
|
|
|
- for (int i = 0; i < numberOfEntries; i++) {
|
|
|
- long entryId = in.readLong();
|
|
|
+ for (int i = 0; i < numDirectives; i++) {
|
|
|
+ long directiveId = in.readLong();
|
|
|
String path = Text.readString(in);
|
|
|
short replication = in.readShort();
|
|
|
String poolName = Text.readString(in);
|
|
|
// Get pool reference by looking it up in the map
|
|
|
CachePool pool = cachePools.get(poolName);
|
|
|
if (pool == null) {
|
|
|
- throw new IOException("Entry refers to pool " + poolName +
|
|
|
+ throw new IOException("Directive refers to pool " + poolName +
|
|
|
", which does not exist.");
|
|
|
}
|
|
|
- CacheDirective entry =
|
|
|
- new CacheDirective(entryId, path, replication, pool);
|
|
|
- if (entriesById.put(entry.getEntryId(), entry) != null) {
|
|
|
- throw new IOException("An entry with ID " + entry.getEntryId() +
|
|
|
+ CacheDirective directive =
|
|
|
+ new CacheDirective(directiveId, path, replication);
|
|
|
+ boolean addedDirective = pool.getDirectiveList().add(directive);
|
|
|
+ assert addedDirective;
|
|
|
+ if (directivesById.put(directive.getId(), directive) != null) {
|
|
|
+ throw new IOException("A directive with ID " + directive.getId() +
|
|
|
" already exists");
|
|
|
}
|
|
|
- List<CacheDirective> entries = entriesByPath.get(entry.getPath());
|
|
|
- if (entries == null) {
|
|
|
- entries = new LinkedList<CacheDirective>();
|
|
|
- entriesByPath.put(entry.getPath(), entries);
|
|
|
+ List<CacheDirective> directives =
|
|
|
+ directivesByPath.get(directive.getPath());
|
|
|
+ if (directives == null) {
|
|
|
+ directives = new LinkedList<CacheDirective>();
|
|
|
+ directivesByPath.put(directive.getPath(), directives);
|
|
|
}
|
|
|
- entries.add(entry);
|
|
|
+ directives.add(directive);
|
|
|
counter.increment();
|
|
|
}
|
|
|
prog.endStep(Phase.LOADING_FSIMAGE, step);
|