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 to...@apache.org on 2012/01/23 22:57:00 UTC

svn commit: r1235017 - in /hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs: CHANGES.HDFS-1623.txt src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestQuotasWithHA.java

Author: todd
Date: Mon Jan 23 21:57:00 2012
New Revision: 1235017

URL: http://svn.apache.org/viewvc?rev=1235017&view=rev
Log:
HDFS-2688. Add tests for quota tracking in an HA cluster. Contributed by Todd Lipcon.

Added:
    hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestQuotasWithHA.java
Modified:
    hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/CHANGES.HDFS-1623.txt

Modified: hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/CHANGES.HDFS-1623.txt
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/CHANGES.HDFS-1623.txt?rev=1235017&r1=1235016&r2=1235017&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/CHANGES.HDFS-1623.txt (original)
+++ hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/CHANGES.HDFS-1623.txt Mon Jan 23 21:57:00 2012
@@ -123,3 +123,5 @@ HDFS-2812. When becoming active, the NN 
 HDFS-2737. Automatically trigger log rolls periodically on the active NN. (todd and atm)
 
 HDFS-2820. Add a simple sanity check for HA config (todd)
+
+HDFS-2688. Add tests for quota tracking in an HA cluster. (todd)

Added: hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestQuotasWithHA.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestQuotasWithHA.java?rev=1235017&view=auto
==============================================================================
--- hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestQuotasWithHA.java (added)
+++ hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestQuotasWithHA.java Mon Jan 23 21:57:00 2012
@@ -0,0 +1,133 @@
+/**
+ * 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.ha;
+
+import static org.junit.Assert.*;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.ContentSummary;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DFSTestUtil;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.HAUtil;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.MiniDFSNNTopology;
+import org.apache.hadoop.hdfs.server.namenode.NameNode;
+import org.apache.hadoop.io.IOUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestQuotasWithHA {
+  private static final Path TEST_DIR = new Path("/test");
+  private static final Path TEST_FILE = new Path(TEST_DIR, "file");
+  private static final String TEST_DIR_STR = TEST_DIR.toUri().getPath();
+  
+  private static final long NS_QUOTA = 10000;
+  private static final long DS_QUOTA = 10000;
+  private static final long BLOCK_SIZE = 1024; // 1KB blocks
+  
+  private MiniDFSCluster cluster;
+  private NameNode nn0;
+  private NameNode nn1;
+  private FileSystem fs;
+
+  @Before
+  public void setupCluster() throws Exception {
+    Configuration conf = new Configuration();
+    conf.setInt(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1);
+    conf.setInt(DFSConfigKeys.DFS_HA_TAILEDITS_PERIOD_KEY, 1);
+    conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
+    HAUtil.setAllowStandbyReads(conf, true);
+    
+    cluster = new MiniDFSCluster.Builder(conf)
+      .nnTopology(MiniDFSNNTopology.simpleHATopology())
+      .numDataNodes(1)
+      .waitSafeMode(false)
+      .build();
+    cluster.waitActive();
+    
+    nn0 = cluster.getNameNode(0);
+    nn1 = cluster.getNameNode(1);
+    fs = HATestUtil.configureFailoverFs(cluster, conf);
+    
+    cluster.transitionToActive(0);
+  }
+  
+  @After
+  public void shutdownCluster() throws IOException {
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
+
+  /**
+   * Test that quotas are properly tracked by the standby through
+   * create, append, delete.
+   */
+  @Test(timeout=60000)
+  public void testQuotasTrackedOnStandby() throws Exception {
+    fs.mkdirs(TEST_DIR);
+    DistributedFileSystem dfs = (DistributedFileSystem)fs;
+    dfs.setQuota(TEST_DIR, NS_QUOTA, DS_QUOTA);
+    long expectedSize = 3 * BLOCK_SIZE + BLOCK_SIZE/2;
+    DFSTestUtil.createFile(fs, TEST_FILE, expectedSize, (short)1, 1L);
+
+    HATestUtil.waitForStandbyToCatchUp(nn0, nn1);
+    ContentSummary cs = nn1.getRpcServer().getContentSummary(TEST_DIR_STR);
+    assertEquals(NS_QUOTA, cs.getQuota());
+    assertEquals(DS_QUOTA, cs.getSpaceQuota());
+    assertEquals(expectedSize, cs.getSpaceConsumed());
+    assertEquals(1, cs.getDirectoryCount());
+    assertEquals(1, cs.getFileCount());
+
+    // Append to the file and make sure quota is updated correctly.
+    FSDataOutputStream stm = fs.append(TEST_FILE);
+    try {
+      byte[] data = new byte[(int) (BLOCK_SIZE * 3 / 2)];
+      stm.write(data);
+      expectedSize += data.length;
+    } finally {
+      IOUtils.closeStream(stm);
+    }
+    
+    HATestUtil.waitForStandbyToCatchUp(nn0, nn1);
+    cs = nn1.getRpcServer().getContentSummary(TEST_DIR_STR);
+    assertEquals(NS_QUOTA, cs.getQuota());
+    assertEquals(DS_QUOTA, cs.getSpaceQuota());
+    assertEquals(expectedSize, cs.getSpaceConsumed());
+    assertEquals(1, cs.getDirectoryCount());
+    assertEquals(1, cs.getFileCount());
+
+    
+    fs.delete(TEST_FILE, true);
+    expectedSize = 0;
+    HATestUtil.waitForStandbyToCatchUp(nn0, nn1);
+    cs = nn1.getRpcServer().getContentSummary(TEST_DIR_STR);
+    assertEquals(NS_QUOTA, cs.getQuota());
+    assertEquals(DS_QUOTA, cs.getSpaceQuota());
+    assertEquals(expectedSize, cs.getSpaceConsumed());
+    assertEquals(1, cs.getDirectoryCount());
+    assertEquals(0, cs.getFileCount());
+  }
+}