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 at...@apache.org on 2012/04/04 05:18:49 UTC

svn commit: r1309227 - in /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs: CHANGES.txt src/main/java/org/apache/hadoop/hdfs/client/ src/main/java/org/apache/hadoop/hdfs/client/HdfsAdmin.java src/test/java/org/apache/hadoop/hdfs/TestHdfsAdmin.java

Author: atm
Date: Wed Apr  4 03:18:49 2012
New Revision: 1309227

URL: http://svn.apache.org/viewvc?rev=1309227&view=rev
Log:
HDFS-3000. Add a public API for setting quotas. Contributed by Aaron T. Myers.

Added:
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/HdfsAdmin.java
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestHdfsAdmin.java
Modified:
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt?rev=1309227&r1=1309226&r2=1309227&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt Wed Apr  4 03:18:49 2012
@@ -185,6 +185,8 @@ Release 2.0.0 - UNRELEASED 
     HDFS-3148. The client should be able to use multiple local interfaces
     for data transfer. (eli)
 
+    HDFS-3000. Add a public API for setting quotas. (atm)
+
   IMPROVEMENTS
 
     HDFS-2018. Move all journal stream management code into one place.

Added: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/HdfsAdmin.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/HdfsAdmin.java?rev=1309227&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/HdfsAdmin.java (added)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/HdfsAdmin.java Wed Apr  4 03:18:49 2012
@@ -0,0 +1,108 @@
+/**
+ * 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.client;
+
+import java.io.IOException;
+import java.net.URI;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.tools.DFSAdmin;
+
+/**
+ * The public API for performing administrative functions on HDFS. Those writing
+ * applications against HDFS should prefer this interface to directly accessing
+ * functionality in DistributedFileSystem or DFSClient.
+ * 
+ * Note that this is distinct from the similarly-named {@link DFSAdmin}, which
+ * is a class that provides the functionality for the CLI `hdfs dfsadmin ...'
+ * commands.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class HdfsAdmin {
+  
+  private DistributedFileSystem dfs;
+  
+  /**
+   * Create a new HdfsAdmin client.
+   * 
+   * @param uri the unique URI of the HDFS file system to administer
+   * @param conf configuration
+   * @throws IOException in the event the file system could not be created
+   */
+  public HdfsAdmin(URI uri, Configuration conf) throws IOException {
+    FileSystem fs = FileSystem.get(uri, conf);
+    if (!(fs instanceof DistributedFileSystem)) {
+      throw new IllegalArgumentException("'" + uri + "' is not an HDFS URI.");
+    } else {
+      dfs = (DistributedFileSystem)fs;
+    }
+  }
+  
+  /**
+   * Set the namespace quota (count of files, directories, and sym links) for a
+   * directory.
+   * 
+   * @param src the path to set the quota for
+   * @param quota the value to set for the quota
+   * @throws IOException in the event of error
+   */
+  public void setQuota(Path src, long quota) throws IOException {
+    dfs.setQuota(src, quota, HdfsConstants.QUOTA_DONT_SET);
+  }
+  
+  /**
+   * Clear the namespace quota (count of files, directories and sym links) for a
+   * directory.
+   * 
+   * @param src the path to clear the quota of
+   * @throws IOException in the event of error
+   */
+  public void clearQuota(Path src) throws IOException {
+    dfs.setQuota(src, HdfsConstants.QUOTA_RESET, HdfsConstants.QUOTA_DONT_SET);
+  }
+  
+  /**
+   * Set the disk space quota (size of files) for a directory. Note that
+   * directories and sym links do not occupy disk space.
+   * 
+   * @param src the path to set the space quota of
+   * @param spaceQuota the value to set for the space quota
+   * @throws IOException in the event of error
+   */
+  public void setSpaceQuota(Path src, long spaceQuota) throws IOException {
+    dfs.setQuota(src, HdfsConstants.QUOTA_DONT_SET, spaceQuota);
+  }
+  
+  /**
+   * Clear the disk space quota (size of files) for a directory. Note that
+   * directories and sym links do not occupy disk space.
+   * 
+   * @param src the path to clear the space quota of
+   * @throws IOException in the event of error
+   */
+  public void clearSpaceQuota(Path src) throws IOException {
+    dfs.setQuota(src, HdfsConstants.QUOTA_DONT_SET, HdfsConstants.QUOTA_RESET);
+  }
+}

Added: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestHdfsAdmin.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestHdfsAdmin.java?rev=1309227&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestHdfsAdmin.java (added)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestHdfsAdmin.java Wed Apr  4 03:18:49 2012
@@ -0,0 +1,95 @@
+/**
+ * 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 static org.junit.Assert.*;
+
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.client.HdfsAdmin;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestHdfsAdmin {
+  
+  private static final Path TEST_PATH = new Path("/test");
+  private Configuration conf = new Configuration();
+  private MiniDFSCluster cluster;
+  
+  @Before
+  public void setUpCluster() throws IOException {
+    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0).build();
+  }
+  
+  @After
+  public void shutDownCluster() {
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
+
+  /**
+   * Test that we can set and clear quotas via {@link HdfsAdmin}.
+   */
+  @Test
+  public void testHdfsAdminSetQuota() throws Exception {
+    HdfsAdmin dfsAdmin = new HdfsAdmin(
+        FileSystem.getDefaultUri(conf), conf);
+    FileSystem fs = null;
+    try {
+      fs = FileSystem.get(conf);
+      assertTrue(fs.mkdirs(TEST_PATH));
+      assertEquals(-1, fs.getContentSummary(TEST_PATH).getQuota());
+      assertEquals(-1, fs.getContentSummary(TEST_PATH).getSpaceQuota());
+      
+      dfsAdmin.setSpaceQuota(TEST_PATH, 10);
+      assertEquals(-1, fs.getContentSummary(TEST_PATH).getQuota());
+      assertEquals(10, fs.getContentSummary(TEST_PATH).getSpaceQuota());
+      
+      dfsAdmin.setQuota(TEST_PATH, 10);
+      assertEquals(10, fs.getContentSummary(TEST_PATH).getQuota());
+      assertEquals(10, fs.getContentSummary(TEST_PATH).getSpaceQuota());
+      
+      dfsAdmin.clearSpaceQuota(TEST_PATH);
+      assertEquals(10, fs.getContentSummary(TEST_PATH).getQuota());
+      assertEquals(-1, fs.getContentSummary(TEST_PATH).getSpaceQuota());
+      
+      dfsAdmin.clearQuota(TEST_PATH);
+      assertEquals(-1, fs.getContentSummary(TEST_PATH).getQuota());
+      assertEquals(-1, fs.getContentSummary(TEST_PATH).getSpaceQuota());
+    } finally {
+      if (fs != null) {
+        fs.close();
+      }
+    }
+  }
+  
+  /**
+   * Make sure that a non-HDFS URI throws a helpful error.
+   */
+  @Test(expected = IllegalArgumentException.class)
+  public void testHdfsAdminWithBadUri() throws IOException, URISyntaxException {
+    new HdfsAdmin(new URI("file:///bad-scheme"), conf);
+  }
+}