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/02 17:47:21 UTC

svn commit: r1591957 - in /hadoop/common/branches/HDFS-2006/hadoop-hdfs-project/hadoop-hdfs: CHANGES-HDFS-2006.txt src/main/java/org/apache/hadoop/fs/Hdfs.java src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileContextXAttr.java

Author: umamahesh
Date: Fri May  2 15:47:21 2014
New Revision: 1591957

URL: http://svn.apache.org/r1591957
Log:
HDFS-6303. HDFS implementation of FileContext API for XAttrs. Contributed by Yi Liu and Charles Lamb.

Added:
    hadoop/common/branches/HDFS-2006/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileContextXAttr.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/fs/Hdfs.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=1591957&r1=1591956&r2=1591957&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 Fri May  2 15:47:21 2014
@@ -16,6 +16,8 @@ HDFS-2006 (Unreleased)
 
    HDFS-6258. Namenode server-side storage for XAttrs. (Yi Liu via umamahesh)
 
+   HDFS-6303. HDFS implementation of FileContext API for XAttrs. (Yi Liu and Charles Lamb via umamahesh)
+
   OPTIMIZATIONS
 
   BUG FIXES

Modified: hadoop/common/branches/HDFS-2006/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/fs/Hdfs.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2006/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/fs/Hdfs.java?rev=1591957&r1=1591956&r2=1591957&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2006/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/fs/Hdfs.java (original)
+++ hadoop/common/branches/HDFS-2006/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/fs/Hdfs.java Fri May  2 15:47:21 2014
@@ -25,6 +25,7 @@ import java.net.URISyntaxException;
 import java.util.ArrayList;
 import java.util.EnumSet;
 import java.util.List;
+import java.util.Map;
 import java.util.NoSuchElementException;
 
 import org.apache.hadoop.classification.InterfaceAudience;
@@ -414,6 +415,33 @@ public class Hdfs extends AbstractFileSy
   public AclStatus getAclStatus(Path path) throws IOException {
     return dfs.getAclStatus(getUriPath(path));
   }
+  
+  @Override
+  public void setXAttr(Path path, String name, byte[] value, 
+      EnumSet<XAttrSetFlag> flag) throws IOException {
+    dfs.setXAttr(getUriPath(path), name, value, flag);
+  }
+  
+  @Override
+  public byte[] getXAttr(Path path, String name) throws IOException {
+    return dfs.getXAttr(getUriPath(path), name);
+  }
+  
+  @Override
+  public Map<String, byte[]> getXAttrs(Path path) throws IOException {
+    return dfs.getXAttrs(getUriPath(path));
+  }
+  
+  @Override
+  public Map<String, byte[]> getXAttrs(Path path, List<String> names) 
+      throws IOException {
+    return dfs.getXAttrs(getUriPath(path), names);
+  }
+  
+  @Override
+  public void removeXAttr(Path path, String name) throws IOException {
+    dfs.removeXAttr(getUriPath(path), name);
+  }
 
   /**
    * Renew an existing delegation token.

Added: hadoop/common/branches/HDFS-2006/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileContextXAttr.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2006/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileContextXAttr.java?rev=1591957&view=auto
==============================================================================
--- hadoop/common/branches/HDFS-2006/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileContextXAttr.java (added)
+++ hadoop/common/branches/HDFS-2006/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileContextXAttr.java Fri May  2 15:47:21 2014
@@ -0,0 +1,103 @@
+/**
+ * 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 java.io.IOException;
+import java.net.URI;
+import java.util.EnumSet;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileContext;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.XAttrSetFlag;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.junit.BeforeClass;
+
+/**
+ * Tests of XAttr operations using FileContext APIs.
+ */
+public class TestFileContextXAttr extends FSXAttrBaseTest  {
+  @BeforeClass
+  public static void init() throws Exception {
+    conf = new Configuration();
+    conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_XATTRS_ENABLED_KEY, true);
+    conf.setInt(DFSConfigKeys.DFS_NAMENODE_INODE_XATTRS_MAX_LIMIT_KEY, 3);
+    initCluster(true);
+  }
+
+  @Override
+  protected FileSystem createFileSystem() throws Exception {
+    FileContextFS fcFs = new FileContextFS();
+    fcFs.initialize(FileSystem.getDefaultUri(conf), conf);
+    return fcFs;
+  }
+
+  /**
+   * This reuses FSXAttrBaseTest's testcases by creating a filesystem
+   * implementation which uses FileContext by only overriding the xattr related
+   * methods. Other operations will use the normal filesystem.
+   */
+  public static class FileContextFS extends DistributedFileSystem {
+
+    private FileContext fc;
+
+    @Override
+    public void initialize(URI uri, Configuration conf) throws IOException {
+      super.initialize(uri, conf);
+      fc = FileContext.getFileContext(conf);
+    }
+    
+    @Override
+    public void setXAttr(Path path, final String name, final byte[] value)
+      throws IOException {
+      fc.setXAttr(path, name, value);
+    }
+    
+    @Override
+    public void setXAttr(Path path, final String name, final byte[] value, 
+        final EnumSet<XAttrSetFlag> flag) throws IOException {
+      fc.setXAttr(path, name, value, flag);
+    }
+    
+    @Override
+    public byte[] getXAttr(Path path, final String name) throws IOException {
+      return fc.getXAttr(path, name);
+    }
+    
+    @Override
+    public Map<String, byte[]> getXAttrs(Path path) throws IOException {
+      return fc.getXAttrs(path);
+    }
+    
+    @Override
+    public Map<String, byte[]> getXAttrs(Path path, final List<String> names) 
+        throws IOException {
+      return fc.getXAttrs(path, names);
+    }
+    
+    @Override
+    public void removeXAttr(Path path, final String name) throws IOException {
+      fc.removeXAttr(path, name);
+    }
+  }
+}