소스 검색

HDFS-5196. Merge r1581427 from trunk.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-2@1581429 13f79535-47bb-0310-9956-ffa450edef68
Haohui Mai 11 년 전
부모
커밋
d3b471155b

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

@@ -13,6 +13,9 @@ Release 2.5.0 - UNRELEASED
 
     HDFS-6125. Cleanup unnecessary cast in HDFS code base. (suresh)
 
+    HDFS-5196. Provide more snapshot information in WebUI.
+    (Shinichi Yamashita via wheat9)
+
   OPTIMIZATIONS
 
   BUG FIXES 

+ 25 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/SnapshotInfo.java

@@ -20,6 +20,7 @@ package org.apache.hadoop.hdfs.protocol;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.FsPermissionProto;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
 
 /**
  * SnapshotInfo maintains information for a snapshot
@@ -79,4 +80,28 @@ public class SnapshotInfo {
         + "; group=" + group
         + "}";
   }
+
+  public static class Bean {
+    private final String snapshotID;
+    private final String snapshotDirectory;
+    private final long modificationTime;
+
+    public Bean(Snapshot s) {
+      this.snapshotID = s.getRoot().getLocalName();
+      this.snapshotDirectory = s.getRoot().getFullPathName();
+      this.modificationTime = s.getRoot().getModificationTime();
+    }
+
+    public String getSnapshotID() {
+      return snapshotID;
+    }
+
+    public String getSnapshotDirectory() {
+      return snapshotDirectory;
+    }
+
+    public long getModificationTime() {
+      return modificationTime;
+    }
+  }
 }

+ 50 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/SnapshottableDirectoryStatus.java

@@ -166,4 +166,54 @@ public class SnapshottableDirectoryStatus {
   private static int maxLength(int n, Object value) {
     return Math.max(n, String.valueOf(value).length());
   }
+
+  public static class Bean {
+    private final String path;
+    private final int snapshotNumber;
+    private final int snapshotQuota;
+    private final long modificationTime;
+    private final short permission;
+    private final String owner;
+    private final String group;
+
+    public Bean(SnapshottableDirectoryStatus s) {
+      this.path = s.getFullPath().toString();
+      this.snapshotNumber = s.getSnapshotNumber();
+      this.snapshotQuota = s.getSnapshotQuota();
+      this.modificationTime = s.getDirStatus().getModificationTime();
+      this.permission =
+          Short.valueOf(Integer.toOctalString(
+              s.getDirStatus().getPermission().toShort()));
+      this.owner = s.getDirStatus().getOwner();
+      this.group = s.getDirStatus().getGroup();
+    }
+
+    public String getPath() {
+      return path;
+    }
+
+    public int getSnapshotNumber() {
+      return snapshotNumber;
+    }
+
+    public int getSnapshotQuota() {
+      return snapshotQuota;
+    }
+
+    public long getModificationTime() {
+      return modificationTime;
+    }
+
+    public short getPermission() {
+      return permission;
+    }
+
+    public String getOwner() {
+      return owner;
+    }
+
+    public String getGroup() {
+      return group;
+    }
+  }
 }

+ 4 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

@@ -960,6 +960,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     
     registerMXBean();
     DefaultMetricsSystem.instance().register(this);
+    snapshotManager.registerMXBean();
   }
   
   /** 
@@ -5631,6 +5632,9 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
    * shutdown FSNamesystem
    */
   void shutdown() {
+    if (snapshotManager != null) {
+      snapshotManager.shutdown();
+    }
     if (mbeanName != null) {
       MBeans.unregister(mbeanName);
       mbeanName = null;

+ 58 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotManager.java

@@ -26,8 +26,11 @@ import java.util.List;
 import java.util.Map;
 import java.util.concurrent.atomic.AtomicInteger;
 
+import javax.management.ObjectName;
+
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.protocol.SnapshotException;
+import org.apache.hadoop.hdfs.protocol.SnapshotInfo;
 import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
 import org.apache.hadoop.hdfs.server.namenode.FSDirectory;
 import org.apache.hadoop.hdfs.server.namenode.FSImageFormat;
@@ -37,6 +40,7 @@ import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo;
 import org.apache.hadoop.hdfs.server.namenode.INodeDirectory;
 import org.apache.hadoop.hdfs.server.namenode.INodesInPath;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectorySnapshottable.SnapshotDiffInfo;
+import org.apache.hadoop.metrics2.util.MBeans;
 
 /**
  * Manage snapshottable directories and their snapshots.
@@ -50,7 +54,7 @@ import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectorySnapshottab
  * 2. Lock the {@link FSDirectory} lock for the {@link SnapshotManager} methods
  * if necessary.
  */
-public class SnapshotManager implements SnapshotStats {
+public class SnapshotManager implements SnapshotStatsMXBean {
   private boolean allowNestedSnapshots = false;
   private final FSDirectory fsdir;
   private static final int SNAPSHOT_ID_BIT_WIDTH = 24;
@@ -259,12 +263,10 @@ public class SnapshotManager implements SnapshotStats {
     srcRoot.renameSnapshot(path, oldSnapshotName, newSnapshotName);
   }
   
-  @Override
   public int getNumSnapshottableDirs() {
     return snapshottables.size();
   }
 
-  @Override
   public int getNumSnapshots() {
     return numSnapshots.get();
   }
@@ -372,4 +374,57 @@ public class SnapshotManager implements SnapshotStats {
    public int getMaxSnapshotID() {
     return ((1 << SNAPSHOT_ID_BIT_WIDTH) - 1);
   }
+
+  private ObjectName mxBeanName;
+
+  public void registerMXBean() {
+    mxBeanName = MBeans.register("NameNode", "SnapshotInfo", this);
+  }
+
+  public void shutdown() {
+    MBeans.unregister(mxBeanName);
+    mxBeanName = null;
+  }
+
+  @Override // SnapshotStatsMXBean
+  public SnapshotDirectoryMXBean getSnapshotStats() {
+    SnapshottableDirectoryStatus[] stats = getSnapshottableDirListing(null);
+    if (stats == null) {
+      return null;
+    }
+    return new SnapshotDirectoryMXBean(stats);
+  }
+
+  public class SnapshotDirectoryMXBean {
+    private List<SnapshottableDirectoryStatus.Bean> directory =
+        new ArrayList<SnapshottableDirectoryStatus.Bean>();
+    private List<SnapshotInfo.Bean> snapshots =
+        new ArrayList<SnapshotInfo.Bean>();
+
+    public SnapshotDirectoryMXBean(SnapshottableDirectoryStatus[] stats) {
+      set(stats);
+    }
+
+    public void set(SnapshottableDirectoryStatus[] stats) {
+      for (SnapshottableDirectoryStatus s : stats) {
+        directory.add(new SnapshottableDirectoryStatus.Bean(s));
+        try {
+          for (Snapshot shot : getSnapshottableRoot(
+              s.getFullPath().toString()).getSnapshotList()) {
+            snapshots.add(new SnapshotInfo.Bean(shot));
+          }
+        } catch (IOException e) {
+          continue;
+        }
+      }
+    }
+
+    public List<SnapshottableDirectoryStatus.Bean> getDirectory() {
+      return directory;
+    }
+
+    public List<SnapshotInfo.Bean> getSnapshots() {
+      return snapshots;
+    }
+  }
 }

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

@@ -17,20 +17,18 @@
  */
 package org.apache.hadoop.hdfs.server.namenode.snapshot;
 
+import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotManager.SnapshotDirectoryMXBean;
+
 /**
  * This is an interface used to retrieve statistic information related to
  * snapshots
  */
-public interface SnapshotStats {
+public interface SnapshotStatsMXBean {
 
   /**
-   * @return The number of snapshottale directories in the system 
-   */
-  public int getNumSnapshottableDirs();
-  
-  /**
-   * @return The number of directories that have been snapshotted
+   * Return the list of snapshottable directories
+   *
+   * @return the list of snapshottable directories
    */
-  public int getNumSnapshots();
-  
+  public SnapshotDirectoryMXBean getSnapshotStats();
 }

+ 26 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfs-dust.js

@@ -55,6 +55,32 @@
       }
 
       return res;
+    },
+
+    'date_tostring' : function (v) {
+      return new Date(Number(v)).toLocaleString();
+    },
+
+    'helper_to_permission': function (v) {
+      var symbols = [ '---', '--x', '-w-', '-wx', 'r--', 'r-x', 'rw-', 'rwx' ];
+      var sticky = v > 1000;
+
+      var res = "";
+      for (var i = 0; i < 3; ++i) {
+        res = symbols[(v % 10)] + res;
+        v = Math.floor(v / 10);
+      }
+
+      if (sticky) {
+        var otherExec = ((v % 10) & 1) == 1;
+        res = res.substr(0, res.length - 1) + (otherExec ? 't' : 'T');
+      }
+
+      return res;
+    },
+
+    'helper_to_directory' : function (v) {
+      return v === 'DIRECTORY' ? 'd' : '-';
     }
   };
   $.extend(dust.filters, filters);

+ 48 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.html

@@ -189,8 +189,54 @@
 
 <script type="text/x-dust-template" id="tmpl-snapshot">
 <div class="page-header"><h1>Snapshot Summary</h1></div>
-<p><b>Snapshottable directories</b>: {SnapshottableDirectories}</p>
-<p><b>Snapshotted directories</b>: {Snapshots}</p>
+<div class="page-header"><h1><small>Snapshottable directories: {@size key=SnapshotStats.directory}{/size}</small></div>
+<small>
+<table class="table">
+  <thead>
+    <tr>
+      <th>Path</th>
+      <th>Snapshot Number</th>
+      <th>Snapshot Quota</th>
+      <th>Modification Time</th>
+      <th>Permission</th>
+      <th>Owner</th>
+      <th>Group</th>
+    </tr>
+  </thead>
+  {#SnapshotStats.directory}
+  <tr>
+    <td>{path}</td>
+    <td>{snapshotNumber}</td>
+    <td>{snapshotQuota}</td>
+    <td>{modificationTime|date_tostring}</td>
+    <td>{permission|helper_to_permission}</td>
+    <td>{owner}</td>
+    <td>{group}</td>
+  </tr>
+  {/SnapshotStats.directory}
+</table>
+</small>
+
+<div class="page-header"><h1><small>Snapshotted directories: {@size key=SnapshotStats.snapshots}{/size}</small></div>
+
+<small>
+<table class="table">
+  <thead>
+    <tr>
+      <th>Snapshot ID</th>
+      <th>Snapshot Directory</th>
+      <th>Modification Time</th>
+    </tr>
+  </thead>
+  {#SnapshotStats.snapshots}
+  <tr>
+    <td>{snapshotID}</td>
+    <td>{snapshotDirectory}</td>
+    <td>{modificationTime|date_tostring}</td>
+  </tr>
+  {/SnapshotStats.snapshots}
+</table>
+</small>
 </script>
 
 <script type="text/x-dust-template" id="tmpl-datanode">

+ 2 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.js

@@ -169,10 +169,9 @@
 
   function load_snapshot_info() {
     $.get(
-      '/jmx?qry=Hadoop:service=NameNode,name=FSNamesystemState',
+      '/jmx?qry=Hadoop:service=NameNode,name=SnapshotInfo',
       guard_with_startup_progress(function (resp) {
-        var data = JSON.parse(resp.beans[0].SnapshotStats);
-        dust.render('snapshot-info', data, function(err, out) {
+      dust.render('snapshot-info', resp.beans[0], function(err, out) {
           $('#tab-snapshot').html(out);
           $('#ui-tabs a[href="#tab-snapshot"]').tab('show');
         });

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.html

@@ -90,7 +90,7 @@
         <tbody>
           {#FileStatus}
           <tr>
-            <td>{#helper_to_permission/}</td>
+            <td>{type|helper_to_directory}{permission|helper_to_permission}</td>
             <td>{owner}</td>
             <td>{group}</td>
             <td>{length|fmt_bytes}</td>

+ 1 - 24
hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.js

@@ -21,30 +21,7 @@
   // The chunk size of tailing the files, i.e., how many bytes will be shown
   // in the preview.
   var TAIL_CHUNK_SIZE = 32768;
-  var helpers = {
-    'helper_to_permission': function(chunk, ctx, bodies, params) {
-      var p = ctx.current().permission;
-      var dir = ctx.current().type == 'DIRECTORY' ? 'd' : '-';
-      var symbols = [ '---', '--x', '-w-', '-wx', 'r--', 'r-x', 'rw-', 'rwx' ];
-      var sticky = p > 1000;
-
-      var res = "";
-      for (var i = 0; i < 3; ++i) {
-	res = symbols[(p % 10)] + res;
-	p = Math.floor(p / 10);
-      }
-
-      if (sticky) {
-        var otherExec = ((ctx.current().permission % 10) & 1) == 1;
-        res = res.substr(0, res.length - 1) + (otherExec ? 't' : 'T');
-      }
-
-      chunk.write(dir + res);
-      return chunk;
-    }
-  };
 
-  var base = dust.makeBase(helpers);
   var current_directory = "";
 
   function show_err_msg(msg) {
@@ -171,7 +148,7 @@
       current_directory = dir;
       $('#directory').val(dir);
       window.location.hash = dir;
-      dust.render('explorer', base.push(d), function(err, out) {
+      dust.render('explorer', d, function(err, out) {
         $('#panel').html(out);
 
         $('.explorer-browse-links').click(function() {

+ 79 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotStatsMXBean.java

@@ -0,0 +1,79 @@
+/**
+ * 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.snapshot;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.lang.management.ManagementFactory;
+import java.lang.reflect.Array;
+
+import javax.management.MBeanServer;
+import javax.management.ObjectName;
+import javax.management.openmbean.CompositeData;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.junit.Test;
+
+public class TestSnapshotStatsMXBean {
+
+  /**
+   * Test getting SnapshotStatsMXBean information
+   */
+  @Test
+  public void testSnapshotStatsMXBeanInfo() throws Exception {
+    Configuration conf = new Configuration();
+    MiniDFSCluster cluster = null;
+    String pathName = "/snapshot";
+    Path path = new Path(pathName);
+
+    try {
+      cluster = new MiniDFSCluster.Builder(conf).build();
+      cluster.waitActive();
+
+      SnapshotManager sm = cluster.getNamesystem().getSnapshotManager();
+      DistributedFileSystem dfs = (DistributedFileSystem) cluster.getFileSystem();
+      dfs.mkdirs(path);
+      dfs.allowSnapshot(path);
+      dfs.createSnapshot(path);
+
+      MBeanServer mbs = ManagementFactory.getPlatformMBeanServer();
+      ObjectName mxbeanName = new ObjectName(
+          "Hadoop:service=NameNode,name=SnapshotInfo");
+
+      CompositeData statsbean =
+          (CompositeData) mbs.getAttribute(mxbeanName, "SnapshotStats");
+      int numDirectories = Array.getLength(statsbean.get("directory"));
+      assertEquals(sm.getNumSnapshottableDirs(), numDirectories);
+      int numSnapshots = Array.getLength(statsbean.get("snapshots"));
+      assertEquals(sm.getNumSnapshots(), numSnapshots);
+
+      CompositeData directory = (CompositeData) Array.get(statsbean.get("directory"), 0);
+      CompositeData snapshots = (CompositeData) Array.get(statsbean.get("snapshots"), 0);
+      assertTrue(((String) directory.get("path")).contains(pathName));
+      assertTrue(((String) snapshots.get("snapshotDirectory")).contains(pathName));
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
+}