소스 검색

HADOOP-1044. Fix HDFS's TestDecommission to not spuriously fail. Contributed by Wendy.

git-svn-id: https://svn.apache.org/repos/asf/lucene/hadoop/trunk@512904 13f79535-47bb-0310-9956-ffa450edef68
Doug Cutting 18 년 전
부모
커밋
8ecd87b4d2
2개의 변경된 파일29개의 추가작업 그리고 9개의 파일을 삭제
  1. 3 0
      CHANGES.txt
  2. 26 9
      src/test/org/apache/hadoop/dfs/TestDecommission.java

+ 3 - 0
CHANGES.txt

@@ -153,6 +153,9 @@ Trunk (unreleased changes)
 45. HADOOP-928.  Make checksums optional per FileSystem.
     (Hairong Kuang via cutting)
 
+46. HADOOP-1044.  Fix HDFS's TestDecommission to not spuriously fail.
+    (Wendy Chien via cutting)
+
 
 Release 0.11.2 - 2007-02-16
 

+ 26 - 9
src/test/org/apache/hadoop/dfs/TestDecommission.java

@@ -20,6 +20,8 @@ package org.apache.hadoop.dfs;
 import junit.framework.TestCase;
 import java.io.*;
 import java.util.Random;
+import java.util.ArrayList;
+import java.util.Iterator;
 import java.net.*;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
@@ -34,23 +36,35 @@ public class TestDecommission extends TestCase {
   static final long seed = 0xDEADBEEFL;
   static final int blockSize = 8192;
   static final int fileSize = 16384;
-  static final int numDatanodes = 5;
+  static final int numIterations = 2;
+  static final int numDatanodes = numIterations + 3;
+
 
   Random myrand = new Random();
   Path hostsFile;
   Path excludeFile;
 
+  ArrayList<String> decommissionedNodes = new ArrayList<String>(numIterations);
+
   private enum NodeState {NORMAL, DECOMMISSION_INPROGRESS, DECOMMISSIONED; }
 
-  private void writeConfigFile(FileSystem fs, Path name, String node) 
+  private void writeConfigFile(FileSystem fs, Path name, ArrayList<String> nodes) 
       throws IOException {
+
     // delete if it already exists
     if (fs.exists(name)) {
       fs.delete(name);
     }
+
     FSDataOutputStream stm = fs.create(name);
-    stm.writeBytes(node);
-    stm.writeBytes("\n");
+    
+    if (nodes != null) {
+      for (Iterator<String> it = nodes.iterator(); it.hasNext();) {
+        String node = it.next();
+        stm.writeBytes(node);
+        stm.writeBytes("\n");
+      }
+    }
     stm.close();
   }
 
@@ -149,8 +163,10 @@ public class TestDecommission extends TestCase {
     String nodename = info[index].getName();
     System.out.println("Decommissioning node: " + nodename);
 
-    // write nodename into the exclude file. 
-    writeConfigFile(localFileSys, excludeFile, nodename);
+    // write nodename into the exclude file.
+    ArrayList<String> nodes = (ArrayList<String>)decommissionedNodes.clone();
+    nodes.add(nodename);
+    writeConfigFile(localFileSys, excludeFile, nodes);
     dfs.refreshNodes();
     return nodename;
   }
@@ -163,7 +179,7 @@ public class TestDecommission extends TestCase {
     DistributedFileSystem dfs = (DistributedFileSystem) filesys;
 
     System.out.println("Commissioning nodes.");
-    writeConfigFile(localFileSys, excludeFile, "");
+    writeConfigFile(localFileSys, excludeFile, null);
     dfs.refreshNodes();
   }
 
@@ -231,7 +247,7 @@ public class TestDecommission extends TestCase {
     hostsFile = new Path(dir, "hosts");
     excludeFile = new Path(dir, "exclude");
     conf.set("dfs.hosts.exclude", excludeFile.toString());
-    writeConfigFile(localFileSys, excludeFile, "");
+    writeConfigFile(localFileSys, excludeFile, null);
 
     MiniDFSCluster cluster = new MiniDFSCluster(65312, conf, numDatanodes, false);
     // Now wait for 15 seconds to give datanodes chance to register
@@ -250,7 +266,7 @@ public class TestDecommission extends TestCase {
     DistributedFileSystem dfs = (DistributedFileSystem) fileSys;
 
     try {
-      for (int iteration = 0; iteration < 2; iteration++) {
+      for (int iteration = 0; iteration < numIterations; iteration++) {
         //
         // Decommission one node. Verify that node is decommissioned.
         // Verify that replication factor of file has increased from 3
@@ -265,6 +281,7 @@ public class TestDecommission extends TestCase {
         commissionNode(fileSys, localFileSys, downnode);
         waitNodeState(fileSys, downnode, NodeState.NORMAL);
         downnode  = decommissionNode(client, fileSys, localFileSys);
+        decommissionedNodes.add(downnode);
         waitNodeState(fileSys, downnode, NodeState.DECOMMISSIONED);
         checkFile(fileSys, file1, 3, downnode);
         cleanupFile(fileSys, file1);