|
@@ -18,6 +18,7 @@
|
|
package org.apache.hadoop.hdfs.tools.offlineImageViewer;
|
|
package org.apache.hadoop.hdfs.tools.offlineImageViewer;
|
|
|
|
|
|
import com.google.common.base.Preconditions;
|
|
import com.google.common.base.Preconditions;
|
|
|
|
+import com.google.common.collect.ImmutableList;
|
|
import com.google.common.collect.Lists;
|
|
import com.google.common.collect.Lists;
|
|
import org.apache.commons.io.FileUtils;
|
|
import org.apache.commons.io.FileUtils;
|
|
import org.apache.hadoop.conf.Configuration;
|
|
import org.apache.hadoop.conf.Configuration;
|
|
@@ -195,13 +196,17 @@ abstract class PBImageTextWriter implements Closeable {
|
|
dirMap.put(p.getId(), dir);
|
|
dirMap.put(p.getId(), dir);
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ @Override
|
|
public String getParentPath(long inode) throws IOException {
|
|
public String getParentPath(long inode) throws IOException {
|
|
if (inode == INodeId.ROOT_INODE_ID) {
|
|
if (inode == INodeId.ROOT_INODE_ID) {
|
|
return "";
|
|
return "";
|
|
}
|
|
}
|
|
Dir parent = dirChildMap.get(inode);
|
|
Dir parent = dirChildMap.get(inode);
|
|
- Preconditions.checkState(parent != null,
|
|
|
|
- "Can not find parent directory for INode: %s", inode);
|
|
|
|
|
|
+ if (parent == null) {
|
|
|
|
+ // The inode is an INodeReference, which is generated from snapshot.
|
|
|
|
+ // For delimited oiv tool, no need to print out metadata in snapshots.
|
|
|
|
+ PBImageTextWriter.ignoreSnapshotName(inode);
|
|
|
|
+ }
|
|
return parent.getPath();
|
|
return parent.getPath();
|
|
}
|
|
}
|
|
|
|
|
|
@@ -353,16 +358,22 @@ abstract class PBImageTextWriter implements Closeable {
|
|
return "/";
|
|
return "/";
|
|
}
|
|
}
|
|
byte[] bytes = dirChildMap.get(toBytes(inode));
|
|
byte[] bytes = dirChildMap.get(toBytes(inode));
|
|
- Preconditions.checkState(bytes != null && bytes.length == 8,
|
|
|
|
- "Can not find parent directory for inode %s, "
|
|
|
|
- + "fsimage might be corrupted", inode);
|
|
|
|
|
|
+ if (bytes == null) {
|
|
|
|
+ // The inode is an INodeReference, which is generated from snapshot.
|
|
|
|
+ // For delimited oiv tool, no need to print out metadata in snapshots.
|
|
|
|
+ PBImageTextWriter.ignoreSnapshotName(inode);
|
|
|
|
+ }
|
|
|
|
+ if (bytes.length != 8) {
|
|
|
|
+ throw new IOException(
|
|
|
|
+ "bytes array length error. Actual length is " + bytes.length);
|
|
|
|
+ }
|
|
long parent = toLong(bytes);
|
|
long parent = toLong(bytes);
|
|
if (!dirPathCache.containsKey(parent)) {
|
|
if (!dirPathCache.containsKey(parent)) {
|
|
bytes = dirMap.get(toBytes(parent));
|
|
bytes = dirMap.get(toBytes(parent));
|
|
- if (parent != INodeId.ROOT_INODE_ID) {
|
|
|
|
- Preconditions.checkState(bytes != null,
|
|
|
|
- "Can not find parent directory for inode %s, "
|
|
|
|
- + ", the fsimage might be corrupted.", parent);
|
|
|
|
|
|
+ if (parent != INodeId.ROOT_INODE_ID && bytes == null) {
|
|
|
|
+ // The parent is an INodeReference, which is generated from snapshot.
|
|
|
|
+ // For delimited oiv tool, no need to print out metadata in snapshots.
|
|
|
|
+ PBImageTextWriter.ignoreSnapshotName(parent);
|
|
}
|
|
}
|
|
String parentName = toString(bytes);
|
|
String parentName = toString(bytes);
|
|
String parentPath =
|
|
String parentPath =
|
|
@@ -401,6 +412,7 @@ abstract class PBImageTextWriter implements Closeable {
|
|
|
|
|
|
@Override
|
|
@Override
|
|
public void close() throws IOException {
|
|
public void close() throws IOException {
|
|
|
|
+ out.flush();
|
|
IOUtils.cleanup(null, metadataMap);
|
|
IOUtils.cleanup(null, metadataMap);
|
|
}
|
|
}
|
|
|
|
|
|
@@ -411,6 +423,11 @@ abstract class PBImageTextWriter implements Closeable {
|
|
*/
|
|
*/
|
|
abstract protected String getEntry(String parent, INode inode);
|
|
abstract protected String getEntry(String parent, INode inode);
|
|
|
|
|
|
|
|
+ /**
|
|
|
|
+ * Get text output for the header line.
|
|
|
|
+ */
|
|
|
|
+ abstract protected String getHeader();
|
|
|
|
+
|
|
public void visit(RandomAccessFile file) throws IOException {
|
|
public void visit(RandomAccessFile file) throws IOException {
|
|
Configuration conf = new Configuration();
|
|
Configuration conf = new Configuration();
|
|
if (!FSImageUtil.checkFileFormat(file)) {
|
|
if (!FSImageUtil.checkFileFormat(file)) {
|
|
@@ -442,6 +459,7 @@ abstract class PBImageTextWriter implements Closeable {
|
|
}
|
|
}
|
|
});
|
|
});
|
|
|
|
|
|
|
|
+ ImmutableList<Long> refIdList = null;
|
|
for (FileSummary.Section section : sections) {
|
|
for (FileSummary.Section section : sections) {
|
|
fin.getChannel().position(section.getOffset());
|
|
fin.getChannel().position(section.getOffset());
|
|
is = FSImageUtil.wrapInputStreamForCompression(conf,
|
|
is = FSImageUtil.wrapInputStreamForCompression(conf,
|
|
@@ -449,15 +467,22 @@ abstract class PBImageTextWriter implements Closeable {
|
|
fin, section.getLength())));
|
|
fin, section.getLength())));
|
|
switch (SectionName.fromString(section.getName())) {
|
|
switch (SectionName.fromString(section.getName())) {
|
|
case STRING_TABLE:
|
|
case STRING_TABLE:
|
|
|
|
+ LOG.info("Loading string table");
|
|
stringTable = FSImageLoader.loadStringTable(is);
|
|
stringTable = FSImageLoader.loadStringTable(is);
|
|
break;
|
|
break;
|
|
|
|
+ case INODE_REFERENCE:
|
|
|
|
+ // Load INodeReference so that all INodes can be processed.
|
|
|
|
+ // Snapshots are not handled and will just be ignored for now.
|
|
|
|
+ LOG.info("Loading inode references");
|
|
|
|
+ refIdList = FSImageLoader.loadINodeReferenceSection(is);
|
|
|
|
+ break;
|
|
default:
|
|
default:
|
|
break;
|
|
break;
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
loadDirectories(fin, sections, summary, conf);
|
|
loadDirectories(fin, sections, summary, conf);
|
|
- loadINodeDirSection(fin, sections, summary, conf);
|
|
|
|
|
|
+ loadINodeDirSection(fin, sections, summary, conf, refIdList);
|
|
metadataMap.sync();
|
|
metadataMap.sync();
|
|
output(conf, summary, fin, sections);
|
|
output(conf, summary, fin, sections);
|
|
}
|
|
}
|
|
@@ -468,6 +493,7 @@ abstract class PBImageTextWriter implements Closeable {
|
|
throws IOException {
|
|
throws IOException {
|
|
InputStream is;
|
|
InputStream is;
|
|
long startTime = Time.monotonicNow();
|
|
long startTime = Time.monotonicNow();
|
|
|
|
+ out.println(getHeader());
|
|
for (FileSummary.Section section : sections) {
|
|
for (FileSummary.Section section : sections) {
|
|
if (SectionName.fromString(section.getName()) == SectionName.INODE) {
|
|
if (SectionName.fromString(section.getName()) == SectionName.INODE) {
|
|
fin.getChannel().position(section.getOffset());
|
|
fin.getChannel().position(section.getOffset());
|
|
@@ -508,7 +534,7 @@ abstract class PBImageTextWriter implements Closeable {
|
|
|
|
|
|
private void loadINodeDirSection(
|
|
private void loadINodeDirSection(
|
|
FileInputStream fin, List<FileSummary.Section> sections,
|
|
FileInputStream fin, List<FileSummary.Section> sections,
|
|
- FileSummary summary, Configuration conf)
|
|
|
|
|
|
+ FileSummary summary, Configuration conf, List<Long> refIdList)
|
|
throws IOException {
|
|
throws IOException {
|
|
LOG.info("Loading INode directory section.");
|
|
LOG.info("Loading INode directory section.");
|
|
long startTime = Time.monotonicNow();
|
|
long startTime = Time.monotonicNow();
|
|
@@ -519,7 +545,7 @@ abstract class PBImageTextWriter implements Closeable {
|
|
InputStream is = FSImageUtil.wrapInputStreamForCompression(conf,
|
|
InputStream is = FSImageUtil.wrapInputStreamForCompression(conf,
|
|
summary.getCodec(), new BufferedInputStream(
|
|
summary.getCodec(), new BufferedInputStream(
|
|
new LimitInputStream(fin, section.getLength())));
|
|
new LimitInputStream(fin, section.getLength())));
|
|
- buildNamespace(is);
|
|
|
|
|
|
+ buildNamespace(is, refIdList);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
long timeTaken = Time.monotonicNow() - startTime;
|
|
long timeTaken = Time.monotonicNow() - startTime;
|
|
@@ -549,7 +575,8 @@ abstract class PBImageTextWriter implements Closeable {
|
|
/**
|
|
/**
|
|
* Scan the INodeDirectory section to construct the namespace.
|
|
* Scan the INodeDirectory section to construct the namespace.
|
|
*/
|
|
*/
|
|
- private void buildNamespace(InputStream in) throws IOException {
|
|
|
|
|
|
+ private void buildNamespace(InputStream in, List<Long> refIdList)
|
|
|
|
+ throws IOException {
|
|
int count = 0;
|
|
int count = 0;
|
|
while (true) {
|
|
while (true) {
|
|
FsImageProto.INodeDirectorySection.DirEntry e =
|
|
FsImageProto.INodeDirectorySection.DirEntry e =
|
|
@@ -562,12 +589,15 @@ abstract class PBImageTextWriter implements Closeable {
|
|
LOG.debug("Scanned {} directories.", count);
|
|
LOG.debug("Scanned {} directories.", count);
|
|
}
|
|
}
|
|
long parentId = e.getParent();
|
|
long parentId = e.getParent();
|
|
- // Referred INode is not support for now.
|
|
|
|
for (int i = 0; i < e.getChildrenCount(); i++) {
|
|
for (int i = 0; i < e.getChildrenCount(); i++) {
|
|
long childId = e.getChildren(i);
|
|
long childId = e.getChildren(i);
|
|
metadataMap.putDirChild(parentId, childId);
|
|
metadataMap.putDirChild(parentId, childId);
|
|
}
|
|
}
|
|
- Preconditions.checkState(e.getRefChildrenCount() == 0);
|
|
|
|
|
|
+ for (int i = e.getChildrenCount();
|
|
|
|
+ i < e.getChildrenCount() + e.getRefChildrenCount(); i++) {
|
|
|
|
+ int refId = e.getRefChildren(i - e.getChildrenCount());
|
|
|
|
+ metadataMap.putDirChild(parentId, refIdList.get(refId));
|
|
|
|
+ }
|
|
}
|
|
}
|
|
LOG.info("Scanned {} INode directories to build namespace.", count);
|
|
LOG.info("Scanned {} INode directories to build namespace.", count);
|
|
}
|
|
}
|
|
@@ -575,15 +605,41 @@ abstract class PBImageTextWriter implements Closeable {
|
|
private void outputINodes(InputStream in) throws IOException {
|
|
private void outputINodes(InputStream in) throws IOException {
|
|
INodeSection s = INodeSection.parseDelimitedFrom(in);
|
|
INodeSection s = INodeSection.parseDelimitedFrom(in);
|
|
LOG.info("Found {} INodes in the INode section", s.getNumInodes());
|
|
LOG.info("Found {} INodes in the INode section", s.getNumInodes());
|
|
|
|
+ long ignored = 0;
|
|
|
|
+ long ignoredSnapshots = 0;
|
|
for (int i = 0; i < s.getNumInodes(); ++i) {
|
|
for (int i = 0; i < s.getNumInodes(); ++i) {
|
|
INode p = INode.parseDelimitedFrom(in);
|
|
INode p = INode.parseDelimitedFrom(in);
|
|
- String parentPath = metadataMap.getParentPath(p.getId());
|
|
|
|
- out.println(getEntry(parentPath, p));
|
|
|
|
|
|
+ try {
|
|
|
|
+ String parentPath = metadataMap.getParentPath(p.getId());
|
|
|
|
+ out.println(getEntry(parentPath, p));
|
|
|
|
+ } catch (IOException ioe) {
|
|
|
|
+ ignored++;
|
|
|
|
+ if (!(ioe instanceof IgnoreSnapshotException)) {
|
|
|
|
+ LOG.warn("Exception caught, ignoring node:{}", p.getId(), ioe);
|
|
|
|
+ } else {
|
|
|
|
+ ignoredSnapshots++;
|
|
|
|
+ if (LOG.isDebugEnabled()) {
|
|
|
|
+ LOG.debug("Exception caught, ignoring node:{}.", p.getId(), ioe);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
|
|
if (LOG.isDebugEnabled() && i % 100000 == 0) {
|
|
if (LOG.isDebugEnabled() && i % 100000 == 0) {
|
|
LOG.debug("Outputted {} INodes.", i);
|
|
LOG.debug("Outputted {} INodes.", i);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
+ if (ignored > 0) {
|
|
|
|
+ LOG.warn("Ignored {} nodes, including {} in snapshots. Please turn on"
|
|
|
|
+ + " debug log for details", ignored, ignoredSnapshots);
|
|
|
|
+ }
|
|
LOG.info("Outputted {} INodes.", s.getNumInodes());
|
|
LOG.info("Outputted {} INodes.", s.getNumInodes());
|
|
}
|
|
}
|
|
|
|
+
|
|
|
|
+ static void ignoreSnapshotName(long inode) throws IOException {
|
|
|
|
+ // Ignore snapshots - we want the output similar to -ls -R.
|
|
|
|
+ if (LOG.isDebugEnabled()) {
|
|
|
|
+ LOG.debug("No snapshot name found for inode {}", inode);
|
|
|
|
+ }
|
|
|
|
+ throw new IgnoreSnapshotException();
|
|
|
|
+ }
|
|
}
|
|
}
|