|
@@ -20,6 +20,8 @@ package org.apache.hadoop.hdfs.tools.offlineImageViewer;
|
|
|
import com.google.common.base.Preconditions;
|
|
|
import com.google.common.collect.ImmutableList;
|
|
|
import com.google.common.collect.Lists;
|
|
|
+import org.apache.commons.lang3.StringUtils;
|
|
|
+import org.apache.commons.text.StringEscapeUtils;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.fs.Path;
|
|
|
import org.apache.hadoop.fs.permission.PermissionStatus;
|
|
@@ -56,6 +58,7 @@ import java.nio.ByteBuffer;
|
|
|
import java.util.ArrayList;
|
|
|
import java.util.Collections;
|
|
|
import java.util.Comparator;
|
|
|
+import java.util.concurrent.atomic.AtomicInteger;
|
|
|
import java.util.HashMap;
|
|
|
import java.util.LinkedHashMap;
|
|
|
import java.util.List;
|
|
@@ -88,6 +91,9 @@ abstract class PBImageTextWriter implements Closeable {
|
|
|
private static final Logger LOG =
|
|
|
LoggerFactory.getLogger(PBImageTextWriter.class);
|
|
|
|
|
|
+ static final String DEFAULT_DELIMITER = "\t";
|
|
|
+ static final String CRLF = StringUtils.CR + StringUtils.LF;
|
|
|
+
|
|
|
/**
|
|
|
* This metadata map is used to construct the namespace before generating
|
|
|
* text outputs.
|
|
@@ -114,6 +120,15 @@ abstract class PBImageTextWriter implements Closeable {
|
|
|
|
|
|
/** Synchronize metadata to persistent storage, if possible */
|
|
|
public void sync() throws IOException;
|
|
|
+
|
|
|
+ /** Returns the name of inode. */
|
|
|
+ String getName(long id) throws IOException;
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Returns the id of the parent's inode, if mentioned in
|
|
|
+ * INodeDirectorySection, throws IgnoreSnapshotException otherwise.
|
|
|
+ */
|
|
|
+ long getParentId(long id) throws IOException;
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -142,18 +157,32 @@ abstract class PBImageTextWriter implements Closeable {
|
|
|
/**
|
|
|
* Returns the full path of this directory.
|
|
|
*/
|
|
|
- private String getPath() {
|
|
|
+ String getPath() throws IgnoreSnapshotException {
|
|
|
if (this.parent == null) {
|
|
|
- return "/";
|
|
|
+ if (this.inode == INodeId.ROOT_INODE_ID) {
|
|
|
+ return "/";
|
|
|
+ } else {
|
|
|
+ if (LOG.isDebugEnabled()) {
|
|
|
+ LOG.debug("Not root inode with id {} having no parent.", inode);
|
|
|
+ }
|
|
|
+ throw PBImageTextWriter.createIgnoredSnapshotException(inode);
|
|
|
+ }
|
|
|
}
|
|
|
if (this.path == null) {
|
|
|
this.path = new Path(parent.getPath(), name.isEmpty() ? "/" : name).
|
|
|
toString();
|
|
|
- this.name = null;
|
|
|
}
|
|
|
return this.path;
|
|
|
}
|
|
|
|
|
|
+ String getName() throws IgnoreSnapshotException {
|
|
|
+ return name;
|
|
|
+ }
|
|
|
+
|
|
|
+ long getId() {
|
|
|
+ return inode;
|
|
|
+ }
|
|
|
+
|
|
|
@Override
|
|
|
public boolean equals(Object o) {
|
|
|
return o instanceof Dir && inode == ((Dir) o).inode;
|
|
@@ -165,6 +194,29 @@ abstract class PBImageTextWriter implements Closeable {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * If the Dir entry does not exist (i.e. the inode was not contained in
|
|
|
+ * INodeSection) we still create a Dir entry which throws exceptions
|
|
|
+ * for calls other than getId().
|
|
|
+ * We can make sure this way, the getId and getParentId calls will
|
|
|
+ * always succeed if we have the information.
|
|
|
+ */
|
|
|
+ private static class CorruptedDir extends Dir {
|
|
|
+ CorruptedDir(long inode) {
|
|
|
+ super(inode, null);
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ String getPath() throws IgnoreSnapshotException {
|
|
|
+ throw PBImageTextWriter.createIgnoredSnapshotException(getId());
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ String getName() throws IgnoreSnapshotException {
|
|
|
+ throw PBImageTextWriter.createIgnoredSnapshotException(getId());
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
/** INode Id to Dir object mapping */
|
|
|
private Map<Long, Dir> dirMap = new HashMap<>();
|
|
|
|
|
@@ -178,13 +230,20 @@ abstract class PBImageTextWriter implements Closeable {
|
|
|
public void close() throws IOException {
|
|
|
}
|
|
|
|
|
|
+ private Dir getOrCreateCorrupted(long id) {
|
|
|
+ Dir dir = dirMap.get(id);
|
|
|
+ if (dir == null) {
|
|
|
+ dir = new CorruptedDir(id);
|
|
|
+ dirMap.put(id, dir);
|
|
|
+ }
|
|
|
+ return dir;
|
|
|
+ }
|
|
|
+
|
|
|
@Override
|
|
|
public void putDirChild(long parentId, long childId) {
|
|
|
- Dir parent = dirMap.get(parentId);
|
|
|
- Dir child = dirMap.get(childId);
|
|
|
- if (child != null) {
|
|
|
- child.setParent(parent);
|
|
|
- }
|
|
|
+ Dir parent = getOrCreateCorrupted(parentId);
|
|
|
+ Dir child = getOrCreateCorrupted(childId);
|
|
|
+ child.setParent(parent);
|
|
|
Preconditions.checkState(!dirChildMap.containsKey(childId));
|
|
|
dirChildMap.put(childId, parent);
|
|
|
}
|
|
@@ -199,13 +258,13 @@ abstract class PBImageTextWriter implements Closeable {
|
|
|
@Override
|
|
|
public String getParentPath(long inode) throws IOException {
|
|
|
if (inode == INodeId.ROOT_INODE_ID) {
|
|
|
- return "";
|
|
|
+ return "/";
|
|
|
}
|
|
|
Dir parent = dirChildMap.get(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);
|
|
|
+ throw PBImageTextWriter.createIgnoredSnapshotException(inode);
|
|
|
}
|
|
|
return parent.getPath();
|
|
|
}
|
|
@@ -213,6 +272,24 @@ abstract class PBImageTextWriter implements Closeable {
|
|
|
@Override
|
|
|
public void sync() {
|
|
|
}
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public String getName(long id) throws IgnoreSnapshotException {
|
|
|
+ Dir dir = dirMap.get(id);
|
|
|
+ if (dir != null) {
|
|
|
+ return dir.getName();
|
|
|
+ }
|
|
|
+ throw PBImageTextWriter.createIgnoredSnapshotException(id);
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public long getParentId(long id) throws IgnoreSnapshotException {
|
|
|
+ Dir parentDir = dirChildMap.get(id);
|
|
|
+ if (parentDir != null) {
|
|
|
+ return parentDir.getId();
|
|
|
+ }
|
|
|
+ throw PBImageTextWriter.createIgnoredSnapshotException(id);
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -353,33 +430,37 @@ abstract class PBImageTextWriter implements Closeable {
|
|
|
dirMap.put(toBytes(dir.getId()), toBytes(dir.getName().toStringUtf8()));
|
|
|
}
|
|
|
|
|
|
- @Override
|
|
|
- public String getParentPath(long inode) throws IOException {
|
|
|
- if (inode == INodeId.ROOT_INODE_ID) {
|
|
|
- return "/";
|
|
|
- }
|
|
|
+ private long getFromDirChildMap(long inode) throws IOException {
|
|
|
byte[] bytes = dirChildMap.get(toBytes(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);
|
|
|
+ throw PBImageTextWriter.createIgnoredSnapshotException(inode);
|
|
|
}
|
|
|
if (bytes.length != 8) {
|
|
|
throw new IOException(
|
|
|
"bytes array length error. Actual length is " + bytes.length);
|
|
|
}
|
|
|
- long parent = toLong(bytes);
|
|
|
+ return toLong(bytes);
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public String getParentPath(long inode) throws IOException {
|
|
|
+ if (inode == INodeId.ROOT_INODE_ID) {
|
|
|
+ return "/";
|
|
|
+ }
|
|
|
+ long parent = getFromDirChildMap(inode);
|
|
|
if (!dirPathCache.containsKey(parent)) {
|
|
|
- bytes = dirMap.get(toBytes(parent));
|
|
|
+ byte[] bytes = dirMap.get(toBytes(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);
|
|
|
+ throw PBImageTextWriter.createIgnoredSnapshotException(inode);
|
|
|
}
|
|
|
String parentName = toString(bytes);
|
|
|
String parentPath =
|
|
|
new Path(getParentPath(parent),
|
|
|
- parentName.isEmpty()? "/" : parentName).toString();
|
|
|
+ parentName.isEmpty() ? "/" : parentName).toString();
|
|
|
dirPathCache.put(parent, parentPath);
|
|
|
}
|
|
|
return dirPathCache.get(parent);
|
|
@@ -390,11 +471,26 @@ abstract class PBImageTextWriter implements Closeable {
|
|
|
dirChildMap.sync();
|
|
|
dirMap.sync();
|
|
|
}
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public String getName(long id) throws IOException {
|
|
|
+ byte[] bytes = dirMap.get(toBytes(id));
|
|
|
+ if (bytes != null) {
|
|
|
+ return toString(bytes);
|
|
|
+ }
|
|
|
+ throw PBImageTextWriter.createIgnoredSnapshotException(id);
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public long getParentId(long id) throws IOException {
|
|
|
+ return getFromDirChildMap(id);
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
private SerialNumberManager.StringTable stringTable;
|
|
|
private PrintStream out;
|
|
|
private MetadataMap metadataMap = null;
|
|
|
+ private String delimiter;
|
|
|
|
|
|
/**
|
|
|
* Construct a PB FsImage writer to generate text file.
|
|
@@ -402,8 +498,10 @@ abstract class PBImageTextWriter implements Closeable {
|
|
|
* @param tempPath the path to store metadata. If it is empty, store metadata
|
|
|
* in memory instead.
|
|
|
*/
|
|
|
- PBImageTextWriter(PrintStream out, String tempPath) throws IOException {
|
|
|
+ PBImageTextWriter(PrintStream out, String delimiter, String tempPath)
|
|
|
+ throws IOException {
|
|
|
this.out = out;
|
|
|
+ this.delimiter = delimiter;
|
|
|
if (tempPath.isEmpty()) {
|
|
|
metadataMap = new InMemoryMetadataDB();
|
|
|
} else {
|
|
@@ -417,6 +515,29 @@ abstract class PBImageTextWriter implements Closeable {
|
|
|
IOUtils.cleanup(null, metadataMap);
|
|
|
}
|
|
|
|
|
|
+ void append(StringBuffer buffer, int field) {
|
|
|
+ buffer.append(delimiter);
|
|
|
+ buffer.append(field);
|
|
|
+ }
|
|
|
+
|
|
|
+ void append(StringBuffer buffer, long field) {
|
|
|
+ buffer.append(delimiter);
|
|
|
+ buffer.append(field);
|
|
|
+ }
|
|
|
+
|
|
|
+ void append(StringBuffer buffer, String field) {
|
|
|
+ buffer.append(delimiter);
|
|
|
+
|
|
|
+ String escapedField = StringEscapeUtils.escapeCsv(field);
|
|
|
+ if (escapedField.contains(CRLF)) {
|
|
|
+ escapedField = escapedField.replace(CRLF, "%x0D%x0A");
|
|
|
+ } else if (escapedField.contains(StringUtils.LF)) {
|
|
|
+ escapedField = escapedField.replace(StringUtils.LF, "%x0A");
|
|
|
+ }
|
|
|
+
|
|
|
+ buffer.append(escapedField);
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* Get text output for the given inode.
|
|
|
* @param parent the path of parent directory
|
|
@@ -429,6 +550,13 @@ abstract class PBImageTextWriter implements Closeable {
|
|
|
*/
|
|
|
abstract protected String getHeader();
|
|
|
|
|
|
+ /**
|
|
|
+ * Method called at the end of output() phase after all the inodes
|
|
|
+ * with known parentPath has been printed out. Can be used to print
|
|
|
+ * additional data depending on the written inodes.
|
|
|
+ */
|
|
|
+ abstract protected void afterOutput() throws IOException;
|
|
|
+
|
|
|
public void visit(RandomAccessFile file) throws IOException {
|
|
|
Configuration conf = new Configuration();
|
|
|
if (!FSImageUtil.checkFileFormat(file)) {
|
|
@@ -489,6 +617,19 @@ abstract class PBImageTextWriter implements Closeable {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ void putDirChildToMetadataMap(long parentId, long childId)
|
|
|
+ throws IOException {
|
|
|
+ metadataMap.putDirChild(parentId, childId);
|
|
|
+ }
|
|
|
+
|
|
|
+ String getNodeName(long id) throws IOException {
|
|
|
+ return metadataMap.getName(id);
|
|
|
+ }
|
|
|
+
|
|
|
+ long getParentId(long id) throws IOException {
|
|
|
+ return metadataMap.getParentId(id);
|
|
|
+ }
|
|
|
+
|
|
|
private void output(Configuration conf, FileSummary summary,
|
|
|
FileInputStream fin, ArrayList<FileSummary.Section> sections)
|
|
|
throws IOException {
|
|
@@ -504,6 +645,7 @@ abstract class PBImageTextWriter implements Closeable {
|
|
|
outputINodes(is);
|
|
|
}
|
|
|
}
|
|
|
+ afterOutput();
|
|
|
long timeTaken = Time.monotonicNow() - startTime;
|
|
|
LOG.debug("Time to output inodes: {}ms", timeTaken);
|
|
|
}
|
|
@@ -553,22 +695,31 @@ abstract class PBImageTextWriter implements Closeable {
|
|
|
LOG.info("Finished loading INode directory section in {}ms", timeTaken);
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * Checks the inode (saves if directory), and counts them. Can be overridden
|
|
|
+ * if additional steps are taken when iterating through INodeSection.
|
|
|
+ */
|
|
|
+ protected void checkNode(INode p, AtomicInteger numDirs) throws IOException {
|
|
|
+ if (p.hasDirectory()) {
|
|
|
+ metadataMap.putDir(p);
|
|
|
+ numDirs.incrementAndGet();
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* Load the filenames of the directories from the INode section.
|
|
|
*/
|
|
|
- private void loadDirectoriesInINodeSection(InputStream in) throws IOException {
|
|
|
+ private void loadDirectoriesInINodeSection(InputStream in)
|
|
|
+ throws IOException {
|
|
|
INodeSection s = INodeSection.parseDelimitedFrom(in);
|
|
|
LOG.info("Loading directories in INode section.");
|
|
|
- int numDirs = 0;
|
|
|
+ AtomicInteger numDirs = new AtomicInteger(0);
|
|
|
for (int i = 0; i < s.getNumInodes(); ++i) {
|
|
|
INode p = INode.parseDelimitedFrom(in);
|
|
|
if (LOG.isDebugEnabled() && i % 10000 == 0) {
|
|
|
LOG.debug("Scanned {} inodes.", i);
|
|
|
}
|
|
|
- if (p.hasDirectory()) {
|
|
|
- metadataMap.putDir(p);
|
|
|
- numDirs++;
|
|
|
- }
|
|
|
+ checkNode(p, numDirs);
|
|
|
}
|
|
|
LOG.info("Found {} directories in INode section.", numDirs);
|
|
|
}
|
|
@@ -576,7 +727,7 @@ abstract class PBImageTextWriter implements Closeable {
|
|
|
/**
|
|
|
* Scan the INodeDirectory section to construct the namespace.
|
|
|
*/
|
|
|
- private void buildNamespace(InputStream in, List<Long> refIdList)
|
|
|
+ protected void buildNamespace(InputStream in, List<Long> refIdList)
|
|
|
throws IOException {
|
|
|
int count = 0;
|
|
|
while (true) {
|
|
@@ -603,6 +754,12 @@ abstract class PBImageTextWriter implements Closeable {
|
|
|
LOG.info("Scanned {} INode directories to build namespace.", count);
|
|
|
}
|
|
|
|
|
|
+ void printIfNotEmpty(String line) {
|
|
|
+ if (!line.isEmpty()) {
|
|
|
+ out.println(line);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
private void outputINodes(InputStream in) throws IOException {
|
|
|
INodeSection s = INodeSection.parseDelimitedFrom(in);
|
|
|
LOG.info("Found {} INodes in the INode section", s.getNumInodes());
|
|
@@ -612,7 +769,7 @@ abstract class PBImageTextWriter implements Closeable {
|
|
|
INode p = INode.parseDelimitedFrom(in);
|
|
|
try {
|
|
|
String parentPath = metadataMap.getParentPath(p.getId());
|
|
|
- out.println(getEntry(parentPath, p));
|
|
|
+ printIfNotEmpty(getEntry(parentPath, p));
|
|
|
} catch (IOException ioe) {
|
|
|
ignored++;
|
|
|
if (!(ioe instanceof IgnoreSnapshotException)) {
|
|
@@ -636,11 +793,12 @@ abstract class PBImageTextWriter implements Closeable {
|
|
|
LOG.info("Outputted {} INodes.", s.getNumInodes());
|
|
|
}
|
|
|
|
|
|
- static void ignoreSnapshotName(long inode) throws IOException {
|
|
|
+ private static IgnoreSnapshotException createIgnoredSnapshotException(
|
|
|
+ long inode) {
|
|
|
// 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();
|
|
|
+ return new IgnoreSnapshotException();
|
|
|
}
|
|
|
}
|