فهرست منبع

HDFS-6164. Remove lsr in OfflineImageViewer. Contributed by Haohui Mai.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1582533 13f79535-47bb-0310-9956-ffa450edef68
Haohui Mai 11 سال پیش
والد
کامیت
9d2d55e490

+ 2 - 0
hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt

@@ -271,6 +271,8 @@ Release 2.5.0 - UNRELEASED
     HDFS-5978. Create a tool to take fsimage and expose read-only WebHDFS API.
     HDFS-5978. Create a tool to take fsimage and expose read-only WebHDFS API.
     (Akira Ajisaka via wheat9)
     (Akira Ajisaka via wheat9)
 
 
+    HDFS-6164. Remove lsr in OfflineImageViewer. (wheat9)
+
   OPTIMIZATIONS
   OPTIMIZATIONS
 
 
   BUG FIXES 
   BUG FIXES 

+ 0 - 282
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/LsrPBImage.java

@@ -1,282 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs.tools.offlineImageViewer;
-
-import java.io.BufferedInputStream;
-import java.io.FileInputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.PrintWriter;
-import java.io.RandomAccessFile;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.HashMap;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.permission.PermissionStatus;
-import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto;
-import org.apache.hadoop.hdfs.server.namenode.FSImageFormatPBINode;
-import org.apache.hadoop.hdfs.server.namenode.FSImageFormatProtobuf.SectionName;
-import org.apache.hadoop.hdfs.server.namenode.FSImageUtil;
-import org.apache.hadoop.hdfs.server.namenode.FsImageProto.FileSummary;
-import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeDirectorySection;
-import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeReferenceSection;
-import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeSection;
-import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeSection.INode;
-import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeSection.INodeDirectory;
-import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeSection.INodeFile;
-import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeSection.INodeSymlink;
-import org.apache.hadoop.hdfs.server.namenode.FsImageProto.StringTableSection;
-import org.apache.hadoop.hdfs.server.namenode.INodeId;
-import org.apache.hadoop.io.IOUtils;
-
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-import com.google.common.io.LimitInputStream;
-
-/**
- * LsrPBImage displays the blocks of the namespace in a format very similar
- * to the output of ls/lsr.  Entries are marked as directories or not,
- * permissions listed, replication, username and groupname, along with size,
- * modification date and full path.
- *
- * Note: A significant difference between the output of the lsr command
- * and this image visitor is that this class cannot sort the file entries;
- * they are listed in the order they are stored within the fsimage file. 
- * Therefore, the output of this class cannot be directly compared to the
- * output of the lsr command.
- */
-final class LsrPBImage {
-
-  private static final Log LOG = LogFactory.getLog(LsrPBImage.class);
-
-  private final Configuration conf;
-  private final PrintWriter out;
-  private String[] stringTable;
-  private final HashMap<Long, INodeSection.INode> inodes = Maps.newHashMap();
-  private final HashMap<Long, long[]> dirmap = Maps.newHashMap();
-  private final ArrayList<INodeReferenceSection.INodeReference> refList = Lists.newArrayList();
-
-  public LsrPBImage(Configuration conf, PrintWriter out) {
-    this.conf = conf;
-    this.out = out;
-  }
-
-  public void visit(RandomAccessFile file) throws IOException {
-    if (!FSImageUtil.checkFileFormat(file)) {
-      throw new IOException("Unrecognized FSImage");
-    }
-
-    FileSummary summary = FSImageUtil.loadSummary(file);
-    FileInputStream fin = null;
-    try {
-      fin = new FileInputStream(file.getFD());
-
-      ArrayList<FileSummary.Section> sections = Lists.newArrayList(summary
-          .getSectionsList());
-      Collections.sort(sections, new Comparator<FileSummary.Section>() {
-        @Override
-        public int compare(FileSummary.Section s1, FileSummary.Section s2) {
-          SectionName n1 = SectionName.fromString(s1.getName());
-          SectionName n2 = SectionName.fromString(s2.getName());
-          if (n1 == null) {
-            return n2 == null ? 0 : -1;
-          } else if (n2 == null) {
-            return -1;
-          } else {
-            return n1.ordinal() - n2.ordinal();
-          }
-        }
-      });
-
-      for (FileSummary.Section s : sections) {
-        fin.getChannel().position(s.getOffset());
-        InputStream is = FSImageUtil.wrapInputStreamForCompression(conf,
-            summary.getCodec(), new BufferedInputStream(new LimitInputStream(
-                fin, s.getLength())));
-
-        switch (SectionName.fromString(s.getName())) {
-        case STRING_TABLE:
-          loadStringTable(is);
-          break;
-        case INODE:
-          loadINodeSection(is);
-          break;
-        case INODE_REFERENCE:
-          loadINodeReferenceSection(is);
-          break;
-        case INODE_DIR:
-          loadINodeDirectorySection(is);
-          break;
-        default:
-          break;
-        }
-      }
-      list("", INodeId.ROOT_INODE_ID);
-    } finally {
-      IOUtils.cleanup(null, fin);
-    }
-  }
-
-  private void list(String parent, long dirId) {
-    INode inode = inodes.get(dirId);
-    if (LOG.isTraceEnabled()) {
-      LOG.trace("Listing directory id " + dirId + " parent '" + parent
-          + "' (INode is " + inode + ")");
-    }
-    listINode(parent.isEmpty() ? "/" : parent, inode);
-    long[] children = dirmap.get(dirId);
-    if (children == null) {
-      return;
-    }
-    String newParent = parent + inode.getName().toStringUtf8() + "/";
-    for (long cid : children) {
-      list(newParent, cid);
-    }
-  }
-
-  private void listINode(String parent, INode inode) {
-    switch (inode.getType()) {
-    case FILE: {
-      INodeFile f = inode.getFile();
-      PermissionStatus p = FSImageFormatPBINode.Loader.loadPermission(
-          f.getPermission(), stringTable);
-      out.print(String.format("-%s %2s %8s %10s %10s %10d %s%s%n", p
-          .getPermission().toString(), f.getReplication(), p.getUserName(), p
-          .getGroupName(), f.getModificationTime(), getFileSize(f), parent,
-          inode.getName().toStringUtf8()));
-    }
-      break;
-    case DIRECTORY: {
-      INodeDirectory d = inode.getDirectory();
-      PermissionStatus p = FSImageFormatPBINode.Loader.loadPermission(
-          d.getPermission(), stringTable);
-      out.print(String.format("d%s  - %8s %10s %10s %10d %s%s%n", p
-          .getPermission().toString(), p.getUserName(), p.getGroupName(), d
-          .getModificationTime(), 0, parent, inode.getName().toStringUtf8()));
-    }
-      break;
-    case SYMLINK: {
-      INodeSymlink d = inode.getSymlink();
-      PermissionStatus p = FSImageFormatPBINode.Loader.loadPermission(
-          d.getPermission(), stringTable);
-      out.print(String.format("-%s  - %8s %10s %10s %10d %s%s -> %s%n", p
-          .getPermission().toString(), p.getUserName(), p.getGroupName(), d
-          .getModificationTime(), 0, parent, inode.getName().toStringUtf8(),
-          d.getTarget().toStringUtf8()));
-    }
-      break;
-    default:
-      break;
-    }
-  }
-
-  private long getFileSize(INodeFile f) {
-    long size = 0;
-    for (BlockProto p : f.getBlocksList()) {
-      size += p.getNumBytes();
-    }
-    return size;
-  }
-
-  private void loadINodeDirectorySection(InputStream in) throws IOException {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Loading directory section");
-    }
-    while (true) {
-      INodeDirectorySection.DirEntry e = INodeDirectorySection.DirEntry
-          .parseDelimitedFrom(in);
-      // note that in is a LimitedInputStream
-      if (e == null) {
-        break;
-      }
-      long[] l = new long[e.getChildrenCount() + e.getRefChildrenCount()];
-      for (int i = 0; i < e.getChildrenCount(); ++i) {
-        l[i] = e.getChildren(i);
-      }
-      for (int i = e.getChildrenCount(); i < l.length; i++) {
-        int refId = e.getRefChildren(i - e.getChildrenCount());
-        l[i] = refList.get(refId).getReferredId();
-      }
-      dirmap.put(e.getParent(), l);
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Loaded directory (parent " + e.getParent()
-            + ") with " + e.getChildrenCount() + " children and "
-            + e.getRefChildrenCount() + " reference children");
-      }
-    }
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Loaded " + dirmap.size() + " directories");
-    }
-  }
-
-  private void loadINodeReferenceSection(InputStream in) throws IOException {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Loading inode reference section");
-    }
-    while (true) {
-      INodeReferenceSection.INodeReference e = INodeReferenceSection
-          .INodeReference.parseDelimitedFrom(in);
-      if (e == null) {
-        break;
-      }
-      refList.add(e);
-      if (LOG.isTraceEnabled()) {
-        LOG.trace("Loaded inode reference named '" + e.getName()
-            + "' referring to id " + e.getReferredId() + "");
-      }
-    }
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Loaded " + refList.size() + " inode references");
-    }
-  }
-
-  private void loadINodeSection(InputStream in) throws IOException {
-    INodeSection s = INodeSection.parseDelimitedFrom(in);
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Found " + s.getNumInodes() + " inodes in inode section");
-    }
-    for (int i = 0; i < s.getNumInodes(); ++i) {
-      INodeSection.INode p = INodeSection.INode.parseDelimitedFrom(in);
-      inodes.put(p.getId(), p);
-      if (LOG.isTraceEnabled()) {
-        LOG.trace("Loaded inode id " + p.getId() + " type " + p.getType()
-            + " name '" + p.getName().toStringUtf8() + "'");
-      }
-    }
-  }
-
-  private void loadStringTable(InputStream in) throws IOException {
-    StringTableSection s = StringTableSection.parseDelimitedFrom(in);
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Found " + s.getNumEntry() + " strings in string section");
-    }
-    stringTable = new String[s.getNumEntry() + 1];
-    for (int i = 0; i < s.getNumEntry(); ++i) {
-      StringTableSection.Entry e = StringTableSection.Entry
-          .parseDelimitedFrom(in);
-      stringTable[e.getId()] = e.getStr();
-      if (LOG.isTraceEnabled()) {
-        LOG.trace("Loaded string " + e.getStr());
-      }
-    }
-  }
-}

+ 0 - 7
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/OfflineImageViewerPB.java

@@ -57,11 +57,6 @@ public class OfflineImageViewerPB {
       + "order to process an image file.\n"
       + "order to process an image file.\n"
       + "\n"
       + "\n"
       + "The following image processors are available:\n"
       + "The following image processors are available:\n"
-      + "  * Ls: The default image processor generates an lsr-style listing\n"
-      + "    of the files in the namespace, with the same fields in the same\n"
-      + "    order.  Note that in order to correctly determine file sizes,\n"
-      + "    this formatter cannot skip blocks and will override the\n"
-      + "    -skipBlocks option.\n"
       + "  * XML: This processor creates an XML document with all elements of\n"
       + "  * XML: This processor creates an XML document with all elements of\n"
       + "    the fsimage enumerated, suitable for further analysis by XML\n"
       + "    the fsimage enumerated, suitable for further analysis by XML\n"
       + "    tools.\n"
       + "    tools.\n"
@@ -169,8 +164,6 @@ public class OfflineImageViewerPB {
         String addr = cmd.getOptionValue("addr", "localhost:5978");
         String addr = cmd.getOptionValue("addr", "localhost:5978");
         new WebImageViewer(NetUtils.createSocketAddr(addr))
         new WebImageViewer(NetUtils.createSocketAddr(addr))
             .initServerAndWait(inputFile);
             .initServerAndWait(inputFile);
-      } else {
-        new LsrPBImage(conf, out).visit(new RandomAccessFile(inputFile, "r"));
       }
       }
       return 0;
       return 0;
     } catch (EOFException e) {
     } catch (EOFException e) {

+ 0 - 53
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewer.java

@@ -79,16 +79,6 @@ public class TestOfflineImageViewer {
   private static final String TEST_RENEWER = "JobTracker";
   private static final String TEST_RENEWER = "JobTracker";
   private static File originalFsimage = null;
   private static File originalFsimage = null;
 
 
-  // Elements of lines of ls-file output to be compared to FileStatus instance
-  private static final class LsElements {
-    private String perms;
-    private int replication;
-    private String username;
-    private String groupname;
-    private long filesize;
-    private boolean isDir;
-  }
-
   // namespace as written to dfs, to be compared with viewer's output
   // namespace as written to dfs, to be compared with viewer's output
   final static HashMap<String, FileStatus> writtenFiles = Maps.newHashMap();
   final static HashMap<String, FileStatus> writtenFiles = Maps.newHashMap();
 
 
@@ -176,37 +166,6 @@ public class TestOfflineImageViewer {
     return hdfs.getFileStatus(new Path(file));
     return hdfs.getFileStatus(new Path(file));
   }
   }
 
 
-  // Verify that we can correctly generate an ls-style output for a valid
-  // fsimage
-  @Test
-  public void outputOfLSVisitor() throws IOException {
-    StringWriter output = new StringWriter();
-    PrintWriter out = new PrintWriter(output);
-    LsrPBImage v = new LsrPBImage(new Configuration(), out);
-    v.visit(new RandomAccessFile(originalFsimage, "r"));
-    out.close();
-    Pattern pattern = Pattern
-        .compile("([d\\-])([rwx\\-]{9})\\s*(-|\\d+)\\s*(\\w+)\\s*(\\w+)\\s*(\\d+)\\s*(\\d+)\\s*([\b/]+)");
-    int count = 0;
-    for (String s : output.toString().split("\n")) {
-      Matcher m = pattern.matcher(s);
-      assertTrue(m.find());
-      LsElements e = new LsElements();
-      e.isDir = m.group(1).equals("d");
-      e.perms = m.group(2);
-      e.replication = m.group(3).equals("-") ? 0 : Integer.parseInt(m.group(3));
-      e.username = m.group(4);
-      e.groupname = m.group(5);
-      e.filesize = Long.parseLong(m.group(7));
-      String path = m.group(8);
-      if (!path.equals("/")) {
-        compareFiles(writtenFiles.get(path), e);
-      }
-      ++count;
-    }
-    assertEquals(writtenFiles.size() + 1, count);
-  }
-
   @Test(expected = IOException.class)
   @Test(expected = IOException.class)
   public void testTruncatedFSImage() throws IOException {
   public void testTruncatedFSImage() throws IOException {
     File truncatedFile = folder.newFile();
     File truncatedFile = folder.newFile();
@@ -216,18 +175,6 @@ public class TestOfflineImageViewer {
         output)).visit(new RandomAccessFile(truncatedFile, "r"));
         output)).visit(new RandomAccessFile(truncatedFile, "r"));
   }
   }
 
 
-  // Compare two files as listed in the original namespace FileStatus and
-  // the output of the ls file from the image processor
-  private void compareFiles(FileStatus fs, LsElements elements) {
-    assertEquals("directory listed as such", fs.isDirectory(), elements.isDir);
-    assertEquals("perms string equal", fs.getPermission().toString(),
-        elements.perms);
-    assertEquals("replication equal", fs.getReplication(), elements.replication);
-    assertEquals("owner equal", fs.getOwner(), elements.username);
-    assertEquals("group equal", fs.getGroup(), elements.groupname);
-    assertEquals("lengths equal", fs.getLen(), elements.filesize);
-  }
-
   private void copyPartOfFile(File src, File dest) throws IOException {
   private void copyPartOfFile(File src, File dest) throws IOException {
     FileInputStream in = null;
     FileInputStream in = null;
     FileOutputStream out = null;
     FileOutputStream out = null;