Browse Source

HDFS-737. Add full path name of the file to the block information and summary of total number of files, blocks, live and deadnodes to metasave output. Contributed by Jitendra Pandey.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/hdfs/trunk@831130 13f79535-47bb-0310-9956-ffa450edef68
Suresh Srinivas 15 years ago
parent
commit
d3d9b58d50

+ 4 - 0
CHANGES.txt

@@ -475,6 +475,10 @@ Release 0.20.2 - Unreleased
     HDFS-127. Reset failure count in DFSClient for each block acquiring
     HDFS-127. Reset failure count in DFSClient for each block acquiring
     operation.  (Igor Bolotin via szetszwo)
     operation.  (Igor Bolotin via szetszwo)
 
 
+    HDFS-737. Add full path name of the file to the block information and 
+    summary of total number of files, blocks, live and deadnodes to 
+    metasave output. (Jitendra Pandey via suresh)
+    
   BUG FIXES
   BUG FIXES
 
 
     HDFS-686. NullPointerException is thrown while merging edit log and image.
     HDFS-686. NullPointerException is thrown while merging edit log and image.

+ 5 - 0
src/java/org/apache/hadoop/hdfs/server/namenode/BlockManager.java

@@ -193,6 +193,11 @@ public class BlockManager {
         chooseSourceDatanode(block, containingNodes, numReplicas);
         chooseSourceDatanode(block, containingNodes, numReplicas);
         int usableReplicas = numReplicas.liveReplicas() +
         int usableReplicas = numReplicas.liveReplicas() +
                              numReplicas.decommissionedReplicas();
                              numReplicas.decommissionedReplicas();
+       
+        if (block instanceof BlockInfo) {
+          String fileName = ((BlockInfo)block).getINode().getFullPathName();
+          out.print(fileName + ": ");
+        }
         // l: == live:, d: == decommissioned c: == corrupt e: == excess
         // l: == live:, d: == decommissioned c: == corrupt e: == excess
         out.print(block + ((usableReplicas > 0)? "" : " MISSING") + 
         out.print(block + ((usableReplicas > 0)? "" : " MISSING") + 
                   " (replicas:" +
                   " (replicas:" +

+ 15 - 4
src/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

@@ -506,11 +506,22 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
    */
    */
   synchronized void metaSave(String filename) throws IOException {
   synchronized void metaSave(String filename) throws IOException {
     checkSuperuserPrivilege();
     checkSuperuserPrivilege();
-    File file = new File(System.getProperty("hadoop.log.dir"), 
-                         filename);
-    PrintWriter out = new PrintWriter(new BufferedWriter(
-                                                         new FileWriter(file, true)));
+    File file = new File(System.getProperty("hadoop.log.dir"), filename);
+    PrintWriter out = new PrintWriter(new BufferedWriter(new FileWriter(file,
+        true)));
 
 
+    long totalInodes = this.dir.totalInodes();
+    long totalBlocks = this.getBlocksTotal();
+
+    ArrayList<DatanodeDescriptor> live = new ArrayList<DatanodeDescriptor>();
+    ArrayList<DatanodeDescriptor> dead = new ArrayList<DatanodeDescriptor>();
+    this.DFSNodesStatus(live, dead);
+    
+    String str = totalInodes + " files and directories, " + totalBlocks
+        + " blocks = " + (totalInodes + totalBlocks) + " total";
+    out.println(str);
+    out.println("Live Datanodes: "+live.size());
+    out.println("Dead Datanodes: "+dead.size());
     blockManager.metaSave(out);
     blockManager.metaSave(out);
 
 
     //
     //

+ 120 - 0
src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestMetaSave.java

@@ -0,0 +1,120 @@
+/**
+ * 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.junit.Test;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import java.io.BufferedReader;
+import java.io.FileInputStream;
+import java.io.DataInputStream;
+import java.io.InputStreamReader;
+import java.io.IOException;
+import java.lang.InterruptedException;
+import java.util.Random;
+import static org.junit.Assert.assertTrue;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+
+import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+
+/**
+ * This class tests the creation and validation of metasave
+ */
+public class TestMetaSave {
+  static final int NUM_DATA_NODES = 2;
+  static final long seed = 0xDEADBEEFL;
+  static final int blockSize = 8192;
+  private static MiniDFSCluster cluster = null;
+  private static FileSystem fileSys = null;
+
+  private void createFile(FileSystem fileSys, Path name) throws IOException {
+    FSDataOutputStream stm = fileSys.create(name, true, fileSys.getConf()
+        .getInt("io.file.buffer.size", 4096), (short) 2, (long) blockSize);
+    byte[] buffer = new byte[1024];
+    Random rand = new Random(seed);
+    rand.nextBytes(buffer);
+    stm.write(buffer);
+    stm.close();
+  }
+
+  @BeforeClass
+  public static void setUp() throws IOException {
+    // start a cluster
+    Configuration conf = new HdfsConfiguration();
+
+    // High value of replication interval
+    // so that blocks remain under-replicated
+    conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_KEY, 1000);
+    conf.setLong(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1L);
+    conf.setLong(DFSConfigKeys.DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY, 1L);
+    cluster = new MiniDFSCluster(conf, NUM_DATA_NODES, true, null);
+    cluster.waitActive();
+    fileSys = cluster.getFileSystem();
+  }
+
+  /**
+   * Tests metasave
+   */
+  @Test
+  public void testMetaSave() throws IOException, InterruptedException {
+
+    final FSNamesystem namesystem = cluster.getNamesystem();
+
+    for (int i = 0; i < 2; i++) {
+      Path file = new Path("/filestatus" + i);
+      createFile(fileSys, file);
+    }
+
+    cluster.stopDataNode(1);
+    // wait for namenode to discover that a datanode is dead
+    Thread.sleep(15000);
+    namesystem.setReplication("/filestatus0", (short) 4);
+
+    namesystem.metaSave("metasave.out.txt");
+
+    // Verification
+    String logFile = System.getProperty("hadoop.log.dir") + "/"
+        + "metasave.out.txt";
+    FileInputStream fstream = new FileInputStream(logFile);
+    DataInputStream in = new DataInputStream(fstream);
+    BufferedReader reader = new BufferedReader(new InputStreamReader(in));
+    String line = reader.readLine();
+    assertTrue(line.equals("3 files and directories, 2 blocks = 5 total"));
+    line = reader.readLine();
+    assertTrue(line.equals("Live Datanodes: 1"));
+    line = reader.readLine();
+    assertTrue(line.equals("Dead Datanodes: 1"));
+    line = reader.readLine();
+    line = reader.readLine();
+    assertTrue(line.matches("^/filestatus[01]:.*"));
+  }
+
+  @AfterClass
+  public static void tearDown() throws IOException {
+    if (fileSys != null)
+      fileSys.close();
+    if (cluster != null)
+      cluster.shutdown();
+  }
+}