Browse Source

HDFS-5067. Merging change r1522776 from branch-2

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-2.1-beta@1522778 13f79535-47bb-0310-9956-ffa450edef68
Brandon Li 11 years ago
parent
commit
073006d290

+ 1 - 2
hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/mount/MountResponse.java

@@ -40,8 +40,7 @@ public class MountResponse {
     RpcAcceptedReply.voidReply(xdr, xid);
     RpcAcceptedReply.voidReply(xdr, xid);
     xdr.writeInt(status);
     xdr.writeInt(status);
     if (status == MNT_OK) {
     if (status == MNT_OK) {
-      xdr.writeInt(handle.length);
-      xdr.writeFixedOpaque(handle);
+      xdr.writeVariableOpaque(handle);
       // Only MountV3 returns a list of supported authFlavors
       // Only MountV3 returns a list of supported authFlavors
       xdr.writeInt(1);
       xdr.writeInt(1);
       xdr.writeInt(AuthFlavor.AUTH_SYS.getValue());
       xdr.writeInt(AuthFlavor.AUTH_SYS.getValue());

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

@@ -149,6 +149,8 @@ public class IdUserGroup {
     checkAndUpdateMaps();
     checkAndUpdateMaps();
     String uname = uidNameMap.get(uid);
     String uname = uidNameMap.get(uid);
     if (uname == null) {
     if (uname == null) {
+      LOG.warn("Can't find user name for uid " + uid
+          + ". Use default user name " + unknown);
       uname = unknown;
       uname = unknown;
     }
     }
     return uname;
     return uname;
@@ -158,6 +160,8 @@ public class IdUserGroup {
     checkAndUpdateMaps();
     checkAndUpdateMaps();
     String gname = gidNameMap.get(gid);
     String gname = gidNameMap.get(gid);
     if (gname == null) {
     if (gname == null) {
+      LOG.warn("Can't find group name for gid " + gid
+          + ". Use default group name " + unknown);
       gname = unknown;
       gname = unknown;
     }
     }
     return gname;
     return gname;

+ 5 - 6
hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/Nfs3FileAttributes.java

@@ -72,19 +72,18 @@ public class Nfs3FileAttributes {
   }
   }
    
    
   public Nfs3FileAttributes() {
   public Nfs3FileAttributes() {
-    this(false, 0, (short)0, 0, 0, 0, 0, 0, 0, 0);
+    this(NfsFileType.NFSREG, 0, (short)0, 0, 0, 0, 0, 0, 0, 0);
   }
   }
 
 
-  public Nfs3FileAttributes(boolean isDir, int nlink, short mode, int uid,
+  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 = isDir ? NfsFileType.NFSDIR.toValue() : NfsFileType.NFSREG
-        .toValue();
+    this.type = nfsType.toValue();
     this.mode = mode;
     this.mode = mode;
-    this.nlink = isDir ? (nlink + 2) : 1;
+    this.nlink = (type == NfsFileType.NFSDIR.toValue()) ? (nlink + 2) : 1;
     this.uid = uid;
     this.uid = uid;
     this.gid = gid;
     this.gid = gid;
     this.size = size;
     this.size = size;
-    if(isDir) {
+    if(type == NfsFileType.NFSDIR.toValue()) {
       this.size = getDirSize(nlink);
       this.size = getDirSize(nlink);
     }
     }
     this.used = this.size;
     this.used = this.size;

+ 2 - 2
hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/request/SYMLINK3Request.java

@@ -25,9 +25,9 @@ import org.apache.hadoop.oncrpc.XDR;
  * SYMLINK3 Request
  * SYMLINK3 Request
  */
  */
 public class SYMLINK3Request extends RequestWithHandle {
 public class SYMLINK3Request extends RequestWithHandle {
-  private final String name;
+  private final String name;     // The name of the link
   private final SetAttr3 symAttr;
   private final SetAttr3 symAttr;
-  private final String symData;
+  private final String symData;  // It contains the target
   
   
   public SYMLINK3Request(XDR xdr) throws IOException {
   public SYMLINK3Request(XDR xdr) throws IOException {
     super(xdr);
     super(xdr);

+ 1 - 1
hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/response/READLINK3Response.java

@@ -46,7 +46,7 @@ public class READLINK3Response extends NFS3Response {
     out.writeBoolean(true); // Attribute follows
     out.writeBoolean(true); // Attribute follows
     postOpSymlinkAttr.serialize(out);
     postOpSymlinkAttr.serialize(out);
     if (getStatus() == Nfs3Status.NFS3_OK) {
     if (getStatus() == Nfs3Status.NFS3_OK) {
-      out.writeFixedOpaque(path, path.length);
+      out.writeVariableOpaque(path);
     }
     }
     return out;
     return out;
   }
   }

+ 1 - 1
hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/response/SYMLINK3Response.java

@@ -60,9 +60,9 @@ public class SYMLINK3Response extends NFS3Response {
     if (this.getStatus() == Nfs3Status.NFS3_OK) {
     if (this.getStatus() == Nfs3Status.NFS3_OK) {
       out.writeBoolean(true);
       out.writeBoolean(true);
       objFileHandle.serialize(out);
       objFileHandle.serialize(out);
+      out.writeBoolean(true);
       objPostOpAttr.serialize(out);
       objPostOpAttr.serialize(out);
     }
     }
-    out.writeBoolean(true);
     dirWcc.serialize(out);
     dirWcc.serialize(out);
     
     
     return out;
     return out;

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

@@ -49,7 +49,7 @@ public class Nfs3Utils {
 
 
   public static HdfsFileStatus getFileStatus(DFSClient client, String fileIdPath)
   public static HdfsFileStatus getFileStatus(DFSClient client, String fileIdPath)
       throws IOException {
       throws IOException {
-    return client.getFileInfo(fileIdPath);
+    return client.getFileLinkInfo(fileIdPath);
   }
   }
 
 
   public static Nfs3FileAttributes getNfs3FileAttrFromFileStatus(
   public static Nfs3FileAttributes getNfs3FileAttrFromFileStatus(
@@ -59,7 +59,10 @@ public class Nfs3Utils {
      * client takes only the lower 32bit of the fileId and treats it as signed
      * client takes only the lower 32bit of the fileId and treats it as signed
      * int. When the 32th bit is 1, the client considers it invalid.
      * int. When the 32th bit is 1, the client considers it invalid.
      */
      */
-    return new Nfs3FileAttributes(fs.isDir(), fs.getChildrenNum(), fs
+    NfsFileType fileType = fs.isDir() ? NfsFileType.NFSDIR : NfsFileType.NFSREG;
+    fileType = fs.isSymlink() ? NfsFileType.NFSLNK : fileType;
+    
+    return new Nfs3FileAttributes(fileType, fs.getChildrenNum(), fs
         .getPermission().toShort(), iug.getUidAllowingUnknown(fs.getOwner()),
         .getPermission().toShort(), iug.getUidAllowingUnknown(fs.getOwner()),
         iug.getGidAllowingUnknown(fs.getGroup()), fs.getLen(), 0 /* fsid */,
         iug.getGidAllowingUnknown(fs.getGroup()), fs.getLen(), 0 /* fsid */,
         fs.getFileId(), fs.getModificationTime(), fs.getAccessTime());
         fs.getFileId(), fs.getModificationTime(), fs.getAccessTime());

+ 125 - 6
hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/RpcProgramNfs3.java

@@ -18,6 +18,7 @@
 package org.apache.hadoop.hdfs.nfs.nfs3;
 package org.apache.hadoop.hdfs.nfs.nfs3;
 
 
 import java.io.File;
 import java.io.File;
+import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.IOException;
 import java.net.InetAddress;
 import java.net.InetAddress;
 import java.nio.ByteBuffer;
 import java.nio.ByteBuffer;
@@ -44,6 +45,7 @@ import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.nfs.AccessPrivilege;
 import org.apache.hadoop.nfs.AccessPrivilege;
 import org.apache.hadoop.nfs.NfsExports;
 import org.apache.hadoop.nfs.NfsExports;
+import org.apache.hadoop.nfs.NfsFileType;
 import org.apache.hadoop.nfs.NfsTime;
 import org.apache.hadoop.nfs.NfsTime;
 import org.apache.hadoop.nfs.nfs3.FileHandle;
 import org.apache.hadoop.nfs.nfs3.FileHandle;
 import org.apache.hadoop.nfs.nfs3.IdUserGroup;
 import org.apache.hadoop.nfs.nfs3.IdUserGroup;
@@ -65,10 +67,12 @@ import org.apache.hadoop.nfs.nfs3.request.PATHCONF3Request;
 import org.apache.hadoop.nfs.nfs3.request.READ3Request;
 import org.apache.hadoop.nfs.nfs3.request.READ3Request;
 import org.apache.hadoop.nfs.nfs3.request.READDIR3Request;
 import org.apache.hadoop.nfs.nfs3.request.READDIR3Request;
 import org.apache.hadoop.nfs.nfs3.request.READDIRPLUS3Request;
 import org.apache.hadoop.nfs.nfs3.request.READDIRPLUS3Request;
+import org.apache.hadoop.nfs.nfs3.request.READLINK3Request;
 import org.apache.hadoop.nfs.nfs3.request.REMOVE3Request;
 import org.apache.hadoop.nfs.nfs3.request.REMOVE3Request;
 import org.apache.hadoop.nfs.nfs3.request.RENAME3Request;
 import org.apache.hadoop.nfs.nfs3.request.RENAME3Request;
 import org.apache.hadoop.nfs.nfs3.request.RMDIR3Request;
 import org.apache.hadoop.nfs.nfs3.request.RMDIR3Request;
 import org.apache.hadoop.nfs.nfs3.request.SETATTR3Request;
 import org.apache.hadoop.nfs.nfs3.request.SETATTR3Request;
+import org.apache.hadoop.nfs.nfs3.request.SYMLINK3Request;
 import org.apache.hadoop.nfs.nfs3.request.SetAttr3;
 import org.apache.hadoop.nfs.nfs3.request.SetAttr3;
 import org.apache.hadoop.nfs.nfs3.request.SetAttr3.SetAttrField;
 import org.apache.hadoop.nfs.nfs3.request.SetAttr3.SetAttrField;
 import org.apache.hadoop.nfs.nfs3.request.WRITE3Request;
 import org.apache.hadoop.nfs.nfs3.request.WRITE3Request;
@@ -476,9 +480,70 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
     }
     }
   }
   }
 
 
-  public READLINK3Response readlink(XDR xdr,
-      SecurityHandler securityHandler, InetAddress client) {
-    return new READLINK3Response(Nfs3Status.NFS3ERR_NOTSUPP);
+  public READLINK3Response readlink(XDR xdr, SecurityHandler securityHandler,
+      InetAddress client) {
+    READLINK3Response response = new READLINK3Response(Nfs3Status.NFS3_OK);
+
+    if (!checkAccessPrivilege(client, AccessPrivilege.READ_ONLY)) {
+      response.setStatus(Nfs3Status.NFS3ERR_ACCES);
+      return response;
+    }
+
+    DFSClient dfsClient = clientCache.get(securityHandler.getUser());
+    if (dfsClient == null) {
+      response.setStatus(Nfs3Status.NFS3ERR_SERVERFAULT);
+      return response;
+    }
+
+    READLINK3Request request = null;
+
+    try {
+      request = new READLINK3Request(xdr);
+    } catch (IOException e) {
+      LOG.error("Invalid READLINK request");
+      return new READLINK3Response(Nfs3Status.NFS3ERR_INVAL);
+    }
+
+    FileHandle handle = request.getHandle();
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("NFS READLINK fileId: " + handle.getFileId());
+    }
+
+    String fileIdPath = Nfs3Utils.getFileIdPath(handle);
+    try {
+      String target = dfsClient.getLinkTarget(fileIdPath);
+
+      Nfs3FileAttributes postOpAttr = Nfs3Utils.getFileAttr(dfsClient,
+          fileIdPath, iug);
+      if (postOpAttr == null) {
+        LOG.info("Can't get path for fileId:" + handle.getFileId());
+        return new READLINK3Response(Nfs3Status.NFS3ERR_STALE);
+      }
+      if (postOpAttr.getType() != NfsFileType.NFSLNK.toValue()) {
+        LOG.error("Not a symlink, fileId:" + handle.getFileId());
+        return new READLINK3Response(Nfs3Status.NFS3ERR_INVAL);
+      }
+      if (target == null) {
+        LOG.error("Symlink target should not be null, fileId:"
+            + handle.getFileId());
+        return new READLINK3Response(Nfs3Status.NFS3ERR_SERVERFAULT);
+      }
+      if (MAX_READ_TRANSFER_SIZE < target.getBytes().length) {
+        return new READLINK3Response(Nfs3Status.NFS3ERR_IO, postOpAttr, null);
+      }
+
+      return new READLINK3Response(Nfs3Status.NFS3_OK, postOpAttr,
+          target.getBytes());
+
+    } catch (IOException e) {
+      LOG.warn("Readlink error: " + e.getClass(), e);
+      if (e instanceof FileNotFoundException) {
+        return new READLINK3Response(Nfs3Status.NFS3ERR_STALE);
+      } else if (e instanceof AccessControlException) {
+        return new READLINK3Response(Nfs3Status.NFS3ERR_ACCES);
+      }
+      return new READLINK3Response(Nfs3Status.NFS3ERR_IO);
+    }
   }
   }
 
 
   @Override
   @Override
@@ -1121,9 +1186,63 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
   }
   }
 
 
   @Override
   @Override
-  public SYMLINK3Response symlink(XDR xdr,
-      SecurityHandler securityHandler, InetAddress client) {
-    return new SYMLINK3Response(Nfs3Status.NFS3ERR_NOTSUPP);
+  public SYMLINK3Response symlink(XDR xdr, SecurityHandler securityHandler,
+      InetAddress client) {
+    SYMLINK3Response response = new SYMLINK3Response(Nfs3Status.NFS3_OK);
+
+    if (!checkAccessPrivilege(client, AccessPrivilege.READ_WRITE)) {
+      response.setStatus(Nfs3Status.NFS3ERR_ACCES);
+      return response;
+    }
+
+    DFSClient dfsClient = clientCache.get(securityHandler.getUser());
+    if (dfsClient == null) {
+      response.setStatus(Nfs3Status.NFS3ERR_SERVERFAULT);
+      return response;
+    }
+
+    SYMLINK3Request request = null;
+    try {
+      request = new SYMLINK3Request(xdr);
+    } catch (IOException e) {
+      LOG.error("Invalid SYMLINK request");
+      response.setStatus(Nfs3Status.NFS3ERR_INVAL);
+      return response;
+    }
+
+    FileHandle dirHandle = request.getHandle();
+    String name = request.getName();
+    String symData = request.getSymData();
+    String linkDirIdPath = Nfs3Utils.getFileIdPath(dirHandle);
+    // Don't do any name check to source path, just leave it to HDFS
+    String linkIdPath = linkDirIdPath + "/" + name;
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("NFS SYMLINK, target: " + symData + " link: " + linkIdPath);
+    }
+
+    try {
+      WccData dirWcc = response.getDirWcc();
+      WccAttr preOpAttr = Nfs3Utils.getWccAttr(dfsClient, linkDirIdPath);
+      dirWcc.setPreOpAttr(preOpAttr);
+
+      dfsClient.createSymlink(symData, linkIdPath, false);
+      // Set symlink attr is considered as to change the attr of the target
+      // file. So no need to set symlink attr here after it's created.
+
+      HdfsFileStatus linkstat = dfsClient.getFileLinkInfo(linkIdPath);
+      Nfs3FileAttributes objAttr = Nfs3Utils.getNfs3FileAttrFromFileStatus(
+          linkstat, iug);
+      dirWcc
+          .setPostOpAttr(Nfs3Utils.getFileAttr(dfsClient, linkDirIdPath, iug));
+
+      return new SYMLINK3Response(Nfs3Status.NFS3_OK, new FileHandle(
+          objAttr.getFileid()), objAttr, dirWcc);
+
+    } catch (IOException e) {
+      LOG.warn("Exception:" + e);
+      response.setStatus(Nfs3Status.NFS3ERR_IO);
+      return response;
+    }
   }
   }
 
 
   public READDIR3Response link(XDR xdr, SecurityHandler securityHandler, InetAddress client) {
   public READDIR3Response link(XDR xdr, SecurityHandler securityHandler, InetAddress client) {

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

@@ -41,6 +41,8 @@ Release 2.1.1-beta - UNRELEASED
     HDFS-5085. Refactor o.a.h.nfs to support different types of 
     HDFS-5085. Refactor o.a.h.nfs to support different types of 
     authentications. (jing9)
     authentications. (jing9)
 
 
+    HDFS-5067 Support symlink operations in NFS gateway. (brandonli)
+
   IMPROVEMENTS
   IMPROVEMENTS
 
 
     HDFS-4513. Clarify in the WebHDFS REST API that all JSON respsonses may
     HDFS-4513. Clarify in the WebHDFS REST API that all JSON respsonses may