You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by cd...@apache.org on 2008/09/26 23:26:15 UTC

svn commit: r699508 - in /hadoop/core/branches/branch-0.19: ./ src/test/org/apache/hadoop/hdfs/

Author: cdouglas
Date: Fri Sep 26 14:26:14 2008
New Revision: 699508

URL: http://svn.apache.org/viewvc?rev=699508&view=rev
Log:
HADOOP-4279. Trace the seeds of random sequences in append unit tests to
make itermitant failures reproducible. Contributed by Tsz Wo (Nicholas), SZE.

Added:
    hadoop/core/branches/branch-0.19/src/test/org/apache/hadoop/hdfs/AppendTestUtil.java
Modified:
    hadoop/core/branches/branch-0.19/CHANGES.txt
    hadoop/core/branches/branch-0.19/src/test/org/apache/hadoop/hdfs/TestDatanodeDeath.java
    hadoop/core/branches/branch-0.19/src/test/org/apache/hadoop/hdfs/TestFileAppend.java
    hadoop/core/branches/branch-0.19/src/test/org/apache/hadoop/hdfs/TestFileAppend2.java
    hadoop/core/branches/branch-0.19/src/test/org/apache/hadoop/hdfs/TestFileCreation.java
    hadoop/core/branches/branch-0.19/src/test/org/apache/hadoop/hdfs/TestLeaseRecovery.java
    hadoop/core/branches/branch-0.19/src/test/org/apache/hadoop/hdfs/TestLeaseRecovery2.java

Modified: hadoop/core/branches/branch-0.19/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/core/branches/branch-0.19/CHANGES.txt?rev=699508&r1=699507&r2=699508&view=diff
==============================================================================
--- hadoop/core/branches/branch-0.19/CHANGES.txt (original)
+++ hadoop/core/branches/branch-0.19/CHANGES.txt Fri Sep 26 14:26:14 2008
@@ -421,6 +421,9 @@
     HADOOP-4006. Clean up FSConstants and move some of the constants to 
     better places. (Sanjay Radia via rangadi)
  
+    HADOOP-4279. Trace the seeds of random sequences in append unit tests to
+    make itermitant failures reproducible. (szetszwo via cdouglas)
+
   OPTIMIZATIONS
 
     HADOOP-3556. Removed lock contention in MD5Hash by changing the 

Added: hadoop/core/branches/branch-0.19/src/test/org/apache/hadoop/hdfs/AppendTestUtil.java
URL: http://svn.apache.org/viewvc/hadoop/core/branches/branch-0.19/src/test/org/apache/hadoop/hdfs/AppendTestUtil.java?rev=699508&view=auto
==============================================================================
--- hadoop/core/branches/branch-0.19/src/test/org/apache/hadoop/hdfs/AppendTestUtil.java (added)
+++ hadoop/core/branches/branch-0.19/src/test/org/apache/hadoop/hdfs/AppendTestUtil.java Fri Sep 26 14:26:14 2008
@@ -0,0 +1,60 @@
+/**
+ * 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.Random;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+
+/** Utilities for append-related tests */ 
+class AppendTestUtil {
+  static final Log LOG = LogFactory.getLog(AppendTestUtil.class);
+
+  private static final Random SEED = new Random();
+  static {
+    final long seed = SEED.nextLong();
+    LOG.info("seed=" + seed);
+    SEED.setSeed(seed);
+  }
+
+  private static final ThreadLocal<Random> RANDOM = new ThreadLocal<Random>() {
+    protected Random initialValue() {
+      final Random r =  new Random();
+      synchronized(SEED) { 
+        final long seed = SEED.nextLong();
+        SEED.setSeed(seed);
+        LOG.info(Thread.currentThread().getName() + ": seed=" + seed);
+      }
+      return r;
+    }
+  };
+  
+  static int nextInt() {return RANDOM.get().nextInt();}
+  static int nextInt(int n) {return RANDOM.get().nextInt(n);}
+  static int nextLong() {return RANDOM.get().nextInt();}
+
+  static byte[] randomBytes(long seed, int size) {
+    LOG.info("seed=" + seed + ", size=" + size);
+    final byte[] b = new byte[size];
+    final Random rand = new Random(seed);
+    rand.nextBytes(b);
+    return b;
+  }
+}
\ No newline at end of file

Modified: hadoop/core/branches/branch-0.19/src/test/org/apache/hadoop/hdfs/TestDatanodeDeath.java
URL: http://svn.apache.org/viewvc/hadoop/core/branches/branch-0.19/src/test/org/apache/hadoop/hdfs/TestDatanodeDeath.java?rev=699508&r1=699507&r2=699508&view=diff
==============================================================================
--- hadoop/core/branches/branch-0.19/src/test/org/apache/hadoop/hdfs/TestDatanodeDeath.java (original)
+++ hadoop/core/branches/branch-0.19/src/test/org/apache/hadoop/hdfs/TestDatanodeDeath.java Fri Sep 26 14:26:14 2008
@@ -17,28 +17,23 @@
  */
 package org.apache.hadoop.hdfs;
 
+import java.io.IOException;
+
 import junit.framework.TestCase;
-import java.io.*;
-import java.util.Random;
-import java.util.Collection;
 
+import org.apache.commons.logging.impl.Log4JLogger;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FsShell;
+import org.apache.hadoop.fs.BlockLocation;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.util.StringUtils;
-import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
-import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.namenode.LeaseManager;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
-import org.apache.hadoop.hdfs.server.datanode.DataNode;
-
 import org.apache.log4j.Level;
-import org.apache.commons.logging.impl.Log4JLogger;
 
 /**
  * This class tests that a file need not be closed before its
@@ -135,47 +130,13 @@
   // writes to file
   //
   private void writeFile(FSDataOutputStream stm, long seed) throws IOException {
-    byte[] buffer = new byte[fileSize];
-    Random rand = new Random(seed);
-    rand.nextBytes(buffer);
+    byte[] buffer = AppendTestUtil.randomBytes(seed, fileSize);
+
     int mid = fileSize/2;
     stm.write(buffer, 0, mid);
     stm.write(buffer, mid, fileSize - mid);
   }
 
-  /**
-   * For blocks that reside on the nodes that are down, verify that their
-   * replication factor is 1 more than the specified one.
-   */
-  private DatanodeInfo[] getPipeline(FileSystem fileSys, Path name, 
-                                     short repl,
-                                     int blockNumber) 
-                                     throws IOException {
-    // need a raw stream
-    assertTrue("Not HDFS:"+fileSys.getUri(), 
-               fileSys instanceof DistributedFileSystem);
-
-    DFSClient.DFSDataInputStream dis = (DFSClient.DFSDataInputStream)
-      ((DistributedFileSystem)fileSys).open(name);
-    Collection<LocatedBlock> dinfo = dis.getAllBlocks();
-    int num = 0;
-    DatanodeInfo[] status = null;
-
-    for (LocatedBlock blk : dinfo) { // for each block
-      int hasdown = 0;
-      DatanodeInfo[] nodes = blk.getLocations();
-      for (int j = 0; j < nodes.length; j++) {     // for each replica
-        System.out.println("Block " + blk.getBlock() + " replica " +
-                           nodes[j].getName());
-      }
-      if (blockNumber == num) {
-        status = nodes;
-      }
-      num++;
-    }
-    return status;
-  }
-
   //
   // verify that the data written are sane
   // 
@@ -226,18 +187,17 @@
       }
     }
     FSDataInputStream stm = fileSys.open(name);
-    byte[] expected = new byte[filesize];
-    Random rand = new Random(seed);
-    rand.nextBytes(expected);
+    final byte[] expected = AppendTestUtil.randomBytes(seed, fileSize);
+
     // do a sanity check. Read the file
     byte[] actual = new byte[filesize];
     stm.readFully(0, actual);
     checkData(actual, 0, expected, "Read 1");
   }
 
-  private void checkData(byte[] actual, int from, byte[] expected, String message) {
+  private static void checkData(byte[] actual, int from, byte[] expected, String message) {
     for (int idx = 0; idx < actual.length; idx++) {
-      this.assertEquals(message+" byte "+(from+idx)+" differs. expected "+
+      assertEquals(message+" byte "+(from+idx)+" differs. expected "+
                         expected[from+idx]+" actual "+actual[idx],
                         actual[idx], expected[from+idx]);
       actual[idx] = 0;
@@ -252,13 +212,11 @@
    * test will fail).
    */
   class Modify extends Thread {
-    Random rand;
     volatile boolean running;
     MiniDFSCluster cluster;
     Configuration conf;
 
     Modify(Configuration conf, MiniDFSCluster cluster) {
-      rand = new Random();
       running = true;
       this.cluster = cluster;
       this.conf = conf;
@@ -292,7 +250,7 @@
 
         for (int i = 0; i < replication - 1; i++) {
           // pick a random datanode to shutdown
-          int victim = rand.nextInt(numDatanodes);
+          int victim = AppendTestUtil.nextInt(numDatanodes);
           try {
             System.out.println("Stopping datanode " + victim);
             cluster.restartDataNode(victim);
@@ -389,13 +347,9 @@
     MiniDFSCluster cluster = new MiniDFSCluster(conf, myMaxNodes, true, null);
     cluster.waitActive();
     FileSystem fs = cluster.getFileSystem();
-    DistributedFileSystem dfs = (DistributedFileSystem) fs;
     short repl = 3;
 
     Path filename = new Path("simpletest.dat");
-    Random rand = new Random();
-    long myseed = rand.nextInt();
-    rand = new Random(myseed);
     try {
 
       // create a file and write one block of data
@@ -405,12 +359,11 @@
                                              (stm.getWrappedStream());
 
       // these are test settings
-      int bytesPerChecksum = conf.getInt( "io.bytes.per.checksum", 512); 
       dfstream.setChunksPerPacket(5);
       dfstream.setArtificialSlowdown(3000);
 
-      byte[] buffer = new byte[fileSize];
-      rand.nextBytes(buffer);
+      final long myseed = AppendTestUtil.nextLong();
+      byte[] buffer = AppendTestUtil.randomBytes(myseed, fileSize);
       int mid = fileSize/4;
       stm.write(buffer, 0, mid);
 
@@ -426,7 +379,7 @@
       }
 
       if (targets == null) {
-        int victim = rand.nextInt(myMaxNodes);
+        int victim = AppendTestUtil.nextInt(myMaxNodes);
         System.out.println("SimpleTest stopping datanode random " + victim);
         cluster.stopDataNode(victim);
       } else {

Modified: hadoop/core/branches/branch-0.19/src/test/org/apache/hadoop/hdfs/TestFileAppend.java
URL: http://svn.apache.org/viewvc/hadoop/core/branches/branch-0.19/src/test/org/apache/hadoop/hdfs/TestFileAppend.java?rev=699508&r1=699507&r2=699508&view=diff
==============================================================================
--- hadoop/core/branches/branch-0.19/src/test/org/apache/hadoop/hdfs/TestFileAppend.java (original)
+++ hadoop/core/branches/branch-0.19/src/test/org/apache/hadoop/hdfs/TestFileAppend.java Fri Sep 26 14:26:14 2008
@@ -20,7 +20,6 @@
 import junit.framework.TestCase;
 import java.io.*;
 import java.net.*;
-import java.util.Random;
 import java.util.List;
 
 import org.apache.hadoop.conf.Configuration;
@@ -42,20 +41,20 @@
  * support HDFS appends.
  */
 public class TestFileAppend extends TestCase {
-  static final long seed = 0xDEADBEEFL;
   static final int blockSize = 1024;
   static final int numBlocks = 10;
   static final int fileSize = numBlocks * blockSize + 1;
   boolean simulatedStorage = false;
-  byte[] fileContents = null;
+
+  private long seed;
+  private byte[] fileContents = null;
 
   //
   // create a buffer that contains the entire test file data.
   //
   private void initBuffer(int size) {
-    Random rand = new Random(seed);
-    fileContents = new byte[size];
-    rand.nextBytes(fileContents);
+    seed = AppendTestUtil.nextLong();
+    fileContents = AppendTestUtil.randomBytes(seed, size);
   }
 
   /*
@@ -73,9 +72,7 @@
   // writes to file but does not close it
   //
   private void writeFile(FSDataOutputStream stm) throws IOException {
-    byte[] buffer = new byte[fileSize];
-    Random rand = new Random(seed);
-    rand.nextBytes(buffer);
+    byte[] buffer = AppendTestUtil.randomBytes(seed, fileSize);
     stm.write(buffer);
   }
 

Modified: hadoop/core/branches/branch-0.19/src/test/org/apache/hadoop/hdfs/TestFileAppend2.java
URL: http://svn.apache.org/viewvc/hadoop/core/branches/branch-0.19/src/test/org/apache/hadoop/hdfs/TestFileAppend2.java?rev=699508&r1=699507&r2=699508&view=diff
==============================================================================
--- hadoop/core/branches/branch-0.19/src/test/org/apache/hadoop/hdfs/TestFileAppend2.java (original)
+++ hadoop/core/branches/branch-0.19/src/test/org/apache/hadoop/hdfs/TestFileAppend2.java Fri Sep 26 14:26:14 2008
@@ -20,7 +20,6 @@
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.Random;
 
 import junit.framework.TestCase;
 
@@ -57,13 +56,12 @@
     ((Log4JLogger)DFSClient.LOG).getLogger().setLevel(Level.ALL);
   }
 
-  static final long seed = 0xDEADBEEFL;
   static final int blockSize = 1024;
   static final int numBlocks = 5;
   static final int fileSize = numBlocks * blockSize + 1;
   boolean simulatedStorage = false;
-  byte[] fileContents = null;
-  Random rand = new Random(seed);
+
+  private byte[] fileContents = null;
 
   int numDatanodes = 5;
   int numberOfFiles = 50;
@@ -82,8 +80,8 @@
   // create a buffer that contains the entire test file data.
   //
   private void initBuffer(int size) {
-    fileContents = new byte[size];
-    rand.nextBytes(fileContents);
+    long seed = AppendTestUtil.nextLong();
+    fileContents = AppendTestUtil.randomBytes(seed, size);
   }
 
   /*
@@ -209,7 +207,7 @@
         int offset = 0;
         try {
           out = fs.create(foo);
-          int len = 10 + rand.nextInt(100);
+          int len = 10 + AppendTestUtil.nextInt(100);
           out.write(fileContents, offset, len);
           offset += len;
         }
@@ -225,7 +223,7 @@
         out = null;
         try {
           out = fs.append(foo);
-          int len = 10 + rand.nextInt(100);
+          int len = 10 + AppendTestUtil.nextInt(100);
           out.write(fileContents, offset, len);
           offset += len;
         }
@@ -288,7 +286,7 @@
             System.out.println("Completed write to almost all files.");
             return;  
           }
-          int index = rand.nextInt(testFiles.size());
+          int index = AppendTestUtil.nextInt(testFiles.size());
           testfile = testFiles.remove(index);
         }
 
@@ -313,7 +311,7 @@
           if (left <= 0) {
             left = 1;
           }
-          sizeToAppend = rand.nextInt(left);
+          sizeToAppend = AppendTestUtil.nextInt(left);
 
           System.out.println("Workload thread " + id +
                              " appending " + sizeToAppend + " bytes " +
@@ -386,7 +384,7 @@
       // Insert them into a linked list.
       //
       for (int i = 0; i < numberOfFiles; i++) {
-        short replication = (short)(rand.nextInt(numDatanodes) + 1);
+        short replication = (short)(AppendTestUtil.nextInt(numDatanodes) + 1);
         Path testFile = new Path("/" + i + ".dat");
         FSDataOutputStream stm = createFile(fs, testFile, replication);
         stm.close();

Modified: hadoop/core/branches/branch-0.19/src/test/org/apache/hadoop/hdfs/TestFileCreation.java
URL: http://svn.apache.org/viewvc/hadoop/core/branches/branch-0.19/src/test/org/apache/hadoop/hdfs/TestFileCreation.java?rev=699508&r1=699507&r2=699508&view=diff
==============================================================================
--- hadoop/core/branches/branch-0.19/src/test/org/apache/hadoop/hdfs/TestFileCreation.java (original)
+++ hadoop/core/branches/branch-0.19/src/test/org/apache/hadoop/hdfs/TestFileCreation.java Fri Sep 26 14:26:14 2008
@@ -19,7 +19,6 @@
 
 import java.io.*;
 import java.net.InetSocketAddress;
-import java.util.Random;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.*;
@@ -76,19 +75,14 @@
   // writes to file but does not close it
   //
   static void writeFile(FSDataOutputStream stm) throws IOException {
-    byte[] buffer = new byte[fileSize];
-    Random rand = new Random(seed);
-    rand.nextBytes(buffer);
-    stm.write(buffer);
+    writeFile(stm, fileSize);
   }
 
   //
   // writes specified bytes to file.
   //
   static void writeFile(FSDataOutputStream stm, int size) throws IOException {
-    byte[] buffer = new byte[fileSize];
-    Random rand = new Random(seed);
-    rand.nextBytes(buffer);
+    byte[] buffer = AppendTestUtil.randomBytes(seed, size);
     stm.write(buffer, 0, size);
   }
 
@@ -119,14 +113,14 @@
       }
     }
     FSDataInputStream stm = fileSys.open(name);
-    byte[] expected = new byte[numBlocks * blockSize];
+    final byte[] expected;
     if (simulatedStorage) {
+      expected = new byte[numBlocks * blockSize];
       for (int i= 0; i < expected.length; i++) {  
         expected[i] = SimulatedFSDataset.DEFAULT_DATABYTE;
       }
     } else {
-      Random rand = new Random(seed);
-      rand.nextBytes(expected);
+      expected = AppendTestUtil.randomBytes(seed, numBlocks*blockSize);
     }
     // do a sanity check. Read the file
     byte[] actual = new byte[numBlocks * blockSize];
@@ -157,9 +151,7 @@
                          " len " + locations[idx].getLength());
     }
 
-    byte[] expected = new byte[fileSize];
-    Random rand = new Random(seed);
-    rand.nextBytes(expected);
+    byte[] expected = AppendTestUtil.randomBytes(seed, fileSize);
     FSDataInputStream stm = fs.open(name);
     byte[] actual = new byte[fileSize];
     stm.readFully(0, actual);
@@ -344,9 +336,7 @@
 
       // write 1 byte to file. 
       // This should fail because all datanodes are dead.
-      byte[] buffer = new byte[1];
-      Random rand = new Random(seed);
-      rand.nextBytes(buffer);
+      byte[] buffer = AppendTestUtil.randomBytes(seed, 1);
       try {
         stm.write(buffer);
         stm.close();
@@ -542,9 +532,7 @@
 
       // write 1 byte to file.  This should succeed because the 
       // namenode should have persisted leases.
-      byte[] buffer = new byte[1];
-      Random rand = new Random(seed);
-      rand.nextBytes(buffer);
+      byte[] buffer = AppendTestUtil.randomBytes(seed, 1);
       stm.write(buffer);
       stm.close();
       stm2.write(buffer);
@@ -746,7 +734,7 @@
         Thread.sleep(1000);                       // let writers get started
 
         //stop a datanode, it should have least recover.
-        cluster.stopDataNode(new Random().nextInt(REPLICATION));
+        cluster.stopDataNode(AppendTestUtil.nextInt(REPLICATION));
         
         //let the slow writer writes a few more seconds
         System.out.println("Wait a few seconds");

Modified: hadoop/core/branches/branch-0.19/src/test/org/apache/hadoop/hdfs/TestLeaseRecovery.java
URL: http://svn.apache.org/viewvc/hadoop/core/branches/branch-0.19/src/test/org/apache/hadoop/hdfs/TestLeaseRecovery.java?rev=699508&r1=699507&r2=699508&view=diff
==============================================================================
--- hadoop/core/branches/branch-0.19/src/test/org/apache/hadoop/hdfs/TestLeaseRecovery.java (original)
+++ hadoop/core/branches/branch-0.19/src/test/org/apache/hadoop/hdfs/TestLeaseRecovery.java Fri Sep 26 14:26:14 2008
@@ -18,10 +18,10 @@
 package org.apache.hadoop.hdfs;
 
 import java.io.IOException;
-import java.util.*;
+import java.util.Arrays;
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.*;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
@@ -33,7 +33,6 @@
 public class TestLeaseRecovery extends junit.framework.TestCase {
   static final int BLOCK_SIZE = 1024;
   static final short REPLICATION_NUM = (short)3;
-  static final Random RANDOM = new Random();
 
   static void checkMetaInfo(Block b, InterDatanodeProtocol idp
       ) throws IOException {
@@ -98,7 +97,7 @@
       int lastblocksize = ORG_FILE_SIZE % BLOCK_SIZE;
       Integer[] newblocksizes = new Integer[REPLICATION_NUM];
       for(int i = 0; i < REPLICATION_NUM; i++) {
-        newblocksizes[i] = RANDOM.nextInt(lastblocksize);
+        newblocksizes[i] = AppendTestUtil.nextInt(lastblocksize);
       }
       DataNode.LOG.info("newblocksizes = " + Arrays.asList(newblocksizes)); 
 
@@ -115,7 +114,7 @@
       cluster.getNameNode().append(filestr, dfs.dfs.clientName);
 
       //block synchronization
-      final int primarydatanodeindex = RANDOM.nextInt(datanodes.length);
+      final int primarydatanodeindex = AppendTestUtil.nextInt(datanodes.length);
       DataNode.LOG.info("primarydatanodeindex  =" + primarydatanodeindex);
       DataNode primary = datanodes[primarydatanodeindex];
       DataNode.LOG.info("primary.dnRegistration=" + primary.dnRegistration);

Modified: hadoop/core/branches/branch-0.19/src/test/org/apache/hadoop/hdfs/TestLeaseRecovery2.java
URL: http://svn.apache.org/viewvc/hadoop/core/branches/branch-0.19/src/test/org/apache/hadoop/hdfs/TestLeaseRecovery2.java?rev=699508&r1=699507&r2=699508&view=diff
==============================================================================
--- hadoop/core/branches/branch-0.19/src/test/org/apache/hadoop/hdfs/TestLeaseRecovery2.java (original)
+++ hadoop/core/branches/branch-0.19/src/test/org/apache/hadoop/hdfs/TestLeaseRecovery2.java Fri Sep 26 14:26:14 2008
@@ -18,38 +18,18 @@
 package org.apache.hadoop.hdfs;
 
 import java.io.IOException;
-import java.util.*;
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.*;
-import org.apache.hadoop.hdfs.protocol.Block;
-import org.apache.hadoop.hdfs.server.datanode.TestInterDatanodeProtocol;
-import org.apache.hadoop.hdfs.server.protocol.InterDatanodeProtocol;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.Path;
 
 public class TestLeaseRecovery2 extends junit.framework.TestCase {
   static final int BLOCK_SIZE = 64;
   static final int FILE_SIZE = 1024;
   static final short REPLICATION_NUM = (short)3;
-  static final Random RANDOM = new Random();
   static byte[] buffer = new byte[FILE_SIZE];
 
-  static void checkMetaInfo(Block b, InterDatanodeProtocol idp
-      ) throws IOException {
-    TestInterDatanodeProtocol.checkMetaInfo(b, idp, null);
-  }
-  
-  static int min(Integer... x) {
-    int m = x[0];
-    for(int i = 1; i < x.length; i++) {
-      if (x[i] < m) {
-        m = x[i];
-      }
-    }
-    return m;
-  }
-
-  /**
-   */
   public void testBlockSynchronization() throws Exception {
     final long softLease = 1000;
     final long hardLease = 60 * 60 *1000;
@@ -67,7 +47,7 @@
       //create a file
       DistributedFileSystem dfs = (DistributedFileSystem)cluster.getFileSystem();
       // create a random file name
-      String filestr = "/foo" + RANDOM.nextInt();
+      String filestr = "/foo" + AppendTestUtil.nextInt();
       Path filepath = new Path(filestr);
       FSDataOutputStream stm = dfs.create(filepath, true,
                                  dfs.getConf().getInt("io.file.buffer.size", 4096),
@@ -75,7 +55,7 @@
       assertTrue(dfs.dfs.exists(filestr));
 
       // write random number of bytes into it.
-      int size = RANDOM.nextInt(FILE_SIZE);
+      int size = AppendTestUtil.nextInt(FILE_SIZE);
       stm.write(buffer, 0, size);
 
       // sync file