浏览代码

HDFS-204. Add a new metrics FilesInGetListingOps to the Namenode. Contributed by Jitendra Nath Pandey

git-svn-id: https://svn.apache.org/repos/asf/hadoop/hdfs/trunk@790733 13f79535-47bb-0310-9956-ffa450edef68
Tsz-wo Sze 16 年之前
父节点
当前提交
d2ca899d5e

+ 3 - 0
CHANGES.txt

@@ -25,6 +25,9 @@ Trunk (unreleased changes)
     HADOOP-6106. Updated hadoop-core and test jars from hudson trunk 
     build #12. (Giridharan Kesavan)
 
+    HDFS-204. Add a new metrics FilesInGetListingOps to the Namenode.
+    (Jitendra Nath Pandey via szetszwo)
+
   BUG FIXES
     HDFS-76. Better error message to users when commands fail because of 
     lack of quota. Allow quota to be set even if the limit is lower than

+ 1 - 0
src/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java

@@ -744,6 +744,7 @@ public class NameNode implements ClientProtocol, DatanodeProtocol,
     FileStatus[] files = namesystem.getListing(src);
     if (files != null) {
       myMetrics.numGetListingOps.inc();
+      myMetrics.numFilesInGetListingOps.inc(files.length);
     }
     return files;
   }

+ 2 - 0
src/java/org/apache/hadoop/hdfs/server/namenode/metrics/NameNodeMetrics.java

@@ -81,6 +81,8 @@ public class NameNodeMetrics implements Updater {
                     new MetricsIntValue("fsImageLoadTime", registry, "Time loading FS Image at Startup");
     public MetricsIntValue numBlocksCorrupted =
                     new MetricsIntValue("BlocksCorrupted", registry);
+    public MetricsTimeVaryingInt numFilesInGetListingOps = 
+                    new MetricsTimeVaryingInt("FilesInGetListingOps", registry);
 
       
     public NameNodeMetrics(Configuration conf, NamenodeRole nameNodeRole) {

+ 81 - 0
src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/metrics/TestNNMetricFilesInGetListingOps.java

@@ -0,0 +1,81 @@
+/**
+ * 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.metrics;
+
+import java.io.IOException;
+import java.util.Random;
+
+import junit.framework.TestCase;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DFSTestUtil;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.server.namenode.NameNode;
+
+/**
+ * Test case for FilesInGetListingOps metric in Namenode
+ */
+public class TestNNMetricFilesInGetListingOps extends TestCase {
+  private static final Configuration CONF = new Configuration();
+  static {
+    CONF.setLong("dfs.block.size", 100);
+    CONF.setInt("io.bytes.per.checksum", 1);
+    CONF.setLong("dfs.heartbeat.interval", 1L);
+    CONF.setInt("dfs.replication.interval", 1);
+  }
+     
+  private MiniDFSCluster cluster;
+  private NameNodeMetrics nnMetrics;
+  private DistributedFileSystem fs;
+  private Random rand = new Random();
+
+  @Override
+  protected void setUp() throws Exception {
+    cluster = new MiniDFSCluster(CONF, 1, true, null);
+    cluster.waitActive();
+    cluster.getNameNode();
+    nnMetrics = NameNode.getNameNodeMetrics();
+    fs = (DistributedFileSystem) cluster.getFileSystem();
+  }
+
+  @Override
+  protected void tearDown() throws Exception {
+    cluster.shutdown();
+  }
+
+  /** create a file with a length of <code>fileLen</code> */
+  private void createFile(String fileName, long fileLen, short replicas) throws IOException {
+    Path filePath = new Path(fileName);
+    DFSTestUtil.createFile(fs, filePath, fileLen, replicas, rand.nextLong());
+  }
+     
+
+  public void testFilesInGetListingOps() throws Exception {
+    createFile("/tmp1/t1", 3200, (short)3);
+    createFile("/tmp1/t2", 3200, (short)3);
+    createFile("/tmp2/t1", 3200, (short)3);
+    createFile("/tmp2/t2", 3200, (short)3);
+    cluster.getNameNode().getListing("/tmp1") ;
+    assertEquals(2,nnMetrics.numFilesInGetListingOps.getCurrentIntervalValue());
+    cluster.getNameNode().getListing("/tmp2") ;
+    assertEquals(4,nnMetrics.numFilesInGetListingOps.getCurrentIntervalValue());
+  }
+}
+