瀏覽代碼

Merge r1581915 from branch-2.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-2.4@1581918 13f79535-47bb-0310-9956-ffa450edef68
Suresh Srinivas 11 年之前
父節點
當前提交
9a3c0e389f

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

@@ -179,6 +179,9 @@ Release 2.4.0 - UNRELEASED
     HDFS-5910. Enhance DataTransferProtocol to allow per-connection choice
     of encryption/plain-text. (Benoy Antony via Arpit Agarwal)
 
+    HDFS-6150. Add inode id information in the logs to make debugging easier.
+    (suresh)
+
   OPTIMIZATIONS
 
     HDFS-5790. LeaseManager.findPath is very slow when many leases need recovery

+ 17 - 19
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

@@ -2261,8 +2261,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     final INodesInPath iip = dir.getINodesInPath4Write(src);
     final INode inode = iip.getLastINode();
     if (inode != null && inode.isDirectory()) {
-      throw new FileAlreadyExistsException("Cannot create file " + src
-          + "; already exists as a directory.");
+      throw new FileAlreadyExistsException(src +
+          " already exists as a directory");
     }
     final INodeFile myFile = INodeFile.valueOf(inode, src, true);
     if (isPermissionEnabled) {
@@ -2280,8 +2280,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     try {
       if (myFile == null) {
         if (!create) {
-          throw new FileNotFoundException("failed to overwrite non-existent file "
-            + src + " on client " + clientMachine);
+          throw new FileNotFoundException("Can't overwrite non-existent " +
+              src + " for client " + clientMachine);
         }
       } else {
         if (overwrite) {
@@ -2294,8 +2294,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
         } else {
           // If lease soft limit time is expired, recover the lease
           recoverLeaseInternal(myFile, src, holder, clientMachine, false);
-          throw new FileAlreadyExistsException("failed to create file " + src
-              + " on client " + clientMachine + " because the file exists");
+          throw new FileAlreadyExistsException(src + " for client " +
+              clientMachine + " already exists");
         }
       }
 
@@ -2306,8 +2306,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       INodeFile newNode = dir.addFile(src, permissions, replication, blockSize,
           holder, clientMachine, clientNode);
       if (newNode == null) {
-        throw new IOException("DIR* NameSystem.startFile: " +
-                              "Unable to add file to namespace.");
+        throw new IOException("Unable to add " + src +  " to namespace");
       }
       leaseManager.addLease(newNode.getFileUnderConstructionFeature()
           .getClientName(), src);
@@ -2315,12 +2314,12 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       // record file record in log, record new generation stamp
       getEditLog().logOpenFile(src, newNode, logRetryEntry);
       if (NameNode.stateChangeLog.isDebugEnabled()) {
-        NameNode.stateChangeLog.debug("DIR* NameSystem.startFile: "
-                                   +"add "+src+" to namespace for "+holder);
+        NameNode.stateChangeLog.debug("DIR* NameSystem.startFile: added " +
+            src + " inode " + newNode.getId() + " " + holder);
       }
     } catch (IOException ie) {
-      NameNode.stateChangeLog.warn("DIR* NameSystem.startFile: "
-                                   +ie.getMessage());
+      NameNode.stateChangeLog.warn("DIR* NameSystem.startFile: " + src + " " +
+          ie.getMessage());
       throw ie;
     }
   }
@@ -2360,7 +2359,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     try {
       if (inode == null) {
         throw new FileNotFoundException("failed to append to non-existent file "
-          + src + " on client " + clientMachine);
+          + src + " for client " + clientMachine);
       }
       INodeFile myFile = INodeFile.valueOf(inode, src, true);
       // Opening an existing file for write - may need to recover lease.
@@ -2484,7 +2483,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
             lease.getHolder().equals(holder)) { 
           throw new AlreadyBeingCreatedException(
             "failed to create file " + src + " for " + holder +
-            " on client " + clientMachine + 
+            " for client " + clientMachine +
             " because current leaseholder is trying to recreate file.");
         }
       }
@@ -2497,7 +2496,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       if (lease == null) {
         throw new AlreadyBeingCreatedException(
           "failed to create file " + src + " for " + holder +
-          " on client " + clientMachine + 
+          " for client " + clientMachine +
           " because pendingCreates is non-null but no leases found.");
       }
       if (force) {
@@ -2530,7 +2529,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
                 + src + "], " + "lease owner [" + lease.getHolder() + "]");
           } else {
             throw new AlreadyBeingCreatedException("Failed to create file ["
-                + src + "] for [" + holder + "] on client [" + clientMachine
+                + src + "] for [" + holder + "] for client [" + clientMachine
                 + "], because this file is already being created by ["
                 + clientName + "] on ["
                 + uc.getClientMachine() + "]");
@@ -2648,9 +2647,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     DatanodeDescriptor clientNode = null;
 
     if(NameNode.stateChangeLog.isDebugEnabled()) {
-      NameNode.stateChangeLog.debug(
-          "BLOCK* NameSystem.getAdditionalBlock: file "
-          +src+" for "+clientName);
+      NameNode.stateChangeLog.debug("BLOCK* NameSystem.getAdditionalBlock: "
+          + src + " inodeId " +  fileId  + " for " + clientName);
     }
 
     // Part I. Analyze the state of the file with respect to the input data.

+ 5 - 4
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCreation.java

@@ -53,6 +53,7 @@ import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileAlreadyExistsException;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FsServerDefaults;
 import org.apache.hadoop.fs.InvalidPathException;
@@ -246,11 +247,11 @@ public class TestFileCreation {
         fs.create(dir1, true); // Create path, overwrite=true
         fs.close();
         assertTrue("Did not prevent directory from being overwritten.", false);
-      } catch (IOException ie) {
-        if (!ie.getMessage().contains("already exists as a directory."))
-          throw ie;
+      } catch (FileAlreadyExistsException e) {
+        // expected
       }
-      
+
+      //
       // create a new file in home directory. Do not close it.
       //
       Path file1 = new Path("filestatus.dat");

+ 8 - 19
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLeaseRecovery2.java

@@ -198,33 +198,22 @@ public class TestLeaseRecovery2 {
   private void recoverLeaseUsingCreate(Path filepath)
   throws IOException, InterruptedException {
     FileSystem dfs2 = getFSAsAnotherUser(conf);
-
-    boolean done = false;
-    for(int i = 0; i < 10 && !done; i++) {
+    for(int i = 0; i < 10; i++) {
       AppendTestUtil.LOG.info("i=" + i);
       try {
         dfs2.create(filepath, false, BUF_SIZE, (short)1, BLOCK_SIZE);
         fail("Creation of an existing file should never succeed.");
-      } catch (IOException ioe) {
-        final String message = ioe.getMessage();
-        if (message.contains("file exists")) {
-          AppendTestUtil.LOG.info("done", ioe);
-          done = true;
-        }
-        else if (message.contains(AlreadyBeingCreatedException.class.getSimpleName())) {
-          AppendTestUtil.LOG.info("GOOD! got " + message);
-        }
-        else {
-          AppendTestUtil.LOG.warn("UNEXPECTED IOException", ioe);
-        }
-      }
-
-      if (!done) {
+      } catch(FileAlreadyExistsException e) {
+        return; // expected
+      } catch(AlreadyBeingCreatedException e) {
+        return; // expected
+      } catch(IOException ioe) {
+        AppendTestUtil.LOG.warn("UNEXPECTED ", ioe);
         AppendTestUtil.LOG.info("sleep " + 5000 + "ms");
         try {Thread.sleep(5000);} catch (InterruptedException e) {}
       }
     }
-    assertTrue(done);
+    fail("recoverLeaseUsingCreate failed");
   }
 
   private void verifyFile(FileSystem dfs, Path filepath, byte[] actual,