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 ar...@apache.org on 2013/09/06 01:01:02 UTC

svn commit: r1520454 - in /hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common: ./ src/main/docs/ src/main/java/ src/main/java/org/apache/hadoop/fs/ src/main/java/org/apache/hadoop/ipc/ src/main/java/org/apache/hadoop/util/ src/test/co...

Author: arp
Date: Thu Sep  5 23:01:00 2013
New Revision: 1520454

URL: http://svn.apache.org/r1520454
Log:
Merging r1519884 through r1520449 from trunk to branch HDFS-2832

Modified:
    hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/CHANGES.txt   (contents, props changed)
    hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/main/docs/   (props changed)
    hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/main/java/   (props changed)
    hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileUtil.java
    hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Stat.java
    hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java
    hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RetryCache.java
    hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/VersionInfo.java
    hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/test/core/   (props changed)
    hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFileUtil.java
    hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestStat.java
    hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestIPC.java

Modified: hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/CHANGES.txt?rev=1520454&r1=1520453&r2=1520454&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/CHANGES.txt (original)
+++ hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/CHANGES.txt Thu Sep  5 23:01:00 2013
@@ -330,6 +330,9 @@ Release 2.3.0 - UNRELEASED
     HADOOP-9889. Refresh the Krb5 configuration when creating a new kdc in
     Hadoop-MiniKDC (Wei Yan via Sandy Ryza)
 
+    HADOOP-9915.  o.a.h.fs.Stat support on Mac OS X  (Binglin Chang via Colin
+    Patrick McCabe)
+
   OPTIMIZATIONS
 
     HADOOP-9748. Reduce blocking on UGI.ensureInitialized (daryn)
@@ -360,6 +363,8 @@ Release 2.3.0 - UNRELEASED
     HADOOP-9909. org.apache.hadoop.fs.Stat should permit other LANG.
     (Shinichi Yamashita via Andrew Wang)
 
+    HADOOP-9908. Fix NPE when versioninfo properties file is missing (todd)
+
 Release 2.1.1-beta - UNRELEASED
 
   INCOMPATIBLE CHANGES
@@ -406,6 +411,8 @@ Release 2.1.1-beta - UNRELEASED
 
   BUG FIXES
 
+    HADOOP-9916. Fix race in ipc.Client retry. (Binglin Chang via llu)
+
     HADOOP-9768. chown and chgrp reject users and groups with spaces on platforms
     where spaces are otherwise acceptable. (cnauroth)
 
@@ -444,6 +451,11 @@ Release 2.1.1-beta - UNRELEASED
     HADOOP-9774. RawLocalFileSystem.listStatus() return absolute paths when
     input path is relative on Windows. (Shanyu Zhao via ivanmi)
 
+    HADOOP-9924. FileUtil.createJarWithClassPath() does not generate relative
+    classpath correctly. (Shanyu Zhao via ivanmi)
+
+    HADOOP-9932. Improper synchronization in RetryCache. (kihwal)
+
 Release 2.1.0-beta - 2013-08-22
 
   INCOMPATIBLE CHANGES

Propchange: hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/CHANGES.txt
------------------------------------------------------------------------------
  Merged /hadoop/common/trunk/hadoop-common-project/hadoop-common/CHANGES.txt:r1519884-1520449

Propchange: hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/main/docs/
------------------------------------------------------------------------------
  Merged /hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/docs:r1519884-1520449

Propchange: hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/main/java/
------------------------------------------------------------------------------
  Merged /hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java:r1519884-1520449

Modified: hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileUtil.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileUtil.java?rev=1520454&r1=1520453&r2=1520454&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileUtil.java (original)
+++ hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileUtil.java Thu Sep  5 23:01:00 2013
@@ -1252,7 +1252,14 @@ public class FileUtil {
         }
       } else {
         // Append just this entry
-        String classPathEntryUrl = new File(classPathEntry).toURI().toURL()
+        File fileCpEntry = null;
+        if(!new Path(classPathEntry).isAbsolute()) {
+          fileCpEntry = new File(workingDir, classPathEntry);
+        }
+        else {
+          fileCpEntry = new File(classPathEntry);
+        }
+        String classPathEntryUrl = fileCpEntry.toURI().toURL()
           .toExternalForm();
 
         // File.toURI only appends trailing '/' if it can determine that it is a

Modified: hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Stat.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Stat.java?rev=1520454&r1=1520453&r2=1520454&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Stat.java (original)
+++ hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Stat.java Thu Sep  5 23:01:00 2013
@@ -80,7 +80,7 @@ public class Stat extends Shell {
    * @return
    */
   public static boolean isAvailable() {
-    if (Shell.LINUX || Shell.FREEBSD) {
+    if (Shell.LINUX || Shell.FREEBSD || Shell.MAC) {
       return true;
     }
     return false;
@@ -100,7 +100,7 @@ public class Stat extends Shell {
     if (Shell.LINUX) {
       return new String[] {
           "stat", derefFlag + "c", "%s,%F,%Y,%X,%a,%U,%G,%N", path.toString() };
-    } else if (Shell.FREEBSD) {
+    } else if (Shell.FREEBSD || Shell.MAC) {
       return new String[] {
           "stat", derefFlag + "f", "%z,%HT,%m,%a,%Op,%Su,%Sg,`link' -> `%Y'",
           path.toString() };

Modified: hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java?rev=1520454&r1=1520453&r2=1520454&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java (original)
+++ hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java Thu Sep  5 23:01:00 2013
@@ -1063,8 +1063,8 @@ public class Client {
         if (status == RpcStatusProto.SUCCESS) {
           Writable value = ReflectionUtils.newInstance(valueClass, conf);
           value.readFields(in);                 // read value
-          call.setRpcResponse(value);
           calls.remove(callId);
+          call.setRpcResponse(value);
           
           // verify that length was correct
           // only for ProtobufEngine where len can be verified easily
@@ -1098,8 +1098,8 @@ public class Client {
                   new RemoteException(exceptionClassName, errorMsg) :
               new RemoteException(exceptionClassName, errorMsg, erCode));
           if (status == RpcStatusProto.ERROR) {
-            call.setException(re);
             calls.remove(callId);
+            call.setException(re);
           } else if (status == RpcStatusProto.FATAL) {
             // Close the connection
             markClosed(re);
@@ -1166,8 +1166,8 @@ public class Client {
       Iterator<Entry<Integer, Call>> itor = calls.entrySet().iterator() ;
       while (itor.hasNext()) {
         Call c = itor.next().getValue(); 
+        itor.remove();
         c.setException(closeException); // local exception
-        itor.remove();         
       }
     }
   }

Modified: hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RetryCache.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RetryCache.java?rev=1520454&r1=1520453&r2=1520454&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RetryCache.java (original)
+++ hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RetryCache.java Thu Sep  5 23:01:00 2013
@@ -76,6 +76,12 @@ public class RetryCache {
       this.expirationTime = expirationTime;
     }
 
+    CacheEntry(byte[] clientId, int callId, long expirationTime,
+        boolean success) {
+      this(clientId, callId, expirationTime);
+      this.state = success ? SUCCESS : FAILED;
+    }
+
     private static int hashCode(long value) {
       return (int)(value ^ (value >>> 32));
     }
@@ -147,6 +153,12 @@ public class RetryCache {
       this.payload = payload;
     }
 
+    CacheEntryWithPayload(byte[] clientId, int callId, Object payload,
+        long expirationTime, boolean success) {
+     super(clientId, callId, expirationTime, success);
+     this.payload = payload;
+   }
+
     /** Override equals to avoid findbugs warnings */
     @Override
     public boolean equals(Object obj) {
@@ -253,18 +265,20 @@ public class RetryCache {
    */
   public void addCacheEntry(byte[] clientId, int callId) {
     CacheEntry newEntry = new CacheEntry(clientId, callId, System.nanoTime()
-        + expirationTime);
-    newEntry.completed(true);
-    set.put(newEntry);
+        + expirationTime, true);
+    synchronized(this) {
+      set.put(newEntry);
+    }
   }
   
   public void addCacheEntryWithPayload(byte[] clientId, int callId,
       Object payload) {
-    CacheEntry newEntry = new CacheEntryWithPayload(clientId, callId, payload,
-        System.nanoTime() + expirationTime);
     // since the entry is loaded from editlog, we can assume it succeeded.    
-    newEntry.completed(true);
-    set.put(newEntry);
+    CacheEntry newEntry = new CacheEntryWithPayload(clientId, callId, payload,
+        System.nanoTime() + expirationTime, true);
+    synchronized(this) {
+      set.put(newEntry);
+    }
   }
 
   private static CacheEntry newEntry(long expirationTime) {

Modified: hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/VersionInfo.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/VersionInfo.java?rev=1520454&r1=1520453&r2=1520454&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/VersionInfo.java (original)
+++ hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/VersionInfo.java Thu Sep  5 23:01:00 2013
@@ -48,6 +48,9 @@ public class VersionInfo {
     try {
       InputStream is = Thread.currentThread().getContextClassLoader()
         .getResourceAsStream(versionInfoFile);
+      if (is == null) {
+        throw new IOException("Resource not found");
+      }
       info.load(is);
     } catch (IOException ex) {
       LogFactory.getLog(getClass()).warn("Could not read '" + 

Propchange: hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/test/core/
------------------------------------------------------------------------------
  Merged /hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/core:r1519884-1520449

Modified: hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFileUtil.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFileUtil.java?rev=1520454&r1=1520453&r2=1520454&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFileUtil.java (original)
+++ hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFileUtil.java Thu Sep  5 23:01:00 2013
@@ -782,14 +782,23 @@ public class TestFileUtil {
             expectedClassPaths.add(wildcardMatch.toURI().toURL()
               .toExternalForm());
           }
-        } else if (nonExistentSubdir.equals(classPath)) {
-          // expect to maintain trailing path separator if present in input, even
-          // if directory doesn't exist yet
-          expectedClassPaths.add(new File(classPath).toURI().toURL()
-            .toExternalForm() + Path.SEPARATOR);
         } else {
-          expectedClassPaths.add(new File(classPath).toURI().toURL()
-            .toExternalForm());
+          File fileCp = null;
+          if(!new Path(classPath).isAbsolute()) {
+            fileCp = new File(tmp, classPath);
+          }
+          else {
+            fileCp = new File(classPath);
+          }
+          if (nonExistentSubdir.equals(classPath)) {
+            // expect to maintain trailing path separator if present in input, even
+            // if directory doesn't exist yet
+            expectedClassPaths.add(fileCp.toURI().toURL()
+              .toExternalForm() + Path.SEPARATOR);
+          } else {
+            expectedClassPaths.add(fileCp.toURI().toURL()
+              .toExternalForm());
+          }
         }
       }
       List<String> actualClassPaths = Arrays.asList(classPathAttr.split(" "));

Modified: hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestStat.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestStat.java?rev=1520454&r1=1520453&r2=1520454&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestStat.java (original)
+++ hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestStat.java Thu Sep  5 23:01:00 2013
@@ -19,6 +19,7 @@ package org.apache.hadoop.fs;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.fail;
 
 import java.io.BufferedReader;
@@ -26,10 +27,11 @@ import java.io.FileNotFoundException;
 import java.io.StringReader;
 
 import org.apache.hadoop.conf.Configuration;
+import org.junit.Assume;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
-public class TestStat {
+public class TestStat extends FileSystemTestHelper {
 
   private static Stat stat;
 
@@ -113,6 +115,7 @@ public class TestStat {
 
   @Test(timeout=10000)
   public void testStatFileNotFound() throws Exception {
+    Assume.assumeTrue(Stat.isAvailable());
     try {
       stat.getFileStatus();
       fail("Expected FileNotFoundException");
@@ -125,4 +128,21 @@ public class TestStat {
   public void testStatEnvironment() throws Exception {
     assertEquals(stat.getEnvironment("LANG"), "C");
   }
+
+  @Test(timeout=10000)
+  public void testStat() throws Exception {
+    Assume.assumeTrue(Stat.isAvailable());
+    FileSystem fs = FileSystem.getLocal(new Configuration());
+    Path testDir = new Path(getTestRootPath(fs), "teststat");
+    fs.mkdirs(testDir);
+    Path sub1 = new Path(testDir, "sub1");
+    Path sub2 = new Path(testDir, "sub2");
+    fs.mkdirs(sub1);
+    fs.createSymlink(sub1, sub2, false);
+    FileStatus stat1 = new Stat(sub1, 4096l, false, fs).getFileStatus();
+    FileStatus stat2 = new Stat(sub2, 0, false, fs).getFileStatus();
+    assertTrue(stat1.isDirectory());
+    assertFalse(stat2.isDirectory());
+    fs.delete(testDir, true);
+  }
 }

Modified: hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestIPC.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestIPC.java?rev=1520454&r1=1520453&r2=1520454&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestIPC.java (original)
+++ hadoop/common/branches/HDFS-2832/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestIPC.java Thu Sep  5 23:01:00 2013
@@ -216,13 +216,13 @@ public class TestIPC {
     }
   }
   
-  @Test
+  @Test(timeout=60000)
   public void testSerial() throws IOException, InterruptedException {
-    testSerial(3, false, 2, 5, 100);
-    testSerial(3, true, 2, 5, 10);
+    internalTestSerial(3, false, 2, 5, 100);
+    internalTestSerial(3, true, 2, 5, 10);
   }
 
-  public void testSerial(int handlerCount, boolean handlerSleep, 
+  public void internalTestSerial(int handlerCount, boolean handlerSleep,
                          int clientCount, int callerCount, int callCount)
     throws IOException, InterruptedException {
     Server server = new TestServer(handlerCount, handlerSleep);
@@ -249,7 +249,7 @@ public class TestIPC {
     server.stop();
   }
 	
-  @Test
+  @Test(timeout=60000)
   public void testStandAloneClient() throws IOException {
     Client client = new Client(LongWritable.class, conf);
     InetSocketAddress address = new InetSocketAddress("127.0.0.1", 10);
@@ -383,7 +383,7 @@ public class TestIPC {
     }
   }
 
-  @Test
+  @Test(timeout=60000)
   public void testIOEOnClientWriteParam() throws Exception {
     doErrorTest(IOEOnWriteWritable.class,
         LongWritable.class,
@@ -391,7 +391,7 @@ public class TestIPC {
         LongWritable.class);
   }
   
-  @Test
+  @Test(timeout=60000)
   public void testRTEOnClientWriteParam() throws Exception {
     doErrorTest(RTEOnWriteWritable.class,
         LongWritable.class,
@@ -399,7 +399,7 @@ public class TestIPC {
         LongWritable.class);
   }
 
-  @Test
+  @Test(timeout=60000)
   public void testIOEOnServerReadParam() throws Exception {
     doErrorTest(LongWritable.class,
         IOEOnReadWritable.class,
@@ -407,7 +407,7 @@ public class TestIPC {
         LongWritable.class);
   }
   
-  @Test
+  @Test(timeout=60000)
   public void testRTEOnServerReadParam() throws Exception {
     doErrorTest(LongWritable.class,
         RTEOnReadWritable.class,
@@ -416,7 +416,7 @@ public class TestIPC {
   }
 
   
-  @Test
+  @Test(timeout=60000)
   public void testIOEOnServerWriteResponse() throws Exception {
     doErrorTest(LongWritable.class,
         LongWritable.class,
@@ -424,7 +424,7 @@ public class TestIPC {
         LongWritable.class);
   }
   
-  @Test
+  @Test(timeout=60000)
   public void testRTEOnServerWriteResponse() throws Exception {
     doErrorTest(LongWritable.class,
         LongWritable.class,
@@ -432,7 +432,7 @@ public class TestIPC {
         LongWritable.class);
   }
   
-  @Test
+  @Test(timeout=60000)
   public void testIOEOnClientReadResponse() throws Exception {
     doErrorTest(LongWritable.class,
         LongWritable.class,
@@ -440,7 +440,7 @@ public class TestIPC {
         IOEOnReadWritable.class);
   }
   
-  @Test
+  @Test(timeout=60000)
   public void testRTEOnClientReadResponse() throws Exception {
     doErrorTest(LongWritable.class,
         LongWritable.class,
@@ -453,7 +453,7 @@ public class TestIPC {
    * that a ping should have been sent. This is a reproducer for a
    * deadlock seen in one iteration of HADOOP-6762.
    */
-  @Test
+  @Test(timeout=60000)
   public void testIOEOnWriteAfterPingClient() throws Exception {
     // start server
     Client.setPingInterval(conf, 100);
@@ -481,7 +481,7 @@ public class TestIPC {
    * Test that, if the socket factory throws an IOE, it properly propagates
    * to the client.
    */
-  @Test
+  @Test(timeout=60000)
   public void testSocketFactoryException() throws IOException {
     SocketFactory mockFactory = mock(SocketFactory.class);
     doThrow(new IOException("Injected fault")).when(mockFactory).createSocket();
@@ -503,7 +503,7 @@ public class TestIPC {
    * failure is handled properly. This is a regression test for
    * HADOOP-7428.
    */
-  @Test
+  @Test(timeout=60000)
   public void testRTEDuringConnectionSetup() throws IOException {
     // Set up a socket factory which returns sockets which
     // throw an RTE when setSoTimeout is called.
@@ -544,7 +544,7 @@ public class TestIPC {
     }
   }
   
-  @Test
+  @Test(timeout=60000)
   public void testIpcTimeout() throws IOException {
     // start server
     Server server = new TestServer(1, true);
@@ -566,7 +566,7 @@ public class TestIPC {
         addr, null, null, 3*PING_INTERVAL+MIN_SLEEP_TIME, conf);
   }
 
-  @Test
+  @Test(timeout=60000)
   public void testIpcConnectTimeout() throws IOException {
     // start server
     Server server = new TestServer(1, true);
@@ -670,31 +670,31 @@ public class TestIPC {
     return FD_DIR.list().length;
   }
 
-  @Test
+  @Test(timeout=60000)
   public void testIpcFromHadoop_0_18_13() throws IOException {
     doIpcVersionTest(NetworkTraces.HADOOP_0_18_3_RPC_DUMP,
         NetworkTraces.RESPONSE_TO_HADOOP_0_18_3_RPC);
   }
   
-  @Test
+  @Test(timeout=60000)
   public void testIpcFromHadoop0_20_3() throws IOException {
     doIpcVersionTest(NetworkTraces.HADOOP_0_20_3_RPC_DUMP,
         NetworkTraces.RESPONSE_TO_HADOOP_0_20_3_RPC);
   }
   
-  @Test
+  @Test(timeout=60000)
   public void testIpcFromHadoop0_21_0() throws IOException {
     doIpcVersionTest(NetworkTraces.HADOOP_0_21_0_RPC_DUMP,
         NetworkTraces.RESPONSE_TO_HADOOP_0_21_0_RPC);
   }
   
-  @Test
+  @Test(timeout=60000)
   public void testHttpGetResponse() throws IOException {
     doIpcVersionTest("GET / HTTP/1.0\r\n\r\n".getBytes(),
         Server.RECEIVED_HTTP_REQ_RESPONSE.getBytes());
   }
   
-  @Test
+  @Test(timeout=60000)
   public void testConnectionRetriesOnSocketTimeoutExceptions() throws IOException {
     Configuration conf = new Configuration();
     // set max retries to 0
@@ -720,7 +720,7 @@ public class TestIPC {
    * (1) the rpc server uses the call id/retry provided by the rpc client, and
    * (2) the rpc client receives the same call id/retry from the rpc server.
    */
-  @Test
+  @Test(timeout=60000)
   public void testCallIdAndRetry() throws IOException {
     final CallInfo info = new CallInfo();
 
@@ -772,7 +772,7 @@ public class TestIPC {
   /**
    * Test the retry count while used in a retry proxy.
    */
-  @Test
+  @Test(timeout=60000)
   public void testRetryProxy() throws IOException {
     final Client client = new Client(LongWritable.class, conf);
     
@@ -785,7 +785,9 @@ public class TestIPC {
       }
     };
 
-    final int totalRetry = 256;
+    // try more times, so it is easier to find race condition bug
+    // 10000 times runs about 6s on a core i7 machine
+    final int totalRetry = 10000;
     DummyProtocol proxy = (DummyProtocol) Proxy.newProxyInstance(
         DummyProtocol.class.getClassLoader(),
         new Class[] { DummyProtocol.class }, new TestInvocationHandler(client,
@@ -807,7 +809,7 @@ public class TestIPC {
   /**
    * Test if the rpc server gets the default retry count (0) from client.
    */
-  @Test
+  @Test(timeout=60000)
   public void testInitialCallRetryCount() throws IOException {
     // Override client to store the call id
     final Client client = new Client(LongWritable.class, conf);
@@ -838,7 +840,7 @@ public class TestIPC {
   /**
    * Test if the rpc server gets the retry count from client.
    */
-  @Test
+  @Test(timeout=60000)
   public void testCallRetryCount() throws IOException {
     final int retryCount = 255;
     // Override client to store the call id
@@ -873,7 +875,7 @@ public class TestIPC {
    * even if multiple threads are using the same client.
  * @throws InterruptedException 
    */
-  @Test
+  @Test(timeout=60000)
   public void testUniqueSequentialCallIds() 
       throws IOException, InterruptedException {
     int serverThreads = 10, callerCount = 100, perCallerCallCount = 100;