瀏覽代碼

HADOOP-5114. Remove timeout for accept() in DataNode. This makes accept()
fail in JDK on Windows and causes many tests to fail. (Raghu Angadi)

git-svn-id: https://svn.apache.org/repos/asf/hadoop/core/branches/branch-0.18@741706 13f79535-47bb-0310-9956-ffa450edef68

Raghu Angadi 16 年之前
父節點
當前提交
409ed7e493

+ 7 - 0
CHANGES.txt

@@ -1,5 +1,12 @@
 Hadoop Change Log
 
+Release 0.18.4 - Unreleased
+
+  BUG FIXES
+
+    HADOOP-5114. Remove timeout for accept() in DataNode. This makes accept() 
+    fail in JDK on Windows and causes many tests to fail. (Raghu Angadi)
+
 Release 0.18.3 - 2009-01-27
 
   IMPROVEMENTS

+ 4 - 7
src/hdfs/org/apache/hadoop/dfs/DataNode.java

@@ -316,7 +316,6 @@ public class DataNode extends Configured
           ServerSocketChannel.open().socket() : new ServerSocket();
     Server.bind(ss, socAddr, 0);
     ss.setReceiveBufferSize(DEFAULT_DATA_SOCKET_SIZE); 
-    ss.setSoTimeout(conf.getInt("dfs.dataXceiver.timeoutInMS", 30000)); //30s
     // adjust machine name with the actual port
     tmpPort = ss.getLocalPort();
     selfAddr = new InetSocketAddress(ss.getInetAddress().getHostAddress(),
@@ -589,6 +588,10 @@ public class DataNode extends Configured
           } catch (InterruptedException e) {}
         }
       }
+      try {
+        this.dataXceiveServer.join();
+      } catch (InterruptedException ie) {
+      }
     }
     
     RPC.stopProxy(namenode); // stop the RPC threads
@@ -2976,12 +2979,6 @@ public class DataNode extends Configured
       }
     }
         
-    // wait for dataXceiveServer to terminate
-    try {
-      this.dataXceiveServer.join();
-    } catch (InterruptedException ie) {
-    }
-        
     LOG.info(dnRegistration + ":Finishing DataNode in: "+data);
     shutdown();
   }

+ 9 - 1
src/test/org/apache/hadoop/dfs/TestDiskError.java

@@ -37,10 +37,18 @@ import junit.framework.TestCase;
 /** Test if a datanode can correctly handle errors during block read/write*/
 public class TestDiskError extends TestCase {
   public void testShutdown() throws Exception {
+    if (System.getProperty("os.name").startsWith("Windows")) {
+      /**
+       * This test depends on OS not allowing file creations on a directory
+       * that does not have write permissions for the user. Apparently it is 
+       * not the case on Windows (at least under Cygwin), and possibly AIX.
+       * This is disabled on Windows.
+       */
+      return;
+    }
     // bring up a cluster of 3
     Configuration conf = new Configuration();
     conf.setLong("dfs.block.size", 512L);
-    conf.setInt("dfs.dataXceiver.timeoutInMS", 1000);
     MiniDFSCluster cluster = new MiniDFSCluster(conf, 3, true, null);
     cluster.waitActive();
     FileSystem fs = cluster.getFileSystem();

+ 12 - 2
src/test/org/apache/hadoop/dfs/TestOverReplicatedBlocks.java

@@ -11,6 +11,7 @@ import org.apache.hadoop.dfs.MiniDFSCluster;
 import org.apache.hadoop.dfs.TestDatanodeBlockScanner;
 import org.apache.hadoop.dfs.Block;
 import org.apache.hadoop.dfs.DatanodeID;
+import org.apache.hadoop.dfs.MiniDFSCluster.DataNodeProperties;
 
 import junit.framework.TestCase;
 
@@ -35,11 +36,20 @@ public class TestOverReplicatedBlocks extends TestCase {
       // corrupt the block on datanode 0
       Block block = DFSTestUtil.getFirstBlock(fs, fileName);
       TestDatanodeBlockScanner.corruptReplica(block.getBlockName(), 0);
+      DataNodeProperties dnProps = cluster.stopDataNode(0);
+      // remove block scanner log to trigger block scanning
       File scanLog = new File(System.getProperty("test.build.data"),
           "dfs/data/data1/current/dncp_block_verification.log.curr");
-      assertTrue(scanLog.delete()); 
+      //wait for one minute for deletion to succeed;
+      for(int i=0; !scanLog.delete(); i++) {
+        assertTrue("Could not delete log file in one minute", i < 60);
+        try {
+          Thread.sleep(1000);
+        } catch (InterruptedException ignored) {}
+      }
+      
       // restart the datanode so the corrupt replica will be detected
-      cluster.restartDataNode(0);
+      cluster.restartDataNode(dnProps);
       DFSTestUtil.waitReplication(fs, fileName, (short)2);
       
       final DatanodeID corruptDataNode =