Browse Source

HADOOP-4314. Simulated datanodes should not include blocks that are still
being written in their block report. (Raghu Angadi)


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

Raghu Angadi 16 years ago
parent
commit
70b60cf8b5

+ 3 - 0
CHANGES.txt

@@ -787,6 +787,9 @@ Release 0.18.2 - Unreleased
     HADOOP-3614. Fix a bug that Datanode may use an old GenerationStamp to get
     HADOOP-3614. Fix a bug that Datanode may use an old GenerationStamp to get
     meta file. (szetszwo)
     meta file. (szetszwo)
 
 
+    HADOOP-4314. Simulated datanodes should not include blocks that are still
+    being written in their block report. (Raghu Angadi)
+
 Release 0.18.1 - 2008-09-17
 Release 0.18.1 - 2008-09-17
 
 
   IMPROVEMENTS
   IMPROVEMENTS

+ 5 - 23
src/test/org/apache/hadoop/hdfs/TestReplication.java

@@ -35,8 +35,6 @@ import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.ipc.RPC;
-import org.apache.hadoop.net.NetUtils;
 
 
 /**
 /**
  * This class tests the replication of a DFS file.
  * This class tests the replication of a DFS file.
@@ -71,27 +69,11 @@ public class TestReplication extends TestCase {
     Configuration conf = fileSys.getConf();
     Configuration conf = fileSys.getConf();
     ClientProtocol namenode = DFSClient.createNamenode(conf);
     ClientProtocol namenode = DFSClient.createNamenode(conf);
       
       
-    LocatedBlocks locations;
-    boolean isReplicationDone;
-    do {
-      locations = namenode.getBlockLocations(name.toString(),0,Long.MAX_VALUE);
-      isReplicationDone = true;
-      for (LocatedBlock blk : locations.getLocatedBlocks()) {
-        DatanodeInfo[] datanodes = blk.getLocations();
-        if (Math.min(numDatanodes, repl) != datanodes.length) {
-          isReplicationDone=false;
-          LOG.warn("File has "+datanodes.length+" replicas, expecting "
-                   +Math.min(numDatanodes, repl));
-          try {
-            Thread.sleep(15000L);
-          } catch (InterruptedException e) {
-            // nothing
-          }
-          break;
-        }
-      }
-    } while(!isReplicationDone);
-      
+    waitForBlockReplication(name.toString(), namenode, 
+                            Math.min(numDatanodes, repl), -1);
+    
+    LocatedBlocks locations = namenode.getBlockLocations(name.toString(),0,
+                                                         Long.MAX_VALUE);
     boolean isOnSameRack = true, isNotOnSameRack = true;
     boolean isOnSameRack = true, isNotOnSameRack = true;
     for (LocatedBlock blk : locations.getLocatedBlocks()) {
     for (LocatedBlock blk : locations.getLocatedBlocks()) {
       DatanodeInfo[] datanodes = blk.getLocations();
       DatanodeInfo[] datanodes = blk.getLocations();

+ 9 - 3
src/test/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java

@@ -20,6 +20,7 @@ package org.apache.hadoop.hdfs.server.datanode;
 import java.io.IOException;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.InputStream;
 import java.io.OutputStream;
 import java.io.OutputStream;
+import java.util.Arrays;
 import java.util.HashMap;
 import java.util.HashMap;
 import java.util.Random;
 import java.util.Random;
 
 
@@ -279,9 +280,14 @@ public class SimulatedFSDataset  implements FSConstants, FSDatasetInterface, Con
 
 
   public synchronized Block[] getBlockReport() {
   public synchronized Block[] getBlockReport() {
     Block[] blockTable = new Block[blockMap.size()];
     Block[] blockTable = new Block[blockMap.size()];
-    int i = 0;
-    for (Block b: blockMap.keySet()) {  
-      blockTable[i++] = blockMap.get(b).theBlock;
+    int count = 0;
+    for (BInfo b : blockMap.values()) {
+      if (b.isFinalized()) {
+        blockTable[count++] = b.theBlock;
+      }
+    }
+    if (count != blockTable.length) {
+      blockTable = Arrays.copyOf(blockTable, count);
     }
     }
     return blockTable;
     return blockTable;
   }
   }