You are viewing a plain text version of this content. The canonical link for it is here.
Posted to hdfs-commits@hadoop.apache.org by um...@apache.org on 2014/05/03 15:51:09 UTC

svn commit: r1592208 - in /hadoop/common/branches/HDFS-2006/hadoop-hdfs-project/hadoop-hdfs: ./ src/main/java/org/apache/hadoop/hdfs/ src/main/java/org/apache/hadoop/hdfs/server/namenode/

Author: umamahesh
Date: Sat May  3 13:51:08 2014
New Revision: 1592208

URL: http://svn.apache.org/r1592208
Log:
HDFS-6324. Shift XAttr helper code out for reuse. Contributed by Yi Liu.

Added:
    hadoop/common/branches/HDFS-2006/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/XAttrHelper.java
Modified:
    hadoop/common/branches/HDFS-2006/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-2006.txt
    hadoop/common/branches/HDFS-2006/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
    hadoop/common/branches/HDFS-2006/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/XAttrPermissionFilter.java

Modified: hadoop/common/branches/HDFS-2006/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-2006.txt
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2006/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-2006.txt?rev=1592208&r1=1592207&r2=1592208&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2006/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-2006.txt (original)
+++ hadoop/common/branches/HDFS-2006/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-2006.txt Sat May  3 13:51:08 2014
@@ -18,6 +18,8 @@ HDFS-2006 (Unreleased)
 
    HDFS-6303. HDFS implementation of FileContext API for XAttrs. (Yi Liu and Charles Lamb via umamahesh)
 
+   HDFS-6324. Shift XAttr helper code out for reuse. (Yi Liu via umamahesh)
+
   OPTIMIZATIONS
 
   BUG FIXES

Modified: hadoop/common/branches/HDFS-2006/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2006/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java?rev=1592208&r1=1592207&r2=1592208&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2006/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java (original)
+++ hadoop/common/branches/HDFS-2006/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java Sat May  3 13:51:08 2014
@@ -91,7 +91,6 @@ import javax.net.SocketFactory;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.BlockLocation;
@@ -194,8 +193,6 @@ import org.apache.hadoop.util.Time;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Joiner;
 import com.google.common.base.Preconditions;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
 import com.google.common.net.InetAddresses;
 
 /********************************************************
@@ -2763,44 +2760,11 @@ public class DFSClient implements java.i
     }
   }
   
-  XAttr buildXAttr(String name, byte[] value) {
-    if (name == null) {
-      throw new NullPointerException("XAttr name can not be null.");
-    }
-    
-    int prefixIndex = name.indexOf(".");
-    if (prefixIndex == -1) {
-      throw new HadoopIllegalArgumentException("XAttr name must be prefixed with" +
-          " user/trusted/security/system and followed by '.'");
-    } else if (prefixIndex == name.length() -1) {
-      throw new HadoopIllegalArgumentException("XAttr name can not be empty.");
-    }
-    
-    XAttr.NameSpace ns;
-    String prefix = name.substring(0, prefixIndex).toUpperCase();
-    if (prefix.equals(XAttr.NameSpace.USER.toString())) {
-      ns = XAttr.NameSpace.USER;
-    } else if (prefix.equals(XAttr.NameSpace.TRUSTED.toString())) {
-      ns = XAttr.NameSpace.TRUSTED;
-    } else if (prefix.equals(XAttr.NameSpace.SECURITY.toString())) {
-      ns = XAttr.NameSpace.SECURITY;
-    } else if (prefix.equals(XAttr.NameSpace.SYSTEM.toString())) {
-      ns = XAttr.NameSpace.SYSTEM;
-    } else {
-      throw new HadoopIllegalArgumentException("XAttr name must be prefixed with" +
-          " user/trusted/security/system and followed by '.'");
-    }
-    XAttr xAttr = (new XAttr.Builder()).setNameSpace(ns).setName(name.
-        substring(prefixIndex + 1)).setValue(value).build();
-    
-    return xAttr;
-  }
-  
   public void setXAttr(String src, String name, byte[] value, 
       EnumSet<XAttrSetFlag> flag) throws IOException {
     checkOpen();
     try {
-      namenode.setXAttr(src, buildXAttr(name, value), flag);
+      namenode.setXAttr(src, XAttrHelper.buildXAttr(name, value), flag);
     } catch (RemoteException re) {
       throw re.unwrapRemoteException(AccessControlException.class,
                                      AclException.class,
@@ -2815,19 +2779,9 @@ public class DFSClient implements java.i
   public byte[] getXAttr(String src, String name) throws IOException {
     checkOpen();
     try {
-      final XAttr xAttr = buildXAttr(name, null);
-      final List<XAttr> xAttrs = Lists.newArrayListWithCapacity(1);
-      xAttrs.add(xAttr);
+      final List<XAttr> xAttrs = XAttrHelper.buildXAttrAsList(name);
       final List<XAttr> result = namenode.getXAttrs(src, xAttrs);
-      byte[] value = null;
-      if (result != null && !result.isEmpty()) {
-        XAttr a = result.get(0);
-        value = a.getValue();
-        if (value == null) {
-          value = new byte[0]; //xattr exists, but no value.
-        }
-      }
-      return value;
+      return XAttrHelper.getFirstXAttrValue(result);
     } catch(RemoteException re) {
       throw re.unwrapRemoteException(AccessControlException.class,
                                      AclException.class,
@@ -2836,28 +2790,10 @@ public class DFSClient implements java.i
     }
   }
   
-  Map<String, byte[]> buildXAttrMap(List<XAttr> xAttrs) {
-    if (xAttrs == null) {
-      return null;
-    }
-    Map<String, byte[]> xAttrMap = Maps.newHashMap();
-    for (XAttr xAttr : xAttrs) {
-      String namespace = xAttr.getNameSpace().toString();
-      String name = namespace.toLowerCase() + "." + xAttr.getName();
-      byte[] value = xAttr.getValue();
-      if (value == null) {
-        value = new byte[0];
-      }
-      xAttrMap.put(name, value);
-    }
-    
-    return xAttrMap;
-  }
-  
   public Map<String, byte[]> getXAttrs(String src) throws IOException {
     checkOpen();
     try {
-      return buildXAttrMap(namenode.getXAttrs(src, null));
+      return XAttrHelper.buildXAttrMap(namenode.getXAttrs(src, null));
     } catch(RemoteException re) {
       throw re.unwrapRemoteException(AccessControlException.class,
                                      AclException.class,
@@ -2866,23 +2802,12 @@ public class DFSClient implements java.i
     }
   }
   
-  List<XAttr> buildXAttrs(List<String> names) {
-    if (names == null || names.isEmpty()) {
-      throw new HadoopIllegalArgumentException("XAttr names can not be null or empty.");
-    }
-    
-    List<XAttr> xAttrs = Lists.newArrayListWithCapacity(names.size());
-    for (String name : names) {
-      xAttrs.add(buildXAttr(name, null));
-    }
-    return xAttrs;
-  }
-  
   public Map<String, byte[]> getXAttrs(String src, List<String> names) 
       throws IOException {
     checkOpen();
     try {
-      return buildXAttrMap(namenode.getXAttrs(src, buildXAttrs(names)));
+      return XAttrHelper.buildXAttrMap(namenode.getXAttrs(
+          src, XAttrHelper.buildXAttrs(names)));
     } catch(RemoteException re) {
       throw re.unwrapRemoteException(AccessControlException.class,
                                      AclException.class,
@@ -2894,7 +2819,7 @@ public class DFSClient implements java.i
   public void removeXAttr(String src, String name) throws IOException {
     checkOpen();
     try {
-      namenode.removeXAttr(src, buildXAttr(name, null));
+      namenode.removeXAttr(src, XAttrHelper.buildXAttr(name));
     } catch(RemoteException re) {
       throw re.unwrapRemoteException(AccessControlException.class,
                                      AclException.class,

Added: hadoop/common/branches/HDFS-2006/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/XAttrHelper.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2006/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/XAttrHelper.java?rev=1592208&view=auto
==============================================================================
--- hadoop/common/branches/HDFS-2006/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/XAttrHelper.java (added)
+++ hadoop/common/branches/HDFS-2006/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/XAttrHelper.java Sat May  3 13:51:08 2014
@@ -0,0 +1,162 @@
+/**
+ * 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;
+
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.HadoopIllegalArgumentException;
+import org.apache.hadoop.fs.XAttr;
+import org.apache.hadoop.fs.XAttr.NameSpace;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+
+public class XAttrHelper {
+  
+  /**
+   * Build <code>XAttr</code> from xattr name with prefix.
+   */
+  public static XAttr buildXAttr(String name) {
+    return buildXAttr(name, null);
+  }
+  
+  /**
+   * Build <code>XAttr</code> from name with prefix and value.
+   * Name can not be null. Value can be null. The name and prefix 
+   * are validated.
+   * Both name and namespace are case sensitive.
+   */
+  public static XAttr buildXAttr(String name, byte[] value) {
+    Preconditions.checkNotNull(name, "XAttr name can not be null.");
+    
+    final int prefixIndex = name.indexOf(".");
+    if (prefixIndex < 4) {// Prefix length is at least 4.
+      throw new HadoopIllegalArgumentException("An XAttr name must be " +
+          "prefixed with user/trusted/security/system, followed by a '.'");
+    } else if (prefixIndex == name.length() - 1) {
+      throw new HadoopIllegalArgumentException("XAttr name can not be empty.");
+    }
+    
+    NameSpace ns;
+    final String prefix = name.substring(0, prefixIndex);
+    if (prefix.equals(NameSpace.USER.toString().toLowerCase())) {
+      ns = NameSpace.USER;
+    } else if (prefix.equals(NameSpace.TRUSTED.toString().toLowerCase())) {
+      ns = NameSpace.TRUSTED;
+    } else if (prefix.equals(NameSpace.SYSTEM.toString().toLowerCase())) {
+      ns = NameSpace.SYSTEM;
+    } else if (prefix.equals(NameSpace.SECURITY.toString().toLowerCase())) {
+      ns = NameSpace.SECURITY;
+    } else {
+      throw new HadoopIllegalArgumentException("An XAttr name must be " +
+          "prefixed with user/trusted/security/system, followed by a '.'");
+    }
+    XAttr xAttr = (new XAttr.Builder()).setNameSpace(ns).setName(name.
+        substring(prefixIndex + 1)).setValue(value).build();
+    
+    return xAttr;
+  }
+  
+  /**
+   * Build xattr name with prefix as <code>XAttr</code> list.
+   */
+  public static List<XAttr> buildXAttrAsList(String name) {
+    XAttr xAttr = buildXAttr(name);
+    List<XAttr> xAttrs = Lists.newArrayListWithCapacity(1);
+    xAttrs.add(xAttr);
+    
+    return xAttrs;
+  }
+  
+  /**
+   * Get value of first xattr from <code>XAttr</code> list
+   */
+  public static byte[] getFirstXAttrValue(List<XAttr> xAttrs) {
+    byte[] value = null;
+    XAttr xAttr = getFirstXAttr(xAttrs);
+    if (xAttr != null) {
+      value = xAttr.getValue();
+      if (value == null) {
+        value = new byte[0]; // xattr exists, but no value.
+      }
+    }
+    return value;
+  }
+  
+  /**
+   * Get first xattr from <code>XAttr</code> list
+   */
+  public static XAttr getFirstXAttr(List<XAttr> xAttrs) {
+    if (xAttrs != null && !xAttrs.isEmpty()) {
+      return xAttrs.get(0);
+    }
+    
+    return null;
+  }
+  
+  /**
+   * Build xattr map from <code>XAttr</code> list, the key is 
+   * xattr name with prefix, and value is xattr value. 
+   */
+  public static Map<String, byte[]> buildXAttrMap(List<XAttr> xAttrs) {
+    if (xAttrs == null) {
+      return null;
+    }
+    Map<String, byte[]> xAttrMap = Maps.newHashMap();
+    for (XAttr xAttr : xAttrs) {
+      String name = getPrefixName(xAttr);
+      byte[] value = xAttr.getValue();
+      if (value == null) {
+        value = new byte[0];
+      }
+      xAttrMap.put(name, value);
+    }
+    
+    return xAttrMap;
+  }
+  
+  /**
+   * Get name with prefix from <code>XAttr</code>
+   */
+  public static String getPrefixName(XAttr xAttr) {
+    if (xAttr == null) {
+      return null;
+    }
+    
+    String namespace = xAttr.getNameSpace().toString();
+    return namespace.toLowerCase() + "." + xAttr.getName();
+  }
+
+  /**
+   * Build <code>XAttr</code> list from xattr name list.
+   */
+  public static List<XAttr> buildXAttrs(List<String> names) {
+    if (names == null || names.isEmpty()) {
+      throw new HadoopIllegalArgumentException("XAttr names can not be " +
+      		"null or empty.");
+    }
+    
+    List<XAttr> xAttrs = Lists.newArrayListWithCapacity(names.size());
+    for (String name : names) {
+      xAttrs.add(buildXAttr(name, null));
+    }
+    return xAttrs;
+  } 
+}

Modified: hadoop/common/branches/HDFS-2006/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/XAttrPermissionFilter.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2006/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/XAttrPermissionFilter.java?rev=1592208&r1=1592207&r2=1592208&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2006/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/XAttrPermissionFilter.java (original)
+++ hadoop/common/branches/HDFS-2006/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/XAttrPermissionFilter.java Sat May  3 13:51:08 2014
@@ -22,6 +22,7 @@ import java.util.List;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.fs.XAttr;
+import org.apache.hadoop.hdfs.XAttrHelper;
 import org.apache.hadoop.security.AccessControlException;
 
 import com.google.common.collect.Lists;
@@ -56,7 +57,7 @@ public class XAttrPermissionFilter {
       return;
     }
     throw new AccessControlException("User doesn't have permission for xattr: "
-        + xAttr.getNameSpace().toString().toLowerCase() + "." + xAttr.getName());
+        + XAttrHelper.getPrefixName(xAttr));
   }
   
   static List<XAttr> filterXAttrsForApi(FSPermissionChecker pc,