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/10/11 08:14:40 UTC

svn commit: r1396918 [3/3] - in /hadoop/common/branches/HDFS-3077/hadoop-hdfs-project: hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/ hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/ hadoop-hdfs/ hadoop-hdfs/src/ h...

Modified: hadoop/common/branches/HDFS-3077/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/common/TestJspHelper.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-3077/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/common/TestJspHelper.java?rev=1396918&r1=1396917&r2=1396918&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-3077/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/common/TestJspHelper.java (original)
+++ hadoop/common/branches/HDFS-3077/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/common/TestJspHelper.java Thu Oct 11 06:14:26 2012
@@ -22,6 +22,7 @@ import static org.mockito.Mockito.when;
 
 import java.io.IOException;
 import java.net.InetSocketAddress;
+import java.util.ArrayList;
 
 import javax.servlet.ServletContext;
 import javax.servlet.http.HttpServletRequest;
@@ -29,7 +30,9 @@ import javax.servlet.http.HttpServletReq
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.namenode.NameNodeHttpServer;
 import org.apache.hadoop.hdfs.web.resources.DoAsParam;
 import org.apache.hadoop.hdfs.web.resources.UserParam;
@@ -399,4 +402,43 @@ public class TestJspHelper {
                           ugi.getAuthenticationMethod());
     }
   }
+
+  @Test
+  public void testSortNodeByFields() throws Exception {
+    DatanodeID dnId1 = new DatanodeID("127.0.0.1", "localhost1", "storage1",
+        1234, 2345, 3456);
+    DatanodeID dnId2 = new DatanodeID("127.0.0.2", "localhost2", "storage2",
+        1235, 2346, 3457);
+    DatanodeDescriptor dnDesc1 = new DatanodeDescriptor(dnId1, "rack1", 1024,
+        100, 924, 100, 10, 2);
+    DatanodeDescriptor dnDesc2 = new DatanodeDescriptor(dnId2, "rack2", 2500,
+        200, 1848, 200, 20, 1);
+    ArrayList<DatanodeDescriptor> live = new ArrayList<DatanodeDescriptor>();
+    live.add(dnDesc1);
+    live.add(dnDesc2);
+
+    // Test sorting by failed volumes
+    JspHelper.sortNodeList(live, "volfails", "ASC");
+    Assert.assertEquals(dnDesc2, live.get(0));
+    Assert.assertEquals(dnDesc1, live.get(1));
+    JspHelper.sortNodeList(live, "volfails", "DSC");
+    Assert.assertEquals(dnDesc1, live.get(0));
+    Assert.assertEquals(dnDesc2, live.get(1));
+
+    // Test sorting by Blockpool used
+    JspHelper.sortNodeList(live, "bpused", "ASC");
+    Assert.assertEquals(dnDesc1, live.get(0));
+    Assert.assertEquals(dnDesc2, live.get(1));
+    JspHelper.sortNodeList(live, "bpused", "DSC");
+    Assert.assertEquals(dnDesc2, live.get(0));
+    Assert.assertEquals(dnDesc1, live.get(1));
+
+    // Test sorting by Percentage Blockpool used
+    JspHelper.sortNodeList(live, "pcbpused", "ASC");
+    Assert.assertEquals(dnDesc2, live.get(0));
+    Assert.assertEquals(dnDesc1, live.get(1));
+    JspHelper.sortNodeList(live, "pcbpused", "DSC");
+    Assert.assertEquals(dnDesc1, live.get(0));
+    Assert.assertEquals(dnDesc2, live.get(1));
+  }
 }

Modified: hadoop/common/branches/HDFS-3077/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeMultipleRegistrations.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-3077/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeMultipleRegistrations.java?rev=1396918&r1=1396917&r2=1396918&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-3077/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeMultipleRegistrations.java (original)
+++ hadoop/common/branches/HDFS-3077/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeMultipleRegistrations.java Thu Oct 11 06:14:26 2012
@@ -36,6 +36,7 @@ import org.apache.hadoop.hdfs.MiniDFSNNT
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.namenode.FSImageTestUtil;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
+import org.apache.hadoop.util.StringUtils;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
@@ -216,48 +217,62 @@ public class TestDataNodeMultipleRegistr
       LOG.info("dn bpos len (still should be 3):" + bposs.length);
       Assert.assertEquals("should've registered with three namenodes", 3, bposs.length);
     } finally {
-      if(cluster != null) 
-        cluster.shutdown();
+      cluster.shutdown();
     }
   }
 
   @Test
   public void testMiniDFSClusterWithMultipleNN() throws IOException {
-
     Configuration conf = new HdfsConfiguration();
     // start Federated cluster and add a node.
     MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
       .nnTopology(MiniDFSNNTopology.simpleFederatedTopology(2))
       .build();
-    Assert.assertNotNull(cluster);
-    Assert.assertEquals("(1)Should be 2 namenodes", 2, cluster.getNumNameNodes());
     
     // add a node
-    cluster.addNameNode(conf, 0);
-    Assert.assertEquals("(1)Should be 3 namenodes", 3, cluster.getNumNameNodes());
-    cluster.shutdown();
+    try {
+      Assert.assertNotNull(cluster);
+      cluster.waitActive();
+      Assert.assertEquals("(1)Should be 2 namenodes", 2, cluster.getNumNameNodes());
+
+      cluster.addNameNode(conf, 0);
+      Assert.assertEquals("(1)Should be 3 namenodes", 3, cluster.getNumNameNodes());
+    } catch (IOException ioe) {
+      Assert.fail("Failed to add NN to cluster:" + StringUtils.stringifyException(ioe));
+    } finally {
+      cluster.shutdown();
+    }
         
     // 2. start with Federation flag set
     conf = new HdfsConfiguration();
     cluster = new MiniDFSCluster.Builder(conf)
       .nnTopology(MiniDFSNNTopology.simpleFederatedTopology(1))
       .build();
-    Assert.assertNotNull(cluster);
-    Assert.assertEquals("(2)Should be 1 namenodes", 1, cluster.getNumNameNodes());
     
-    // add a node
-    cluster.addNameNode(conf, 0);
-    Assert.assertEquals("(2)Should be 2 namenodes", 2, cluster.getNumNameNodes());
-    cluster.shutdown();
+    try {
+      Assert.assertNotNull(cluster);
+      cluster.waitActive();
+      Assert.assertEquals("(2)Should be 1 namenodes", 1, cluster.getNumNameNodes());
+    
+      // add a node
+      cluster.addNameNode(conf, 0);
+      Assert.assertEquals("(2)Should be 2 namenodes", 2, cluster.getNumNameNodes());
+    } catch (IOException ioe) {
+      Assert.fail("Failed to add NN to cluster:" + StringUtils.stringifyException(ioe));
+    } finally {
+      cluster.shutdown();
+    }
 
     // 3. start non-federated
     conf = new HdfsConfiguration();
     cluster = new MiniDFSCluster.Builder(conf).build();
-    Assert.assertNotNull(cluster);
-    Assert.assertEquals("(2)Should be 1 namenodes", 1, cluster.getNumNameNodes());
     
     // add a node
     try {
+      cluster.waitActive();
+      Assert.assertNotNull(cluster);
+      Assert.assertEquals("(2)Should be 1 namenodes", 1, cluster.getNumNameNodes());
+
       cluster.addNameNode(conf, 9929);
       Assert.fail("shouldn't be able to add another NN to non federated cluster");
     } catch (IOException e) {
@@ -268,6 +283,4 @@ public class TestDataNodeMultipleRegistr
       cluster.shutdown();
     }
   }
-      
-
 }

Modified: hadoop/common/branches/HDFS-3077/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestBackupNode.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-3077/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestBackupNode.java?rev=1396918&r1=1396917&r2=1396918&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-3077/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestBackupNode.java (original)
+++ hadoop/common/branches/HDFS-3077/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestBackupNode.java Thu Oct 11 06:14:26 2012
@@ -36,6 +36,7 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.HAUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
@@ -64,6 +65,10 @@ public class TestBackupNode {
   }
   
   static final String BASE_DIR = MiniDFSCluster.getBaseDirectory();
+  
+  static final long seed = 0xDEADBEEFL;
+  static final int blockSize = 4096;
+  static final int fileSize = 8192;
 
   @Before
   public void setUp() throws Exception {
@@ -350,14 +355,17 @@ public class TestBackupNode {
           + NetUtils.getHostPortString(add)).toUri(), conf);
       boolean canWrite = true;
       try {
-        TestCheckpoint.writeFile(bnFS, file3, replication);
+        DFSTestUtil.createFile(bnFS, file3, fileSize, fileSize, blockSize,
+            replication, seed);
       } catch (IOException eio) {
         LOG.info("Write to BN failed as expected: ", eio);
         canWrite = false;
       }
       assertFalse("Write to BackupNode must be prohibited.", canWrite);
 
-      TestCheckpoint.writeFile(fileSys, file3, replication);
+      DFSTestUtil.createFile(fileSys, file3, fileSize, fileSize, blockSize,
+          replication, seed);
+      
       TestCheckpoint.checkFile(fileSys, file3, replication);
       // should also be on BN right away
       assertTrue("file3 does not exist on BackupNode",

Modified: hadoop/common/branches/HDFS-3077/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-3077/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java?rev=1396918&r1=1396917&r2=1396918&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-3077/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java (original)
+++ hadoop/common/branches/HDFS-3077/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java Thu Oct 11 06:14:26 2012
@@ -28,22 +28,21 @@ import static org.junit.Assert.assertTru
 import static org.junit.Assert.fail;
 
 import java.io.File;
-import java.io.FilenameFilter;
 import java.io.IOException;
 import java.lang.management.ManagementFactory;
+import java.lang.management.ThreadInfo;
+import java.lang.management.ThreadMXBean;
 import java.net.InetSocketAddress;
 import java.net.URI;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;
-import java.util.Random;
 
 import org.apache.commons.cli.ParseException;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.impl.Log4JLogger;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileContext;
@@ -51,6 +50,7 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
@@ -76,6 +76,7 @@ import org.apache.hadoop.test.GenericTes
 import org.apache.hadoop.test.GenericTestUtils.LogCapturer;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.log4j.Level;
+import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
 import org.mockito.ArgumentMatcher;
@@ -116,19 +117,22 @@ public class TestCheckpoint {
     faultInjector = Mockito.mock(CheckpointFaultInjector.class);
     CheckpointFaultInjector.instance = faultInjector;
   }
-
-  static void writeFile(FileSystem fileSys, Path name, int repl)
-    throws IOException {
-    FSDataOutputStream stm = fileSys.create(name, true, fileSys.getConf()
-        .getInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096),
-        (short) repl, blockSize);
-    byte[] buffer = new byte[TestCheckpoint.fileSize];
-    Random rand = new Random(TestCheckpoint.seed);
-    rand.nextBytes(buffer);
-    stm.write(buffer);
-    stm.close();
-  }
   
+  @After
+  public void checkForSNNThreads() {
+    ThreadMXBean threadBean = ManagementFactory.getThreadMXBean();
+    
+    ThreadInfo[] infos = threadBean.getThreadInfo(threadBean.getAllThreadIds(), 20);
+    for (ThreadInfo info : infos) {
+      if (info == null) continue;
+      LOG.info("Check thread: " + info.getThreadName());
+      if (info.getThreadName().contains("SecondaryNameNode")) {
+        fail("Leaked thread: " + info + "\n" +
+            Joiner.on("\n").join(info.getStackTrace()));
+      }
+    }
+    LOG.info("--------");
+  }
   
   static void checkFile(FileSystem fileSys, Path name, int repl)
     throws IOException {
@@ -259,7 +263,8 @@ public class TestCheckpoint {
       //
       // Create a new file
       //
-      writeFile(fileSys, file1, replication);
+      DFSTestUtil.createFile(fileSys, file1, fileSize, fileSize, blockSize,
+          replication, seed);
       checkFile(fileSys, file1, replication);
     } finally {
       fileSys.close();
@@ -323,7 +328,8 @@ public class TestCheckpoint {
       //
       // Create a new file
       //
-      writeFile(fileSys, file1, replication);
+      DFSTestUtil.createFile(fileSys, file1, fileSize, fileSize, blockSize,
+          replication, seed);
       checkFile(fileSys, file1, replication);
     } finally {
       fileSys.close();
@@ -394,7 +400,8 @@ public class TestCheckpoint {
       //
       // Create a new file
       //
-      writeFile(fileSys, file1, replication);
+      DFSTestUtil.createFile(fileSys, file1, fileSize, fileSize, blockSize,
+          replication, seed);
       checkFile(fileSys, file1, replication);
     } finally {
       fileSys.close();
@@ -580,7 +587,8 @@ public class TestCheckpoint {
       //
       // Create a new file
       //
-      writeFile(fileSys, file1, replication);
+      DFSTestUtil.createFile(fileSys, file1, fileSize, fileSize, blockSize,
+          replication, seed);
       checkFile(fileSys, file1, replication);
     } finally {
       fileSys.close();
@@ -906,7 +914,8 @@ public class TestCheckpoint {
       //
       // Create file1
       //
-      writeFile(fileSys, file1, replication);
+      DFSTestUtil.createFile(fileSys, file1, fileSize, fileSize, blockSize,
+          replication, seed);
       checkFile(fileSys, file1, replication);
 
       //
@@ -933,7 +942,8 @@ public class TestCheckpoint {
       cleanupFile(fileSys, file1);
 
       // create new file file2
-      writeFile(fileSys, file2, replication);
+      DFSTestUtil.createFile(fileSys, file2, fileSize, fileSize, blockSize,
+          replication, seed);
       checkFile(fileSys, file2, replication);
 
       //
@@ -999,7 +1009,8 @@ public class TestCheckpoint {
       }
       // create new file
       Path file = new Path("namespace.dat");
-      writeFile(fs, file, replication);
+      DFSTestUtil.createFile(fs, file, fileSize, fileSize, blockSize,
+          replication, seed);
       checkFile(fs, file, replication);
 
       // create new link
@@ -1746,7 +1757,7 @@ public class TestCheckpoint {
   /**
    * Test that the 2NN triggers a checkpoint after the configurable interval
    */
-  @Test
+  @Test(timeout=30000)
   public void testCheckpointTriggerOnTxnCount() throws Exception {
     MiniDFSCluster cluster = null;
     SecondaryNameNode secondary = null;
@@ -1760,8 +1771,7 @@ public class TestCheckpoint {
           .format(true).build();
       FileSystem fs = cluster.getFileSystem();
       secondary = startSecondaryNameNode(conf);
-      Thread t = new Thread(secondary);
-      t.start();
+      secondary.startCheckpointThread();
       final NNStorage storage = secondary.getFSImage().getStorage();
 
       // 2NN should checkpoint at startup

Modified: hadoop/common/branches/HDFS-3077/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDecommissioningStatus.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-3077/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDecommissioningStatus.java?rev=1396918&r1=1396917&r2=1396918&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-3077/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDecommissioningStatus.java (original)
+++ hadoop/common/branches/HDFS-3077/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDecommissioningStatus.java Thu Oct 11 06:14:26 2012
@@ -34,6 +34,7 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.DFSClient;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
@@ -116,19 +117,6 @@ public class TestDecommissioningStatus {
     stm.close();
   }
 
-  private void writeFile(FileSystem fileSys, Path name, short repl)
-      throws IOException {
-    // create and write a file that contains three blocks of data
-    FSDataOutputStream stm = fileSys.create(name, true, fileSys.getConf()
-        .getInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096), repl,
-        blockSize);
-    byte[] buffer = new byte[fileSize];
-    Random rand = new Random(seed);
-    rand.nextBytes(buffer);
-    stm.write(buffer);
-    stm.close();
-  }
- 
   private FSDataOutputStream writeIncompleteFile(FileSystem fileSys, Path name,
       short repl) throws IOException {
     // create and write a file that contains three blocks of data
@@ -198,7 +186,8 @@ public class TestDecommissioningStatus {
     // Decommission one node. Verify the decommission status
     // 
     Path file1 = new Path("decommission.dat");
-    writeFile(fileSys, file1, replicas);
+    DFSTestUtil.createFile(fileSys, file1, fileSize, fileSize, blockSize,
+        replicas, seed);
 
     Path file2 = new Path("decommission1.dat");
     FSDataOutputStream st1 = writeIncompleteFile(fileSys, file2, replicas);

Modified: hadoop/common/branches/HDFS-3077/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileLimit.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-3077/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileLimit.java?rev=1396918&r1=1396917&r2=1396918&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-3077/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileLimit.java (original)
+++ hadoop/common/branches/HDFS-3077/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileLimit.java Thu Oct 11 06:14:26 2012
@@ -20,14 +20,12 @@ package org.apache.hadoop.hdfs.server.na
 import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
-import java.util.Random;
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.CommonConfigurationKeys;
-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.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset;
@@ -43,19 +41,6 @@ public class TestFileLimit {
   static final int blockSize = 8192;
   boolean simulatedStorage = false;
 
-  // creates a zero file.
-  private void createFile(FileSystem fileSys, Path name)
-    throws IOException {
-    FSDataOutputStream stm = fileSys.create(name, true, fileSys.getConf()
-        .getInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096),
-        (short) 1, blockSize);
-    byte[] buffer = new byte[1024];
-    Random rand = new Random(seed);
-    rand.nextBytes(buffer);
-    stm.write(buffer);
-    stm.close();
-  }
-
   private void waitForLimit(FSNamesystem namesys, long num)
   {
     // wait for number of blocks to decrease
@@ -106,7 +91,7 @@ public class TestFileLimit {
       //
       for (int i = 0; i < maxObjects/2; i++) {
         Path file = new Path("/filestatus" + i);
-        createFile(fs, file);
+        DFSTestUtil.createFile(fs, file, 1024, 1024, blockSize, (short) 1, seed);
         System.out.println("Created file " + file);
         currentNodes += 2;      // two more objects for this creation.
       }
@@ -115,7 +100,7 @@ public class TestFileLimit {
       boolean hitException = false;
       try {
         Path file = new Path("/filestatus");
-        createFile(fs, file);
+        DFSTestUtil.createFile(fs, file, 1024, 1024, blockSize, (short) 1, seed);
         System.out.println("Created file " + file);
       } catch (IOException e) {
         hitException = true;
@@ -132,7 +117,7 @@ public class TestFileLimit {
       waitForLimit(namesys, currentNodes);
 
       // now, we shud be able to create a new file
-      createFile(fs, file0);
+      DFSTestUtil.createFile(fs, file0, 1024, 1024, blockSize, (short) 1, seed);
       System.out.println("Created file " + file0 + " again.");
       currentNodes += 2;
 

Modified: hadoop/common/branches/HDFS-3077/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestMetaSave.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-3077/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestMetaSave.java?rev=1396918&r1=1396917&r2=1396918&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-3077/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestMetaSave.java (original)
+++ hadoop/common/branches/HDFS-3077/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestMetaSave.java Thu Oct 11 06:14:26 2012
@@ -32,6 +32,7 @@ import org.apache.hadoop.fs.FSDataOutput
 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.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.junit.AfterClass;
@@ -48,17 +49,6 @@ public class TestMetaSave {
   private static MiniDFSCluster cluster = null;
   private static FileSystem fileSys = null;
 
-  private void createFile(FileSystem fileSys, Path name) throws IOException {
-    FSDataOutputStream stm = fileSys.create(name, true, fileSys.getConf()
-        .getInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096),
-        (short) 2, blockSize);
-    byte[] buffer = new byte[1024];
-    Random rand = new Random(seed);
-    rand.nextBytes(buffer);
-    stm.write(buffer);
-    stm.close();
-  }
-
   @BeforeClass
   public static void setUp() throws IOException {
     // start a cluster
@@ -84,7 +74,8 @@ public class TestMetaSave {
 
     for (int i = 0; i < 2; i++) {
       Path file = new Path("/filestatus" + i);
-      createFile(fileSys, file);
+      DFSTestUtil.createFile(fileSys, file, 1024, 1024, blockSize, (short) 2,
+          seed);
     }
 
     cluster.stopDataNode(1);

Modified: hadoop/common/branches/HDFS-3077/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameEditsConfigs.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-3077/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameEditsConfigs.java?rev=1396918&r1=1396917&r2=1396918&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-3077/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameEditsConfigs.java (original)
+++ hadoop/common/branches/HDFS-3077/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameEditsConfigs.java Thu Oct 11 06:14:26 2012
@@ -25,17 +25,15 @@ import static org.junit.Assert.fail;
 import java.io.File;
 import java.io.IOException;
 import java.util.List;
-import java.util.Random;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.CommonConfigurationKeys;
-import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType;
@@ -71,18 +69,6 @@ public class TestNameEditsConfigs {
     }
   }
 
-  private void writeFile(FileSystem fileSys, Path name, int repl)
-      throws IOException {
-    FSDataOutputStream stm = fileSys.create(name, true, fileSys.getConf()
-        .getInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096),
-        (short) repl, BLOCK_SIZE);
-    byte[] buffer = new byte[FILE_SIZE];
-    Random rand = new Random(SEED);
-    rand.nextBytes(buffer);
-    stm.write(buffer);
-    stm.close();
-  }
-
   void checkImageAndEditsFilesExistence(File dir, 
                                         boolean shouldHaveImages,
                                         boolean shouldHaveEdits)
@@ -187,7 +173,8 @@ public class TestNameEditsConfigs {
 
     try {
       assertTrue(!fileSys.exists(file1));
-      writeFile(fileSys, file1, replication);
+      DFSTestUtil.createFile(fileSys, file1, FILE_SIZE, FILE_SIZE, BLOCK_SIZE,
+          (short) replication, SEED);
       checkFile(fileSys, file1, replication);
       secondary.doCheckpoint();
     } finally {
@@ -224,7 +211,8 @@ public class TestNameEditsConfigs {
       assertTrue(fileSys.exists(file1));
       checkFile(fileSys, file1, replication);
       cleanupFile(fileSys, file1);
-      writeFile(fileSys, file2, replication);
+      DFSTestUtil.createFile(fileSys, file2, FILE_SIZE, FILE_SIZE, BLOCK_SIZE,
+          (short) replication, SEED);
       checkFile(fileSys, file2, replication);
       secondary.doCheckpoint();
     } finally {
@@ -260,7 +248,8 @@ public class TestNameEditsConfigs {
       assertTrue(fileSys.exists(file2));
       checkFile(fileSys, file2, replication);
       cleanupFile(fileSys, file2);
-      writeFile(fileSys, file3, replication);
+      DFSTestUtil.createFile(fileSys, file3, FILE_SIZE, FILE_SIZE, BLOCK_SIZE,
+          (short) replication, SEED);
       checkFile(fileSys, file3, replication);
       secondary.doCheckpoint();
     } finally {
@@ -364,7 +353,8 @@ public class TestNameEditsConfigs {
       fileSys = cluster.getFileSystem();
 
       assertTrue(!fileSys.exists(file1));
-      writeFile(fileSys, file1, replication);
+      DFSTestUtil.createFile(fileSys, file1, FILE_SIZE, FILE_SIZE, BLOCK_SIZE,
+          (short) replication, SEED);
       checkFile(fileSys, file1, replication);
     } finally  {
       fileSys.close();
@@ -402,7 +392,8 @@ public class TestNameEditsConfigs {
       assertTrue(fileSys.exists(file1));
       checkFile(fileSys, file1, replication);
       cleanupFile(fileSys, file1);
-      writeFile(fileSys, file2, replication);
+      DFSTestUtil.createFile(fileSys, file2, FILE_SIZE, FILE_SIZE, BLOCK_SIZE,
+          (short) replication, SEED);
       checkFile(fileSys, file2, replication);
     } finally {
       fileSys.close();
@@ -429,7 +420,8 @@ public class TestNameEditsConfigs {
       assertTrue(fileSys.exists(file2));
       checkFile(fileSys, file2, replication);
       cleanupFile(fileSys, file2);
-      writeFile(fileSys, file3, replication);
+      DFSTestUtil.createFile(fileSys, file3, FILE_SIZE, FILE_SIZE, BLOCK_SIZE,
+          (short) replication, SEED);
       checkFile(fileSys, file3, replication);
     } finally {
       fileSys.close();
@@ -483,7 +475,8 @@ public class TestNameEditsConfigs {
       assertTrue(fileSys.exists(file3));
       checkFile(fileSys, file3, replication);
       cleanupFile(fileSys, file3);
-      writeFile(fileSys, file3, replication);
+      DFSTestUtil.createFile(fileSys, file3, FILE_SIZE, FILE_SIZE, BLOCK_SIZE,
+          (short) replication, SEED);
       checkFile(fileSys, file3, replication);
     } finally {
       fileSys.close();

Modified: hadoop/common/branches/HDFS-3077/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStartup.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-3077/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStartup.java?rev=1396918&r1=1396917&r2=1396918&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-3077/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStartup.java (original)
+++ hadoop/common/branches/HDFS-3077/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStartup.java Thu Oct 11 06:14:26 2012
@@ -77,20 +77,6 @@ public class TestStartup {
   static final int fileSize = 8192;
   private long editsLength=0, fsimageLength=0;
 
-
-  private void writeFile(FileSystem fileSys, Path name, int repl)
-  throws IOException {
-    FSDataOutputStream stm = fileSys.create(name, true, fileSys.getConf()
-        .getInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096),
-        (short) repl, blockSize);
-    byte[] buffer = new byte[fileSize];
-    Random rand = new Random(seed);
-    rand.nextBytes(buffer);
-    stm.write(buffer);
-    stm.close();
-  }
-
-
   @Before
   public void setUp() throws Exception {
     config = new HdfsConfiguration();
@@ -150,7 +136,8 @@ public class TestStartup {
       // create a file
       FileSystem fileSys = cluster.getFileSystem();
       Path file1 = new Path("t1");
-      this.writeFile(fileSys, file1, 1);
+      DFSTestUtil.createFile(fileSys, file1, fileSize, fileSize, blockSize, 
+          (short) 1, seed);
 
       LOG.info("--doing checkpoint");
       sn.doCheckpoint();  // this shouldn't fail

Modified: hadoop/common/branches/HDFS-3077/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/security/TestPermission.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-3077/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/security/TestPermission.java?rev=1396918&r1=1396917&r2=1396918&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-3077/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/security/TestPermission.java (original)
+++ hadoop/common/branches/HDFS-3077/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/security/TestPermission.java Thu Oct 11 06:14:26 2012
@@ -166,7 +166,7 @@ public class TestPermission {
   }
 
   @Test
-  public void testFilePermision() throws Exception {
+  public void testFilePermission() throws Exception {
     final Configuration conf = new HdfsConfiguration();
     conf.setBoolean(DFSConfigKeys.DFS_PERMISSIONS_ENABLED_KEY, true);
     MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(3).build();
@@ -244,6 +244,10 @@ public class TestPermission {
       fs.mkdirs(p);
       return true;
     } catch(AccessControlException e) {
+      // We check that AccessControlExceptions contain absolute paths.
+      Path parent = p.getParent();
+      assertTrue(parent.isUriPathAbsolute());
+      assertTrue(e.getMessage().contains(parent.toString()));
       return false;
     }
   }
@@ -253,6 +257,9 @@ public class TestPermission {
       fs.create(p);
       return true;
     } catch(AccessControlException e) {
+      Path parent = p.getParent();
+      assertTrue(parent.isUriPathAbsolute());
+      assertTrue(e.getMessage().contains(parent.toString()));
       return false;
     }
   }
@@ -262,6 +269,8 @@ public class TestPermission {
       fs.open(p);
       return true;
     } catch(AccessControlException e) {
+      assertTrue(p.isUriPathAbsolute());
+      assertTrue(e.getMessage().contains(p.toString()));
       return false;
     }
   }
@@ -272,6 +281,9 @@ public class TestPermission {
       fs.rename(src, dst);
       return true;
     } catch(AccessControlException e) {
+      Path parent = dst.getParent();
+      assertTrue(parent.isUriPathAbsolute());
+      assertTrue(e.getMessage().contains(parent.toString()));
       return false;
     }
   }

Modified: hadoop/common/branches/HDFS-3077/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tools/TestDelegationTokenFetcher.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-3077/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tools/TestDelegationTokenFetcher.java?rev=1396918&r1=1396917&r2=1396918&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-3077/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tools/TestDelegationTokenFetcher.java (original)
+++ hadoop/common/branches/HDFS-3077/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tools/TestDelegationTokenFetcher.java Thu Oct 11 06:14:26 2012
@@ -36,9 +36,7 @@ import org.apache.hadoop.hdfs.security.t
 import org.apache.hadoop.hdfs.tools.DelegationTokenFetcher;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.security.Credentials;
-import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
-import org.apache.hadoop.security.token.TokenRenewer;
 import org.junit.Before;
 import org.junit.Test;
 import org.mockito.invocation.InvocationOnMock;
@@ -50,7 +48,6 @@ public class TestDelegationTokenFetcher 
   private Configuration conf;
   private URI uri;
   private static final String SERVICE_VALUE = "localhost:2005";
-  private static final Text KIND = new Text("TESTING-TOKEN-KIND");
   private static String tokenFile = "file.dta";
 
   @Before 
@@ -61,37 +58,6 @@ public class TestDelegationTokenFetcher 
     FileSystemTestHelper.addFileSystemForTesting(uri, conf, dfs);
   }
   
-  public static class FakeRenewer extends TokenRenewer {
-    static Token<?> lastRenewed = null;
-    static Token<?> lastCanceled = null;
-
-    @Override
-    public boolean handleKind(Text kind) {
-      return KIND.equals(kind);
-    }
-
-    @Override
-    public boolean isManaged(Token<?> token) throws IOException {
-      return true;
-    }
-
-    @Override
-    public long renew(Token<?> token, Configuration conf) {
-      lastRenewed = token;
-      return 0;
-    }
-
-    @Override
-    public void cancel(Token<?> token, Configuration conf) {
-      lastCanceled = token;
-    }
-    
-    public static void reset() {
-      lastRenewed = null;
-      lastCanceled = null;
-    }
-  }
-
   /**
    * Verify that when the DelegationTokenFetcher runs, it talks to the Namenode,
    * pulls out the correct user's token and successfully serializes it to disk.
@@ -103,13 +69,11 @@ public class TestDelegationTokenFetcher 
         new Text("renewer"), new Text("realuser")).getBytes();
     final byte[] pw = new byte[] { 42 };
     final Text service = new Text(uri.toString());
-    final String user = 
-        UserGroupInformation.getCurrentUser().getShortUserName();
 
     // Create a token for the fetcher to fetch, wire NN to return it when asked
     // for this particular user.
     final Token<DelegationTokenIdentifier> t = 
-      new Token<DelegationTokenIdentifier>(ident, pw, KIND, service);
+      new Token<DelegationTokenIdentifier>(ident, pw, FakeRenewer.KIND, service);
     when(dfs.addDelegationTokens(eq((String) null), any(Credentials.class))).thenAnswer(
         new Answer<Token<?>[]>() {
           @Override

Modified: hadoop/common/branches/HDFS-3077/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tools/TestJMXGet.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-3077/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tools/TestJMXGet.java?rev=1396918&r1=1396917&r2=1396918&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-3077/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tools/TestJMXGet.java (original)
+++ hadoop/common/branches/HDFS-3077/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tools/TestJMXGet.java Thu Oct 11 06:14:26 2012
@@ -32,6 +32,7 @@ import org.apache.hadoop.fs.FSDataOutput
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.tools.JMXGet;
@@ -53,19 +54,6 @@ public class TestJMXGet {
   static final int blockSize = 4096;
   static final int fileSize = 8192;
 
-  private void writeFile(FileSystem fileSys, Path name, int repl)
-  throws IOException {
-    FSDataOutputStream stm = fileSys.create(name, true,
-        fileSys.getConf().getInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096),
-        (short)repl, blockSize);
-    byte[] buffer = new byte[fileSize];
-    Random rand = new Random(seed);
-    rand.nextBytes(buffer);
-    stm.write(buffer);
-    stm.close();
-  }
-
-
   @Before
   public void setUp() throws Exception {
     config = new HdfsConfiguration();
@@ -96,7 +84,8 @@ public class TestJMXGet {
     cluster = new MiniDFSCluster.Builder(config).numDataNodes(numDatanodes).build();
     cluster.waitActive();
 
-    writeFile(cluster.getFileSystem(), new Path("/test1"), 2);
+    DFSTestUtil.createFile(cluster.getFileSystem(), new Path("/test1"),
+        fileSize, fileSize, blockSize, (short) 2, seed);
 
     JMXGet jmx = new JMXGet();
     //jmx.setService("*"); // list all hadoop services
@@ -125,7 +114,8 @@ public class TestJMXGet {
     cluster = new MiniDFSCluster.Builder(config).numDataNodes(numDatanodes).build();
     cluster.waitActive();
 
-    writeFile(cluster.getFileSystem(), new Path("/test"), 2);
+    DFSTestUtil.createFile(cluster.getFileSystem(), new Path("/test"),
+        fileSize, fileSize, blockSize, (short) 2, seed);
 
     JMXGet jmx = new JMXGet();
     //jmx.setService("*"); // list all hadoop services

Modified: hadoop/common/branches/HDFS-3077/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/META-INF/services/org.apache.hadoop.security.token.TokenRenewer
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-3077/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/META-INF/services/org.apache.hadoop.security.token.TokenRenewer?rev=1396918&r1=1396917&r2=1396918&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-3077/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/META-INF/services/org.apache.hadoop.security.token.TokenRenewer (original)
+++ hadoop/common/branches/HDFS-3077/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/META-INF/services/org.apache.hadoop.security.token.TokenRenewer Thu Oct 11 06:14:26 2012
@@ -1 +1 @@
-org.apache.hadoop.tools.TestDelegationTokenFetcher$FakeRenewer
+org.apache.hadoop.tools.FakeRenewer