|
@@ -50,8 +50,10 @@ import org.apache.hadoop.conf.Configuration;
|
|
import org.apache.hadoop.fs.BatchedRemoteIterator.BatchedListEntries;
|
|
import org.apache.hadoop.fs.BatchedRemoteIterator.BatchedListEntries;
|
|
import org.apache.hadoop.fs.CacheFlag;
|
|
import org.apache.hadoop.fs.CacheFlag;
|
|
import org.apache.hadoop.fs.InvalidRequestException;
|
|
import org.apache.hadoop.fs.InvalidRequestException;
|
|
|
|
+import org.apache.hadoop.fs.Path;
|
|
import org.apache.hadoop.fs.UnresolvedLinkException;
|
|
import org.apache.hadoop.fs.UnresolvedLinkException;
|
|
import org.apache.hadoop.fs.permission.FsAction;
|
|
import org.apache.hadoop.fs.permission.FsAction;
|
|
|
|
+import org.apache.hadoop.fs.permission.FsPermission;
|
|
import org.apache.hadoop.hdfs.DFSUtil;
|
|
import org.apache.hadoop.hdfs.DFSUtil;
|
|
import org.apache.hadoop.hdfs.protocol.CacheDirective;
|
|
import org.apache.hadoop.hdfs.protocol.CacheDirective;
|
|
import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry;
|
|
import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry;
|
|
@@ -62,11 +64,15 @@ import org.apache.hadoop.hdfs.protocol.CachePoolEntry;
|
|
import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
|
|
import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
|
|
import org.apache.hadoop.hdfs.protocol.DatanodeID;
|
|
import org.apache.hadoop.hdfs.protocol.DatanodeID;
|
|
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
|
|
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
|
|
|
|
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CacheDirectiveInfoExpirationProto;
|
|
|
|
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CacheDirectiveInfoProto;
|
|
|
|
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CachePoolInfoProto;
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.CacheReplicationMonitor;
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.CacheReplicationMonitor;
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
|
|
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;
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor.CachedBlocksList.Type;
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor.CachedBlocksList.Type;
|
|
|
|
+import org.apache.hadoop.hdfs.server.namenode.FsImageProto.CacheManagerSection;
|
|
import org.apache.hadoop.hdfs.server.namenode.metrics.NameNodeMetrics;
|
|
import org.apache.hadoop.hdfs.server.namenode.metrics.NameNodeMetrics;
|
|
import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
|
|
import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
|
|
import org.apache.hadoop.hdfs.server.namenode.startupprogress.Phase;
|
|
import org.apache.hadoop.hdfs.server.namenode.startupprogress.Phase;
|
|
@@ -81,6 +87,7 @@ import org.apache.hadoop.util.LightWeightGSet;
|
|
import org.apache.hadoop.util.Time;
|
|
import org.apache.hadoop.util.Time;
|
|
|
|
|
|
import com.google.common.annotations.VisibleForTesting;
|
|
import com.google.common.annotations.VisibleForTesting;
|
|
|
|
+import com.google.common.collect.Lists;
|
|
|
|
|
|
/**
|
|
/**
|
|
* The Cache Manager handles caching on DataNodes.
|
|
* The Cache Manager handles caching on DataNodes.
|
|
@@ -167,6 +174,19 @@ public final class CacheManager {
|
|
*/
|
|
*/
|
|
private CacheReplicationMonitor monitor;
|
|
private CacheReplicationMonitor monitor;
|
|
|
|
|
|
|
|
+ public static final class PersistState {
|
|
|
|
+ public final CacheManagerSection section;
|
|
|
|
+ public final List<CachePoolInfoProto> pools;
|
|
|
|
+ public final List<CacheDirectiveInfoProto> directives;
|
|
|
|
+
|
|
|
|
+ public PersistState(CacheManagerSection section,
|
|
|
|
+ List<CachePoolInfoProto> pools, List<CacheDirectiveInfoProto> directives) {
|
|
|
|
+ this.section = section;
|
|
|
|
+ this.pools = pools;
|
|
|
|
+ this.directives = directives;
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
CacheManager(FSNamesystem namesystem, Configuration conf,
|
|
CacheManager(FSNamesystem namesystem, Configuration conf,
|
|
BlockManager blockManager) {
|
|
BlockManager blockManager) {
|
|
this.namesystem = namesystem;
|
|
this.namesystem = namesystem;
|
|
@@ -933,6 +953,57 @@ public final class CacheManager {
|
|
serializerCompat.save(out, sdPath);
|
|
serializerCompat.save(out, sdPath);
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ public PersistState saveState() throws IOException {
|
|
|
|
+ ArrayList<CachePoolInfoProto> pools = Lists
|
|
|
|
+ .newArrayListWithCapacity(cachePools.size());
|
|
|
|
+ ArrayList<CacheDirectiveInfoProto> directives = Lists
|
|
|
|
+ .newArrayListWithCapacity(directivesById.size());
|
|
|
|
+
|
|
|
|
+ for (CachePool pool : cachePools.values()) {
|
|
|
|
+ CachePoolInfo p = pool.getInfo(true);
|
|
|
|
+ CachePoolInfoProto.Builder b = CachePoolInfoProto.newBuilder();
|
|
|
|
+
|
|
|
|
+ if (p.getOwnerName() != null)
|
|
|
|
+ b.setOwnerName(p.getOwnerName());
|
|
|
|
+
|
|
|
|
+ if (p.getGroupName() != null)
|
|
|
|
+ b.setGroupName(p.getGroupName());
|
|
|
|
+
|
|
|
|
+ if (p.getMode() != null)
|
|
|
|
+ b.setMode(p.getMode().toShort());
|
|
|
|
+
|
|
|
|
+ if (p.getLimit() != null)
|
|
|
|
+ b.setLimit(p.getLimit());
|
|
|
|
+
|
|
|
|
+ pools.add(b.build());
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ for (CacheDirective directive : directivesById.values()) {
|
|
|
|
+ CacheDirectiveInfo info = directive.toInfo();
|
|
|
|
+ CacheDirectiveInfoProto.Builder b = CacheDirectiveInfoProto.newBuilder();
|
|
|
|
+
|
|
|
|
+ if (info.getPath() != null)
|
|
|
|
+ b.setPath(info.getPath().toUri().getPath());
|
|
|
|
+
|
|
|
|
+ if (info.getReplication() != null)
|
|
|
|
+ b.setReplication(info.getReplication());
|
|
|
|
+
|
|
|
|
+ if (info.getPool() != null)
|
|
|
|
+ b.setPool(info.getPool());
|
|
|
|
+
|
|
|
|
+ if (info.getExpiration() != null)
|
|
|
|
+ b.setExpiration(CacheDirectiveInfoExpirationProto.newBuilder()
|
|
|
|
+ .setMillis(info.getExpiration().getMillis()));
|
|
|
|
+
|
|
|
|
+ directives.add(b.build());
|
|
|
|
+ }
|
|
|
|
+ CacheManagerSection s = CacheManagerSection.newBuilder()
|
|
|
|
+ .setNextDirectiveId(nextDirectiveId).setNumPools(pools.size())
|
|
|
|
+ .setNumDirectives(directives.size()).build();
|
|
|
|
+
|
|
|
|
+ return new PersistState(s, pools, directives);
|
|
|
|
+ }
|
|
|
|
+
|
|
/**
|
|
/**
|
|
* Reloads CacheManager state from the passed DataInput. Used during namenode
|
|
* Reloads CacheManager state from the passed DataInput. Used during namenode
|
|
* startup to restore CacheManager state from an FSImage.
|
|
* startup to restore CacheManager state from an FSImage.
|
|
@@ -943,6 +1014,56 @@ public final class CacheManager {
|
|
serializerCompat.load(in);
|
|
serializerCompat.load(in);
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ public void loadState(PersistState s) throws IOException {
|
|
|
|
+ nextDirectiveId = s.section.getNextDirectiveId();
|
|
|
|
+ for (CachePoolInfoProto p : s.pools) {
|
|
|
|
+ CachePoolInfo info = new CachePoolInfo(p.getPoolName());
|
|
|
|
+ if (p.hasOwnerName())
|
|
|
|
+ info.setOwnerName(p.getOwnerName());
|
|
|
|
+
|
|
|
|
+ if (p.hasGroupName())
|
|
|
|
+ info.setGroupName(p.getGroupName());
|
|
|
|
+
|
|
|
|
+ if (p.hasMode())
|
|
|
|
+ info.setMode(new FsPermission((short) p.getMode()));
|
|
|
|
+
|
|
|
|
+ if (p.hasLimit())
|
|
|
|
+ info.setLimit(p.getLimit());
|
|
|
|
+
|
|
|
|
+ addCachePool(info);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ for (CacheDirectiveInfoProto p : s.directives) {
|
|
|
|
+ // Get pool reference by looking it up in the map
|
|
|
|
+ final String poolName = p.getPool();
|
|
|
|
+ CacheDirective directive = new CacheDirective(p.getId(), new Path(
|
|
|
|
+ p.getPath()).toUri().getPath(), (short) p.getReplication(), p
|
|
|
|
+ .getExpiration().getMillis());
|
|
|
|
+ addCacheDirective(poolName, directive);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ private void addCacheDirective(final String poolName,
|
|
|
|
+ final CacheDirective directive) throws IOException {
|
|
|
|
+ CachePool pool = cachePools.get(poolName);
|
|
|
|
+ if (pool == null) {
|
|
|
|
+ throw new IOException("Directive refers to pool " + poolName
|
|
|
|
+ + ", which does not exist.");
|
|
|
|
+ }
|
|
|
|
+ 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> directives = directivesByPath.get(directive.getPath());
|
|
|
|
+ if (directives == null) {
|
|
|
|
+ directives = new LinkedList<CacheDirective>();
|
|
|
|
+ directivesByPath.put(directive.getPath(), directives);
|
|
|
|
+ }
|
|
|
|
+ directives.add(directive);
|
|
|
|
+ }
|
|
|
|
+
|
|
private final class SerializerCompat {
|
|
private final class SerializerCompat {
|
|
private void save(DataOutputStream out, String sdPath) throws IOException {
|
|
private void save(DataOutputStream out, String sdPath) throws IOException {
|
|
out.writeLong(nextDirectiveId);
|
|
out.writeLong(nextDirectiveId);
|
|
@@ -1025,27 +1146,10 @@ public final class CacheManager {
|
|
CacheDirectiveInfo info = FSImageSerialization.readCacheDirectiveInfo(in);
|
|
CacheDirectiveInfo info = FSImageSerialization.readCacheDirectiveInfo(in);
|
|
// Get pool reference by looking it up in the map
|
|
// Get pool reference by looking it up in the map
|
|
final String poolName = info.getPool();
|
|
final String poolName = info.getPool();
|
|
- CachePool pool = cachePools.get(poolName);
|
|
|
|
- if (pool == null) {
|
|
|
|
- throw new IOException("Directive refers to pool " + poolName +
|
|
|
|
- ", which does not exist.");
|
|
|
|
- }
|
|
|
|
CacheDirective directive =
|
|
CacheDirective directive =
|
|
new CacheDirective(info.getId(), info.getPath().toUri().getPath(),
|
|
new CacheDirective(info.getId(), info.getPath().toUri().getPath(),
|
|
info.getReplication(), info.getExpiration().getAbsoluteMillis());
|
|
info.getReplication(), info.getExpiration().getAbsoluteMillis());
|
|
- 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> directives =
|
|
|
|
- directivesByPath.get(directive.getPath());
|
|
|
|
- if (directives == null) {
|
|
|
|
- directives = new LinkedList<CacheDirective>();
|
|
|
|
- directivesByPath.put(directive.getPath(), directives);
|
|
|
|
- }
|
|
|
|
- directives.add(directive);
|
|
|
|
|
|
+ addCacheDirective(poolName, directive);
|
|
counter.increment();
|
|
counter.increment();
|
|
}
|
|
}
|
|
prog.endStep(Phase.LOADING_FSIMAGE, step);
|
|
prog.endStep(Phase.LOADING_FSIMAGE, step);
|