瀏覽代碼

HDFS-6050. Merging change r1581059 from branch-2

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-2.4@1581064 13f79535-47bb-0310-9956-ffa450edef68
Brandon Li 11 年之前
父節點
當前提交
8180cb47e9

+ 3 - 4
hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/IdUserGroup.java

@@ -166,8 +166,7 @@ public class IdUserGroup {
         try {
           br.close();
         } catch (IOException e1) {
-          LOG.error("Can't close BufferedReader of command result");
-          e1.printStackTrace();
+          LOG.error("Can't close BufferedReader of command result", e1);
         }
       }
     }
@@ -249,7 +248,7 @@ public class IdUserGroup {
       uid = getUid(user);
     } catch (IOException e) {
       uid = user.hashCode();
-      LOG.info("Can't map user " + user + ". Use its string hashcode:" + uid);
+      LOG.info("Can't map user " + user + ". Use its string hashcode:" + uid, e);
     }
     return uid;
   }
@@ -262,7 +261,7 @@ public class IdUserGroup {
       gid = getGid(group);
     } catch (IOException e) {
       gid = group.hashCode();
-      LOG.debug("Can't map group " + group + ". Use its string hashcode:" + gid);
+      LOG.info("Can't map group " + group + ". Use its string hashcode:" + gid, e);
     }
     return gid;
   }

+ 9 - 13
hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/Nfs3FileAttributes.java

@@ -35,7 +35,7 @@ public class Nfs3FileAttributes {
   private long used;
   private Specdata3 rdev;
   private long fsid;
-  private long fileid;
+  private long fileId;
   private NfsTime atime;
   private NfsTime mtime;
   private NfsTime ctime;
@@ -76,7 +76,7 @@ public class Nfs3FileAttributes {
   }
 
   public Nfs3FileAttributes(NfsFileType nfsType, int nlink, short mode, int uid,
-      int gid, long size, long fsid, long fileid, long mtime, long atime) {
+      int gid, long size, long fsid, long fileId, long mtime, long atime) {
     this.type = nfsType.toValue();
     this.mode = mode;
     this.nlink = (type == NfsFileType.NFSDIR.toValue()) ? (nlink + 2) : 1;
@@ -89,7 +89,7 @@ public class Nfs3FileAttributes {
     this.used = this.size;
     this.rdev = new Specdata3();
     this.fsid = fsid;
-    this.fileid = fileid;
+    this.fileId = fileId;
     this.mtime = new NfsTime(mtime);
     this.atime = atime != 0 ? new NfsTime(atime) : this.mtime;
     this.ctime = this.mtime;
@@ -105,7 +105,7 @@ public class Nfs3FileAttributes {
     this.used = other.getUsed();
     this.rdev = new Specdata3();
     this.fsid = other.getFsid();
-    this.fileid = other.getFileid();
+    this.fileId = other.getFileId();
     this.mtime = new NfsTime(other.getMtime());
     this.atime = new NfsTime(other.getAtime());
     this.ctime = new NfsTime(other.getCtime());
@@ -122,7 +122,7 @@ public class Nfs3FileAttributes {
     xdr.writeInt(rdev.getSpecdata1());
     xdr.writeInt(rdev.getSpecdata2());
     xdr.writeLongAsHyper(fsid);
-    xdr.writeLongAsHyper(fileid);
+    xdr.writeLongAsHyper(fileId);
     atime.serialize(xdr);
     mtime.serialize(xdr);
     ctime.serialize(xdr);
@@ -142,7 +142,7 @@ public class Nfs3FileAttributes {
     xdr.readInt();
     attr.rdev = new Specdata3();
     attr.fsid = xdr.readHyper();
-    attr.fileid = xdr.readHyper();
+    attr.fileId = xdr.readHyper();
     attr.atime = NfsTime.deserialize(xdr);
     attr.mtime = NfsTime.deserialize(xdr);
     attr.ctime = NfsTime.deserialize(xdr);
@@ -154,7 +154,7 @@ public class Nfs3FileAttributes {
     return String.format("type:%d, mode:%d, nlink:%d, uid:%d, gid:%d, " + 
             "size:%d, used:%d, rdev:%s, fsid:%d, fileid:%d, atime:%s, " + 
             "mtime:%s, ctime:%s",
-            type, mode, nlink, uid, gid, size, used, rdev, fsid, fileid, atime,
+            type, mode, nlink, uid, gid, size, used, rdev, fsid, fileId, atime,
             mtime, ctime);
   }
 
@@ -170,8 +170,8 @@ public class Nfs3FileAttributes {
     return fsid;
   }
 
-  public long getFileid() {
-    return fileid;
+  public long getFileId() {
+    return fileId;
   }
 
   public NfsTime getAtime() {
@@ -194,10 +194,6 @@ public class Nfs3FileAttributes {
     return new WccAttr(size, mtime, ctime);
   }
   
-  public long getFileId() {
-    return fileid;
-  }
-  
   public long getSize() {
     return size;
   }

+ 1 - 1
hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/portmap/Portmap.java

@@ -65,7 +65,7 @@ final class Portmap {
       pm.start(DEFAULT_IDLE_TIME_MILLISECONDS,
           new InetSocketAddress(port), new InetSocketAddress(port));
     } catch (Throwable e) {
-      LOG.fatal("Failed to start the server. Cause:" + e.getMessage());
+      LOG.fatal("Failed to start the server. Cause:", e);
       pm.shutdown();
       System.exit(-1);
     }

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/mount/RpcProgramMountd.java

@@ -131,7 +131,7 @@ public class RpcProgramMountd extends RpcProgram implements MountInterface {
       
       handle = new FileHandle(exFileStatus.getFileId());
     } catch (IOException e) {
-      LOG.error("Can't get handle for export:" + path + ", exception:" + e);
+      LOG.error("Can't get handle for export:" + path, e);
       MountResponse.writeMNTResponse(Nfs3Status.NFS3ERR_NOENT, out, xid, null);
       return out;
     }

+ 1 - 2
hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/AsyncDataService.java

@@ -135,8 +135,7 @@ public class AsyncDataService {
       try {
         openFileCtx.executeWriteBack();
       } catch (Throwable t) {
-        LOG.error("Asyn data service got error:"
-            + ExceptionUtils.getFullStackTrace(t));
+        LOG.error("Asyn data service got error:", t);
       }
     }
   }

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/Nfs3Utils.java

@@ -94,7 +94,8 @@ public class Nfs3Utils {
   }
 
   public static WccAttr getWccAttr(Nfs3FileAttributes attr) {
-    return new WccAttr(attr.getSize(), attr.getMtime(), attr.getCtime());
+    return attr == null ? new WccAttr() : new WccAttr(attr.getSize(),
+        attr.getMtime(), attr.getCtime());
   }
 
   // TODO: maybe not efficient

+ 15 - 15
hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/OpenFileCtx.java

@@ -365,7 +365,7 @@ class OpenFileCtx {
           }
         } catch (Throwable t) {
           LOG.info("Dumper get Throwable: " + t + ". dumpFilePath: "
-              + OpenFileCtx.this.dumpFilePath);
+              + OpenFileCtx.this.dumpFilePath, t);
         }
       }
     }
@@ -660,8 +660,8 @@ class OpenFileCtx {
       // Sync file data and length to avoid partial read failure
       fos.hsync(EnumSet.of(SyncFlag.UPDATE_LENGTH));
     } catch (ClosedChannelException closedException) {
-      LOG.info("The FSDataOutputStream has been closed. " +
-      		"Continue processing the perfect overwrite.");
+      LOG.info("The FSDataOutputStream has been closed. "
+          + "Continue processing the perfect overwrite.");
     } catch (IOException e) {
       LOG.info("hsync failed when processing possible perfect overwrite, path="
           + path + " error:" + e);
@@ -680,7 +680,7 @@ class OpenFileCtx {
       }
     } catch (IOException e) {
       LOG.info("Read failed when processing possible perfect overwrite, path="
-          + path + " error:" + e);
+          + path, e);
       return new WRITE3Response(Nfs3Status.NFS3ERR_IO, wccData, 0, stableHow,
           Nfs3Constant.WRITE_COMMIT_VERF);
     } finally {
@@ -914,7 +914,7 @@ class OpenFileCtx {
   /** Invoked by AsynDataService to write back to HDFS */
   void executeWriteBack() {
     Preconditions.checkState(asyncStatus,
-        "openFileCtx has false asyncStatus, fileId:" + latestAttr.getFileid());
+        "openFileCtx has false asyncStatus, fileId:" + latestAttr.getFileId());
     final long startOffset = asyncWriteBackStartOffset;  
     try {
       while (activeState) {
@@ -940,7 +940,7 @@ class OpenFileCtx {
           asyncStatus = false;
         } else {
           LOG.info("Another asyn task is already started before this one"
-              + " is finalized. fileId:" + latestAttr.getFileid()
+              + " is finalized. fileId:" + latestAttr.getFileId()
               + " asyncStatus:" + asyncStatus + " original startOffset:"
               + startOffset + " new startOffset:" + asyncWriteBackStartOffset
               + ". Won't change asyncStatus here.");
@@ -969,11 +969,11 @@ class OpenFileCtx {
     } catch (ClosedChannelException cce) {
       if (!pendingWrites.isEmpty()) {
         LOG.error("Can't sync for fileId: " + latestAttr.getFileId()
-            + ". Channel closed with writes pending");
+            + ". Channel closed with writes pending.", cce);
       }
       status = Nfs3Status.NFS3ERR_IO;
     } catch (IOException e) {
-      LOG.error("Got stream error during data sync:" + e);
+      LOG.error("Got stream error during data sync:", e);
       // Do nothing. Stream will be closed eventually by StreamMonitor.
       status = Nfs3Status.NFS3ERR_IO;
     }
@@ -983,7 +983,7 @@ class OpenFileCtx {
       latestAttr = Nfs3Utils.getFileAttr(client,
           Nfs3Utils.getFileIdPath(latestAttr.getFileId()), iug);
     } catch (IOException e) {
-      LOG.error("Can't get new file attr for fileId: " + latestAttr.getFileId());
+      LOG.error("Can't get new file attr, fileId: " + latestAttr.getFileId(), e);
       status = Nfs3Status.NFS3ERR_IO;
     }
 
@@ -1006,7 +1006,7 @@ class OpenFileCtx {
               new VerifierNone()), commit.getXid());
       
       if (LOG.isDebugEnabled()) {
-        LOG.debug("FileId: " + latestAttr.getFileid() + " Service time:"
+        LOG.debug("FileId: " + latestAttr.getFileId() + " Service time:"
             + (System.currentTimeMillis() - commit.getStartTime())
             + "ms. Sent response for commit:" + commit);
       }
@@ -1069,7 +1069,7 @@ class OpenFileCtx {
               fos.hsync(EnumSet.of(SyncFlag.UPDATE_LENGTH));
             }
           } catch (IOException e) {
-            LOG.error("hsync failed with writeCtx:" + writeCtx + " error:" + e);
+            LOG.error("hsync failed with writeCtx:" + writeCtx, e);
             throw e;
           }
         }
@@ -1101,7 +1101,7 @@ class OpenFileCtx {
       }
 
       LOG.info("Clean up open file context for fileId: "
-          + latestAttr.getFileid());
+          + latestAttr.getFileId());
       cleanup();
     }
   }
@@ -1128,7 +1128,7 @@ class OpenFileCtx {
         fos.close();
       }
     } catch (IOException e) {
-      LOG.info("Can't close stream for fileId:" + latestAttr.getFileid()
+      LOG.info("Can't close stream for fileId:" + latestAttr.getFileId()
           + ", error:" + e);
     }
     
@@ -1156,7 +1156,7 @@ class OpenFileCtx {
       try {
         dumpOut.close();
       } catch (IOException e) {
-        e.printStackTrace();
+        LOG.error("Failed to close outputstream of dump file" + dumpFilePath, e);
       }
       File dumpFile = new File(dumpFilePath);
       if (dumpFile.exists() && !dumpFile.delete()) {
@@ -1167,7 +1167,7 @@ class OpenFileCtx {
       try {
         raf.close();
       } catch (IOException e) {
-        e.printStackTrace();
+        LOG.error("Got exception when closing input stream of dump file.", e);
       }
     }
   }

+ 18 - 23
hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/RpcProgramNfs3.java

@@ -268,7 +268,7 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
     try {
       attrs = writeManager.getFileAttr(dfsClient, handle, iug);
     } catch (IOException e) {
-      LOG.info("Can't get file attribute, fileId=" + handle.getFileId());
+      LOG.info("Can't get file attribute, fileId=" + handle.getFileId(), e);
       response.setStatus(Nfs3Status.NFS3ERR_IO);
       return response;
     }
@@ -381,7 +381,7 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
         wccData = Nfs3Utils.createWccData(Nfs3Utils.getWccAttr(preOpAttr),
             dfsClient, fileIdPath, iug);
       } catch (IOException e1) {
-        LOG.info("Can't get postOpAttr for fileIdPath: " + fileIdPath);
+        LOG.info("Can't get postOpAttr for fileIdPath: " + fileIdPath, e1);
       }
       if (e instanceof AccessControlException) {
         return new SETATTR3Response(Nfs3Status.NFS3ERR_ACCES, wccData);
@@ -614,13 +614,11 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
       // Only do access check.
       try {
         // Don't read from cache. Client may not have read permission.
-        attrs = Nfs3Utils.getFileAttr(
-                  dfsClient,
-                  Nfs3Utils.getFileIdPath(handle),
-                  iug);
+        attrs = Nfs3Utils.getFileAttr(dfsClient,
+            Nfs3Utils.getFileIdPath(handle), iug);
       } catch (IOException e) {
         if (LOG.isDebugEnabled()) {
-          LOG.debug("Get error accessing file, fileId:" + handle.getFileId());
+          LOG.debug("Get error accessing file, fileId:" + handle.getFileId(), e);
         }
         return new READ3Response(Nfs3Status.NFS3ERR_IO);
       }
@@ -764,7 +762,7 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
       try {
         postOpAttr = writeManager.getFileAttr(dfsClient, handle, iug);
       } catch (IOException e1) {
-        LOG.info("Can't get postOpAttr for fileId: " + handle.getFileId());
+        LOG.info("Can't get postOpAttr for fileId: " + handle.getFileId(), e1);
       }
       WccAttr attr = preOpAttr == null ? null : Nfs3Utils.getWccAttr(preOpAttr);
       WccData fileWcc = new WccData(attr, postOpAttr);
@@ -881,7 +879,7 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
           fos.close();
         } catch (IOException e1) {
           LOG.error("Can't close stream for dirFileId:" + dirHandle.getFileId()
-              + " filename: " + fileName);
+              + " filename: " + fileName, e1);
         }
       }
       if (dirWcc == null) {
@@ -890,7 +888,7 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
               dfsClient, dirFileIdPath, iug);
         } catch (IOException e1) {
           LOG.error("Can't get postOpDirAttr for dirFileId:"
-              + dirHandle.getFileId());
+              + dirHandle.getFileId(), e1);
         }
       }
       if (e instanceof AccessControlException) {
@@ -982,7 +980,7 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
         try {
           postOpDirAttr = Nfs3Utils.getFileAttr(dfsClient, dirFileIdPath, iug);
         } catch (IOException e1) {
-          LOG.info("Can't get postOpDirAttr for " + dirFileIdPath);
+          LOG.info("Can't get postOpDirAttr for " + dirFileIdPath, e);
         }
       }
       WccData dirWcc = new WccData(Nfs3Utils.getWccAttr(preOpDirAttr),
@@ -1064,7 +1062,7 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
         try {
           postOpDirAttr = Nfs3Utils.getFileAttr(dfsClient, dirFileIdPath, iug);
         } catch (IOException e1) {
-          LOG.info("Can't get postOpDirAttr for " + dirFileIdPath);
+          LOG.info("Can't get postOpDirAttr for " + dirFileIdPath, e1);
         }
       }
       WccData dirWcc = new WccData(Nfs3Utils.getWccAttr(preOpDirAttr),
@@ -1146,7 +1144,7 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
         try {
           postOpDirAttr = Nfs3Utils.getFileAttr(dfsClient, dirFileIdPath, iug);
         } catch (IOException e1) {
-          LOG.info("Can't get postOpDirAttr for " + dirFileIdPath);
+          LOG.info("Can't get postOpDirAttr for " + dirFileIdPath, e1);
         }
       }
       WccData dirWcc = new WccData(Nfs3Utils.getWccAttr(preOpDirAttr),
@@ -1236,7 +1234,7 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
             dfsClient, toDirFileIdPath, iug);
       } catch (IOException e1) {
         LOG.info("Can't get postOpDirAttr for " + fromDirFileIdPath + " or"
-            + toDirFileIdPath);
+            + toDirFileIdPath, e1);
       }
       if (e instanceof AccessControlException) {
         return new RENAME3Response(Nfs3Status.NFS3ERR_PERM, fromDirWcc,
@@ -1298,7 +1296,7 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
           .setPostOpAttr(Nfs3Utils.getFileAttr(dfsClient, linkDirIdPath, iug));
 
       return new SYMLINK3Response(Nfs3Status.NFS3_OK, new FileHandle(
-          objAttr.getFileid()), objAttr, dirWcc);
+          objAttr.getFileId()), objAttr, dirWcc);
 
     } catch (IOException e) {
       LOG.warn("Exception:" + e);
@@ -1585,7 +1583,7 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
       
       entries[0] = new READDIRPLUS3Response.EntryPlus3(
           postOpDirAttr.getFileId(), ".", 0, postOpDirAttr, new FileHandle(
-              postOpDirAttr.getFileid()));
+              postOpDirAttr.getFileId()));
       entries[1] = new READDIRPLUS3Response.EntryPlus3(dotdotFileId, "..",
           dotdotFileId, postOpDirAttr, new FileHandle(dotdotFileId));
 
@@ -1596,8 +1594,7 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
         try {
           attr = writeManager.getFileAttr(dfsClient, childHandle, iug);
         } catch (IOException e) {
-          LOG.error("Can't get file attributes for fileId:" + fileId
-              + " error:" + e);
+          LOG.error("Can't get file attributes for fileId:" + fileId, e);
           continue;
         }
         entries[i] = new READDIRPLUS3Response.EntryPlus3(fileId,
@@ -1614,8 +1611,7 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
         try {
           attr = writeManager.getFileAttr(dfsClient, childHandle, iug);
         } catch (IOException e) {
-          LOG.error("Can't get file attributes for fileId:" + fileId
-              + " error:" + e);
+          LOG.error("Can't get file attributes for fileId:" + fileId, e);
           continue;
         }
         entries[i] = new READDIRPLUS3Response.EntryPlus3(fileId,
@@ -1842,10 +1838,9 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
       try {
         postOpAttr = writeManager.getFileAttr(dfsClient, handle, iug);
       } catch (IOException e1) {
-        LOG.info("Can't get postOpAttr for fileId: " + handle.getFileId());
+        LOG.info("Can't get postOpAttr for fileId: " + handle.getFileId(), e1);
       }
-      WccData fileWcc = new WccData(preOpAttr == null ? null
-          : Nfs3Utils.getWccAttr(preOpAttr), postOpAttr);
+      WccData fileWcc = new WccData(Nfs3Utils.getWccAttr(preOpAttr), postOpAttr);
       return new COMMIT3Response(Nfs3Status.NFS3ERR_IO, fileWcc,
           Nfs3Constant.WRITE_COMMIT_VERF);
     }

+ 4 - 4
hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/WriteManager.java

@@ -156,7 +156,7 @@ public class WriteManager {
         }
         throw e;
       } catch (IOException e) {
-        LOG.error("Can't apapend to file:" + fileIdPath + ", error:" + e);
+        LOG.error("Can't apapend to file:" + fileIdPath, e);
         if (fos != null) {
           fos.close();
         }
@@ -181,7 +181,7 @@ public class WriteManager {
         try {
           fos.close();
         } catch (IOException e) {
-          LOG.error("Can't close stream for fileId:" + handle.getFileId());
+          LOG.error("Can't close stream for fileId:" + handle.getFileId(), e);
         }
         // Notify client to retry
         WccData fileWcc = new WccData(latestAttr.getWccAttr(), latestAttr);
@@ -284,10 +284,10 @@ public class WriteManager {
     // Send out the response
     Nfs3FileAttributes postOpAttr = null;
     try {
-      String fileIdPath = Nfs3Utils.getFileIdPath(preOpAttr.getFileid());
+      String fileIdPath = Nfs3Utils.getFileIdPath(preOpAttr.getFileId());
       postOpAttr = Nfs3Utils.getFileAttr(dfsClient, fileIdPath, iug);
     } catch (IOException e1) {
-      LOG.info("Can't get postOpAttr for fileId: " + preOpAttr.getFileid());
+      LOG.info("Can't get postOpAttr for fileId: " + preOpAttr.getFileId(), e1);
     }
     WccData fileWcc = new WccData(Nfs3Utils.getWccAttr(preOpAttr), postOpAttr);
     COMMIT3Response response = new COMMIT3Response(status, fileWcc,

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

@@ -168,6 +168,9 @@ Release 2.4.0 - UNRELEASED
 
     HDFS-6120. Fix and improve safe mode log messages. (Arpit Agarwal)
 
+    HDFS-6050. NFS does not handle exceptions correctly in a few places 
+    (brandonli)
+
   OPTIMIZATIONS
 
     HDFS-5790. LeaseManager.findPath is very slow when many leases need recovery