ソースを参照

HDFS-5915. Merging r1566824 from trunk to branch-2

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-2@1568552 13f79535-47bb-0310-9956-ffa450edef68
Arpit Agarwal 11 年 前
コミット
9829487fff

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

@@ -204,6 +204,9 @@ Release 2.4.0 - UNRELEASED
     HDFS-5911. The id of a CacheDirective instance does not get serialized in
     HDFS-5911. The id of a CacheDirective instance does not get serialized in
     the protobuf-fsimage. (Haohui Mai via jing9)
     the protobuf-fsimage. (Haohui Mai via jing9)
 
 
+    HDFS-5915. Refactor FSImageFormatProtobuf to simplify cross section reads.
+    (Haohui Mai via cnauroth)
+
 Release 2.3.1 - UNRELEASED
 Release 2.3.1 - UNRELEASED
 
 
   INCOMPATIBLE CHANGES
   INCOMPATIBLE CHANGES

+ 16 - 13
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatPBINode.java

@@ -38,7 +38,7 @@ import org.apache.hadoop.hdfs.protocolPB.PBHelper;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
-import org.apache.hadoop.hdfs.server.namenode.FSImageFormatProtobuf.StringMap;
+import org.apache.hadoop.hdfs.server.namenode.FSImageFormatProtobuf.SaverContext;
 import org.apache.hadoop.hdfs.server.namenode.FsImageProto.FileSummary;
 import org.apache.hadoop.hdfs.server.namenode.FsImageProto.FileSummary;
 import org.apache.hadoop.hdfs.server.namenode.FsImageProto.FilesUnderConstructionSection.FileUnderConstructionEntry;
 import org.apache.hadoop.hdfs.server.namenode.FsImageProto.FilesUnderConstructionSection.FileUnderConstructionEntry;
 import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeDirectorySection;
 import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeDirectorySection;
@@ -208,7 +208,7 @@ public final class FSImageFormatPBINode {
       case FILE:
       case FILE:
         return loadINodeFile(n);
         return loadINodeFile(n);
       case DIRECTORY:
       case DIRECTORY:
-        return loadINodeDirectory(n, parent.getStringTable());
+        return loadINodeDirectory(n, parent.getLoaderContext().getStringTable());
       case SYMLINK:
       case SYMLINK:
         return loadINodeSymlink(n);
         return loadINodeSymlink(n);
       default:
       default:
@@ -228,7 +228,7 @@ public final class FSImageFormatPBINode {
         blocks[i] = new BlockInfo(PBHelper.convert(bp.get(i)), replication);
         blocks[i] = new BlockInfo(PBHelper.convert(bp.get(i)), replication);
       }
       }
       final PermissionStatus permissions = loadPermission(f.getPermission(),
       final PermissionStatus permissions = loadPermission(f.getPermission(),
-          parent.getStringTable());
+          parent.getLoaderContext().getStringTable());
 
 
       final INodeFile file = new INodeFile(n.getId(),
       final INodeFile file = new INodeFile(n.getId(),
           n.getName().toByteArray(), permissions, f.getModificationTime(),
           n.getName().toByteArray(), permissions, f.getModificationTime(),
@@ -253,13 +253,14 @@ public final class FSImageFormatPBINode {
       assert n.getType() == INodeSection.INode.Type.SYMLINK;
       assert n.getType() == INodeSection.INode.Type.SYMLINK;
       INodeSection.INodeSymlink s = n.getSymlink();
       INodeSection.INodeSymlink s = n.getSymlink();
       final PermissionStatus permissions = loadPermission(s.getPermission(),
       final PermissionStatus permissions = loadPermission(s.getPermission(),
-          parent.getStringTable());
+          parent.getLoaderContext().getStringTable());
       return new INodeSymlink(n.getId(), n.getName().toByteArray(), permissions,
       return new INodeSymlink(n.getId(), n.getName().toByteArray(), permissions,
           0, 0, s.getTarget().toStringUtf8());
           0, 0, s.getTarget().toStringUtf8());
     }
     }
 
 
     private void loadRootINode(INodeSection.INode p) {
     private void loadRootINode(INodeSection.INode p) {
-      INodeDirectory root = loadINodeDirectory(p, parent.getStringTable());
+      INodeDirectory root = loadINodeDirectory(p, parent.getLoaderContext()
+          .getStringTable());
       final Quota.Counts q = root.getQuotaCounts();
       final Quota.Counts q = root.getQuotaCounts();
       final long nsQuota = q.get(Quota.NAMESPACE);
       final long nsQuota = q.get(Quota.NAMESPACE);
       final long dsQuota = q.get(Quota.DISKSPACE);
       final long dsQuota = q.get(Quota.DISKSPACE);
@@ -273,16 +274,17 @@ public final class FSImageFormatPBINode {
 
 
   public final static class Saver {
   public final static class Saver {
     private static long buildPermissionStatus(INodeAttributes n,
     private static long buildPermissionStatus(INodeAttributes n,
-        final StringMap stringMap) {
-      long userId = stringMap.getStringId(n.getUserName());
-      long groupId = stringMap.getStringId(n.getGroupName());
+        final SaverContext.DeduplicationMap<String> stringMap) {
+      long userId = stringMap.getId(n.getUserName());
+      long groupId = stringMap.getId(n.getGroupName());
       return ((userId & USER_GROUP_STRID_MASK) << USER_STRID_OFFSET)
       return ((userId & USER_GROUP_STRID_MASK) << USER_STRID_OFFSET)
           | ((groupId & USER_GROUP_STRID_MASK) << GROUP_STRID_OFFSET)
           | ((groupId & USER_GROUP_STRID_MASK) << GROUP_STRID_OFFSET)
           | n.getFsPermissionShort();
           | n.getFsPermissionShort();
     }
     }
 
 
     public static INodeSection.INodeFile.Builder buildINodeFile(
     public static INodeSection.INodeFile.Builder buildINodeFile(
-        INodeFileAttributes file, final StringMap stringMap) {
+        INodeFileAttributes file,
+        final SaverContext.DeduplicationMap<String> stringMap) {
       INodeSection.INodeFile.Builder b = INodeSection.INodeFile.newBuilder()
       INodeSection.INodeFile.Builder b = INodeSection.INodeFile.newBuilder()
           .setAccessTime(file.getAccessTime())
           .setAccessTime(file.getAccessTime())
           .setModificationTime(file.getModificationTime())
           .setModificationTime(file.getModificationTime())
@@ -293,7 +295,8 @@ public final class FSImageFormatPBINode {
     }
     }
 
 
     public static INodeSection.INodeDirectory.Builder buildINodeDirectory(
     public static INodeSection.INodeDirectory.Builder buildINodeDirectory(
-        INodeDirectoryAttributes dir, final StringMap stringMap) {
+        INodeDirectoryAttributes dir,
+        final SaverContext.DeduplicationMap<String> stringMap) {
       Quota.Counts quota = dir.getQuotaCounts();
       Quota.Counts quota = dir.getQuotaCounts();
       INodeSection.INodeDirectory.Builder b = INodeSection.INodeDirectory
       INodeSection.INodeDirectory.Builder b = INodeSection.INodeDirectory
           .newBuilder().setModificationTime(dir.getModificationTime())
           .newBuilder().setModificationTime(dir.getModificationTime())
@@ -416,7 +419,7 @@ public final class FSImageFormatPBINode {
 
 
     private void save(OutputStream out, INodeDirectory n) throws IOException {
     private void save(OutputStream out, INodeDirectory n) throws IOException {
       INodeSection.INodeDirectory.Builder b = buildINodeDirectory(n,
       INodeSection.INodeDirectory.Builder b = buildINodeDirectory(n,
-          parent.getStringMap());
+          parent.getSaverContext().getStringMap());
       INodeSection.INode r = buildINodeCommon(n)
       INodeSection.INode r = buildINodeCommon(n)
           .setType(INodeSection.INode.Type.DIRECTORY).setDirectory(b).build();
           .setType(INodeSection.INode.Type.DIRECTORY).setDirectory(b).build();
       r.writeDelimitedTo(out);
       r.writeDelimitedTo(out);
@@ -424,7 +427,7 @@ public final class FSImageFormatPBINode {
 
 
     private void save(OutputStream out, INodeFile n) throws IOException {
     private void save(OutputStream out, INodeFile n) throws IOException {
       INodeSection.INodeFile.Builder b = buildINodeFile(n,
       INodeSection.INodeFile.Builder b = buildINodeFile(n,
-          parent.getStringMap());
+          parent.getSaverContext().getStringMap());
 
 
       for (Block block : n.getBlocks()) {
       for (Block block : n.getBlocks()) {
         b.addBlocks(PBHelper.convert(block));
         b.addBlocks(PBHelper.convert(block));
@@ -447,7 +450,7 @@ public final class FSImageFormatPBINode {
     private void save(OutputStream out, INodeSymlink n) throws IOException {
     private void save(OutputStream out, INodeSymlink n) throws IOException {
       INodeSection.INodeSymlink.Builder b = INodeSection.INodeSymlink
       INodeSection.INodeSymlink.Builder b = INodeSection.INodeSymlink
           .newBuilder()
           .newBuilder()
-          .setPermission(buildPermissionStatus(n, parent.getStringMap()))
+          .setPermission(buildPermissionStatus(n, parent.getSaverContext().getStringMap()))
           .setTarget(ByteString.copyFrom(n.getSymlink()));
           .setTarget(ByteString.copyFrom(n.getSymlink()));
       INodeSection.INode r = buildINodeCommon(n)
       INodeSection.INode r = buildINodeCommon(n)
           .setType(INodeSection.INode.Type.SYMLINK).setSymlink(b).build();
           .setType(INodeSection.INode.Type.SYMLINK).setSymlink(b).build();

+ 60 - 42
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatProtobuf.java

@@ -73,12 +73,56 @@ import com.google.protobuf.CodedOutputStream;
 public final class FSImageFormatProtobuf {
 public final class FSImageFormatProtobuf {
   private static final Log LOG = LogFactory.getLog(FSImageFormatProtobuf.class);
   private static final Log LOG = LogFactory.getLog(FSImageFormatProtobuf.class);
 
 
+  public static final class LoaderContext {
+    private String[] stringTable;
+
+    public String[] getStringTable() {
+      return stringTable;
+    }
+  }
+
+  public static final class SaverContext {
+    public static class DeduplicationMap<E> {
+      private final Map<E, Integer> map = Maps.newHashMap();
+      private DeduplicationMap() {}
+
+      static <T> DeduplicationMap<T> newMap() {
+        return new DeduplicationMap<T>();
+      }
+
+      int getId(E value) {
+        if (value == null) {
+          return 0;
+        }
+        Integer v = map.get(value);
+        if (v == null) {
+          int nv = map.size() + 1;
+          map.put(value, nv);
+          return nv;
+        }
+        return v;
+      }
+
+      int size() {
+        return map.size();
+      }
+
+      Set<Entry<E, Integer>> entrySet() {
+        return map.entrySet();
+      }
+    }
+    private final DeduplicationMap<String> stringMap = DeduplicationMap.newMap();
+
+    public DeduplicationMap<String> getStringMap() {
+      return stringMap;
+    }
+  }
+
   public static final class Loader implements FSImageFormat.AbstractLoader {
   public static final class Loader implements FSImageFormat.AbstractLoader {
     static final int MINIMUM_FILE_LENGTH = 8;
     static final int MINIMUM_FILE_LENGTH = 8;
     private final Configuration conf;
     private final Configuration conf;
     private final FSNamesystem fsn;
     private final FSNamesystem fsn;
-
-    private String[] stringTable;
+    private final LoaderContext ctx;
 
 
     /** The MD5 sum of the loaded file */
     /** The MD5 sum of the loaded file */
     private MD5Hash imgDigest;
     private MD5Hash imgDigest;
@@ -88,6 +132,7 @@ public final class FSImageFormatProtobuf {
     Loader(Configuration conf, FSNamesystem fsn) {
     Loader(Configuration conf, FSNamesystem fsn) {
       this.conf = conf;
       this.conf = conf;
       this.fsn = fsn;
       this.fsn = fsn;
+      this.ctx = new LoaderContext();
     }
     }
 
 
     @Override
     @Override
@@ -100,8 +145,8 @@ public final class FSImageFormatProtobuf {
       return imgTxId;
       return imgTxId;
     }
     }
 
 
-    public String[] getStringTable() {
-      return stringTable;
+    public LoaderContext getLoaderContext() {
+      return ctx;
     }
     }
 
 
     void load(File file) throws IOException {
     void load(File file) throws IOException {
@@ -226,11 +271,11 @@ public final class FSImageFormatProtobuf {
 
 
     private void loadStringTableSection(InputStream in) throws IOException {
     private void loadStringTableSection(InputStream in) throws IOException {
       StringTableSection s = StringTableSection.parseDelimitedFrom(in);
       StringTableSection s = StringTableSection.parseDelimitedFrom(in);
-      stringTable = new String[s.getNumEntry() + 1];
+      ctx.stringTable = new String[s.getNumEntry() + 1];
       for (int i = 0; i < s.getNumEntry(); ++i) {
       for (int i = 0; i < s.getNumEntry(); ++i) {
         StringTableSection.Entry e = StringTableSection.Entry
         StringTableSection.Entry e = StringTableSection.Entry
             .parseDelimitedFrom(in);
             .parseDelimitedFrom(in);
-        stringTable[e.getId()] = e.getStr();
+        ctx.stringTable[e.getId()] = e.getStr();
       }
       }
     }
     }
 
 
@@ -269,9 +314,10 @@ public final class FSImageFormatProtobuf {
 
 
   public static final class Saver {
   public static final class Saver {
     private final SaveNamespaceContext context;
     private final SaveNamespaceContext context;
+    private final SaverContext saverContext;
+
     private long currentOffset = FSImageUtil.MAGIC_HEADER.length;
     private long currentOffset = FSImageUtil.MAGIC_HEADER.length;
     private MD5Hash savedDigest;
     private MD5Hash savedDigest;
-    private StringMap stringMap = new StringMap();
 
 
     private FileChannel fileChannel;
     private FileChannel fileChannel;
     // OutputStream for the section data
     // OutputStream for the section data
@@ -282,6 +328,7 @@ public final class FSImageFormatProtobuf {
 
 
     Saver(SaveNamespaceContext context) {
     Saver(SaveNamespaceContext context) {
       this.context = context;
       this.context = context;
+      this.saverContext = new SaverContext();
     }
     }
 
 
     public MD5Hash getSavedDigest() {
     public MD5Hash getSavedDigest() {
@@ -292,6 +339,10 @@ public final class FSImageFormatProtobuf {
       return context;
       return context;
     }
     }
 
 
+    public SaverContext getSaverContext() {
+      return saverContext;
+    }
+
     public void commitSection(FileSummary.Builder summary, SectionName name)
     public void commitSection(FileSummary.Builder summary, SectionName name)
         throws IOException {
         throws IOException {
       long oldOffset = currentOffset;
       long oldOffset = currentOffset;
@@ -465,48 +516,15 @@ public final class FSImageFormatProtobuf {
         throws IOException {
         throws IOException {
       OutputStream out = sectionOutputStream;
       OutputStream out = sectionOutputStream;
       StringTableSection.Builder b = StringTableSection.newBuilder()
       StringTableSection.Builder b = StringTableSection.newBuilder()
-          .setNumEntry(stringMap.size());
+          .setNumEntry(saverContext.stringMap.size());
       b.build().writeDelimitedTo(out);
       b.build().writeDelimitedTo(out);
-      for (Entry<String, Integer> e : stringMap.entrySet()) {
+      for (Entry<String, Integer> e : saverContext.stringMap.entrySet()) {
         StringTableSection.Entry.Builder eb = StringTableSection.Entry
         StringTableSection.Entry.Builder eb = StringTableSection.Entry
             .newBuilder().setId(e.getValue()).setStr(e.getKey());
             .newBuilder().setId(e.getValue()).setStr(e.getKey());
         eb.build().writeDelimitedTo(out);
         eb.build().writeDelimitedTo(out);
       }
       }
       commitSection(summary, SectionName.STRING_TABLE);
       commitSection(summary, SectionName.STRING_TABLE);
     }
     }
-
-    public StringMap getStringMap() {
-      return stringMap;
-    }
-  }
-
-  public static class StringMap {
-    private final Map<String, Integer> stringMap;
-
-    public StringMap() {
-      stringMap = Maps.newHashMap();
-    }
-
-    int getStringId(String str) {
-      if (str == null) {
-        return 0;
-      }
-      Integer v = stringMap.get(str);
-      if (v == null) {
-        int nv = stringMap.size() + 1;
-        stringMap.put(str, nv);
-        return nv;
-      }
-      return v;
-    }
-
-    int size() {
-      return stringMap.size();
-    }
-
-    Set<Entry<String, Integer>> entrySet() {
-      return stringMap.entrySet();
-    }
   }
   }
 
 
   /**
   /**

+ 9 - 7
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FSImageFormatPBSnapshot.java

@@ -115,7 +115,7 @@ public class FSImageFormatPBSnapshot {
         SnapshotSection.Snapshot pbs = SnapshotSection.Snapshot
         SnapshotSection.Snapshot pbs = SnapshotSection.Snapshot
             .parseDelimitedFrom(in);
             .parseDelimitedFrom(in);
         INodeDirectory root = loadINodeDirectory(pbs.getRoot(),
         INodeDirectory root = loadINodeDirectory(pbs.getRoot(),
-            parent.getStringTable());
+            parent.getLoaderContext().getStringTable());
         int sid = pbs.getSnapshotId();
         int sid = pbs.getSnapshotId();
         INodeDirectorySnapshottable parent = (INodeDirectorySnapshottable) fsDir
         INodeDirectorySnapshottable parent = (INodeDirectorySnapshottable) fsDir
             .getInode(root.getId()).asDirectory();
             .getInode(root.getId()).asDirectory();
@@ -162,7 +162,8 @@ public class FSImageFormatPBSnapshot {
         if (pbf.hasSnapshotCopy()) {
         if (pbf.hasSnapshotCopy()) {
           INodeSection.INodeFile fileInPb = pbf.getSnapshotCopy();
           INodeSection.INodeFile fileInPb = pbf.getSnapshotCopy();
           PermissionStatus permission = loadPermission(
           PermissionStatus permission = loadPermission(
-              fileInPb.getPermission(), parent.getStringTable());
+              fileInPb.getPermission(), parent.getLoaderContext()
+                  .getStringTable());
           copy = new INodeFileAttributes.SnapshotCopy(pbf.getName()
           copy = new INodeFileAttributes.SnapshotCopy(pbf.getName()
               .toByteArray(), permission, fileInPb.getModificationTime(),
               .toByteArray(), permission, fileInPb.getModificationTime(),
               fileInPb.getAccessTime(), (short) fileInPb.getReplication(),
               fileInPb.getAccessTime(), (short) fileInPb.getReplication(),
@@ -249,8 +250,9 @@ public class FSImageFormatPBSnapshot {
         }else if (diffInPb.hasSnapshotCopy()) {
         }else if (diffInPb.hasSnapshotCopy()) {
           INodeSection.INodeDirectory dirCopyInPb = diffInPb.getSnapshotCopy();
           INodeSection.INodeDirectory dirCopyInPb = diffInPb.getSnapshotCopy();
           final byte[] name = diffInPb.getName().toByteArray();
           final byte[] name = diffInPb.getName().toByteArray();
-          PermissionStatus permission = loadPermission(dirCopyInPb
-              .getPermission(), parent.getStringTable());
+          PermissionStatus permission = loadPermission(
+              dirCopyInPb.getPermission(), parent.getLoaderContext()
+                  .getStringTable());
           long modTime = dirCopyInPb.getModificationTime();
           long modTime = dirCopyInPb.getModificationTime();
           boolean noQuota = dirCopyInPb.getNsQuota() == -1
           boolean noQuota = dirCopyInPb.getNsQuota() == -1
               && dirCopyInPb.getDsQuota() == -1;
               && dirCopyInPb.getDsQuota() == -1;
@@ -311,7 +313,7 @@ public class FSImageFormatPBSnapshot {
           SnapshotSection.Snapshot.Builder sb = SnapshotSection.Snapshot
           SnapshotSection.Snapshot.Builder sb = SnapshotSection.Snapshot
               .newBuilder().setSnapshotId(s.getId());
               .newBuilder().setSnapshotId(s.getId());
           INodeSection.INodeDirectory.Builder db = buildINodeDirectory(sroot,
           INodeSection.INodeDirectory.Builder db = buildINodeDirectory(sroot,
-              parent.getStringMap());
+              parent.getSaverContext().getStringMap());
           INodeSection.INode r = INodeSection.INode.newBuilder()
           INodeSection.INode r = INodeSection.INode.newBuilder()
               .setId(sroot.getId())
               .setId(sroot.getId())
               .setType(INodeSection.INode.Type.DIRECTORY)
               .setType(INodeSection.INode.Type.DIRECTORY)
@@ -369,7 +371,7 @@ public class FSImageFormatPBSnapshot {
           INodeFileAttributes copy = diff.snapshotINode;
           INodeFileAttributes copy = diff.snapshotINode;
           if (copy != null) {
           if (copy != null) {
             fb.setName(ByteString.copyFrom(copy.getLocalNameBytes()))
             fb.setName(ByteString.copyFrom(copy.getLocalNameBytes()))
-                .setSnapshotCopy(buildINodeFile(copy, parent.getStringMap()));
+                .setSnapshotCopy(buildINodeFile(copy, parent.getSaverContext().getStringMap()));
           }
           }
           fb.build().writeDelimitedTo(out);
           fb.build().writeDelimitedTo(out);
         }
         }
@@ -410,7 +412,7 @@ public class FSImageFormatPBSnapshot {
           if (!diff.isSnapshotRoot() && copy != null) {
           if (!diff.isSnapshotRoot() && copy != null) {
             db.setName(ByteString.copyFrom(copy.getLocalNameBytes()))
             db.setName(ByteString.copyFrom(copy.getLocalNameBytes()))
                 .setSnapshotCopy(
                 .setSnapshotCopy(
-                    buildINodeDirectory(copy, parent.getStringMap()));
+                    buildINodeDirectory(copy, parent.getSaverContext().getStringMap()));
           }
           }
           // process created list and deleted list
           // process created list and deleted list
           List<INode> created = diff.getChildrenDiff()
           List<INode> created = diff.getChildrenDiff()

+ 36 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDeduplicationMap.java

@@ -0,0 +1,36 @@
+/**
+ * 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.server.namenode;
+
+import org.apache.hadoop.hdfs.server.namenode.FSImageFormatProtobuf.SaverContext.DeduplicationMap;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestDeduplicationMap {
+  @Test
+  public void testDeduplicationMap() {
+    DeduplicationMap<String> m = DeduplicationMap.newMap();
+    Assert.assertEquals(1, m.getId("1"));
+    Assert.assertEquals(2, m.getId("2"));
+    Assert.assertEquals(3, m.getId("3"));
+    Assert.assertEquals(1, m.getId("1"));
+    Assert.assertEquals(2, m.getId("2"));
+    Assert.assertEquals(3, m.getId("3"));
+  }
+}

+ 0 - 5
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImageStorageInspector.java

@@ -27,17 +27,12 @@ import static org.junit.Assert.assertTrue;
 import java.io.File;
 import java.io.File;
 import java.io.IOException;
 import java.io.IOException;
 
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
 import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
 import org.apache.hadoop.hdfs.server.namenode.FSImageStorageInspector.FSImageFile;
 import org.apache.hadoop.hdfs.server.namenode.FSImageStorageInspector.FSImageFile;
 import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType;
 import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType;
 import org.junit.Test;
 import org.junit.Test;
 
 
 public class TestFSImageStorageInspector {
 public class TestFSImageStorageInspector {
-  private static final Log LOG = LogFactory.getLog(
-      TestFSImageStorageInspector.class);
-
   /**
   /**
    * Simple test with image, edits, and inprogress edits
    * Simple test with image, edits, and inprogress edits
    */
    */