浏览代码

Merging changes r1068968:r1076024 from trunk into federation

git-svn-id: https://svn.apache.org/repos/asf/hadoop/hdfs/branches/HDFS-1052@1079610 13f79535-47bb-0310-9956-ffa450edef68
Suresh Srinivas 14 年之前
父节点
当前提交
435a5c7e4d

+ 14 - 2
CHANGES.txt

@@ -262,6 +262,9 @@ Trunk (unreleased changes)
     
     
     HDFS-560 Enhancements/tuning to hadoop-hdfs/build.xml
     HDFS-560 Enhancements/tuning to hadoop-hdfs/build.xml
 
 
+    HDFS-1629. Add a method to BlockPlacementPolicy for keeping the chosen
+    nodes in the output array.  (szetszwo)
+
   OPTIMIZATIONS
   OPTIMIZATIONS
 
 
     HDFS-1458. Improve checkpoint performance by avoiding unnecessary image
     HDFS-1458. Improve checkpoint performance by avoiding unnecessary image
@@ -310,8 +313,6 @@ Trunk (unreleased changes)
     HDFS-1610. Fix TestClientProtocolWithDelegationToken and TestBlockToken
     HDFS-1610. Fix TestClientProtocolWithDelegationToken and TestBlockToken
     on trunk after HADOOP-6904 (todd)
     on trunk after HADOOP-6904 (todd)
 
 
-    HDFS-1602. Fix HADOOP-4885 for it is doesn't work as expected. (boryas)
-
     HDFS-1600. Fix release audit warnings on trunk. (todd)
     HDFS-1600. Fix release audit warnings on trunk. (todd)
 
 
 Release 0.22.0 - Unreleased
 Release 0.22.0 - Unreleased
@@ -757,6 +758,13 @@ Release 0.22.0 - Unreleased
     HDFS-1597. Batched edit log syncs can reset synctxid and throw assertions
     HDFS-1597. Batched edit log syncs can reset synctxid and throw assertions
     (todd)
     (todd)
 
 
+    HDFS-1602. Fix HADOOP-4885 for it is doesn't work as expected. (boryas)
+
+    HDFS-1618. configure files that are generated as part of the released
+    tarball need to have executable bit set (Roman Shaposhnik via cos)
+
+    HDFS-981. test-contrib fails due to test-cactus failure (cos)
+
 Release 0.21.1 - Unreleased
 Release 0.21.1 - Unreleased
 
 
     HDFS-1411. Correct backup node startup command in hdfs user guide.
     HDFS-1411. Correct backup node startup command in hdfs user guide.
@@ -802,6 +810,10 @@ Release 0.21.1 - Unreleased
 
 
     HDFS-996. JUnit tests should never depend on anything in conf (cos)
     HDFS-996. JUnit tests should never depend on anything in conf (cos)
 
 
+    HDFS-1612. Update HDFS design documentation for append, quota, symlink,
+    block placement and checkpoint/backup node features.  (Joe Crobak
+    via szetszwo)
+
   INCOMPATIBLE CHANGES
   INCOMPATIBLE CHANGES
 
 
     HDFS-538. Per the contract elucidated in HADOOP-6201, throw
     HDFS-538. Per the contract elucidated in HADOOP-6201, throw

+ 3 - 0
build.xml

@@ -1045,6 +1045,7 @@
 
 
     <copy todir="${dist.dir}/" file="build.xml"/>
     <copy todir="${dist.dir}/" file="build.xml"/>
 
 
+    <chmod perm="ugo+x" file="${dist.dir}/src/c++/libhdfs/configure" />
     <chmod perm="ugo+x" type="file" parallel="false">
     <chmod perm="ugo+x" type="file" parallel="false">
         <fileset dir="${dist.dir}/bin"/>
         <fileset dir="${dist.dir}/bin"/>
         <fileset dir="${dist.dir}/src/contrib/">
         <fileset dir="${dist.dir}/src/contrib/">
@@ -1063,11 +1064,13 @@
         <tarfileset dir="${build.dir}" mode="664">
         <tarfileset dir="${build.dir}" mode="664">
           <exclude name="${final.name}/bin/*" />
           <exclude name="${final.name}/bin/*" />
           <exclude name="${final.name}/contrib/*/bin/*" />
           <exclude name="${final.name}/contrib/*/bin/*" />
+          <exclude name="${final.name}/src/c++/libhdfs/configure" />
           <include name="${final.name}/**" />
           <include name="${final.name}/**" />
         </tarfileset>
         </tarfileset>
         <tarfileset dir="${build.dir}" mode="755">
         <tarfileset dir="${build.dir}" mode="755">
           <include name="${final.name}/bin/*" />
           <include name="${final.name}/bin/*" />
           <include name="${final.name}/contrib/*/bin/*" />
           <include name="${final.name}/contrib/*/bin/*" />
+          <include name="${final.name}/src/c++/libhdfs/configure" />
         </tarfileset>
         </tarfileset>
       </param.listofitems>
       </param.listofitems>
     </macro_tar>
     </macro_tar>

+ 1 - 1
src/contrib/hdfsproxy/build.xml

@@ -301,7 +301,7 @@
 			<containerset>
 			<containerset>
 				<cargo containerId="${tomcat.container.id}" timeout="30000" output="${logs.dir}/output.log" log="${logs.dir}/cargo.log">
 				<cargo containerId="${tomcat.container.id}" timeout="30000" output="${logs.dir}/output.log" log="${logs.dir}/cargo.log">
 				 <zipUrlInstaller
 				 <zipUrlInstaller
-            installUrl="http://apache.osuosl.org/tomcat/tomcat-6/v6.0.24/bin/apache-tomcat-6.0.24.zip"
+            installUrl="http://archive.apache.org/dist/tomcat/tomcat-6/v6.0.24/bin/apache-tomcat-6.0.24.zip"
             installDir="${target.dir}/${tomcat.container.id}"/>
             installDir="${target.dir}/${tomcat.container.id}"/>
 				  <configuration type="existing" home="${tomcatconfig.dir}">
 				  <configuration type="existing" home="${tomcatconfig.dir}">
 						<property name="cargo.servlet.port" value="${cargo.servlet.http.port}"/>
 						<property name="cargo.servlet.port" value="${cargo.servlet.http.port}"/>

+ 24 - 14
src/docs/src/documentation/content/xdocs/hdfs_design.xml

@@ -73,18 +73,22 @@
         <title> Large Data Sets </title>
         <title> Large Data Sets </title>
         <p>
         <p>
         Applications that run on HDFS have large data sets. A typical file in HDFS is gigabytes to terabytes in size. Thus, HDFS is tuned to 
         Applications that run on HDFS have large data sets. A typical file in HDFS is gigabytes to terabytes in size. Thus, HDFS is tuned to 
-        support large files. It should provide high aggregate data bandwidth and scale to hundreds of nodes in a single cluster. It should support 
+        support large files. It should provide high aggregate data bandwidth and scale to thousands of nodes in a single cluster. It should support 
         tens of millions of files in a single instance.
         tens of millions of files in a single instance.
         </p>
         </p>
       </section>
       </section>
 
 
  
  
       <section> 
       <section> 
-        <title> Simple Coherency Model </title>
+        <title> Appending-Writes and File Syncs </title>
         <p>
         <p>
-        HDFS applications need a write-once-read-many access model for files. A file once created, written, and closed need not be changed. 
-        This assumption simplifies data coherency issues and enables high throughput data access. A MapReduce application or a web crawler 
-        application fits perfectly with this model. There is a plan to support appending-writes to files in the future. 
+        Most HDFS applications need a write-once-read-many access model for files. HDFS provides two additional advanced features: hflush and
+        append.  Hflush makes the last block of an unclosed file visible to readers while providing read consistency and data durability.  Append
+        provides a mechanism for opening a closed file to add additional data.
+        </p>
+        <p>
+        For complete details of the hflush and append design, see the 
+        <a href="https://issues.apache.org/jira/secure/attachment/12445209/appendDesign3.pdf">Append/Hflush/Read Design document</a> (PDF).
         </p>
         </p>
       </section>
       </section>
 
 
@@ -145,8 +149,10 @@
       <p>
       <p>
       HDFS supports a traditional hierarchical file organization. A user or an application can create directories and store files inside 
       HDFS supports a traditional hierarchical file organization. A user or an application can create directories and store files inside 
       these directories. The file system namespace hierarchy is similar to most other existing file systems; one can create and 
       these directories. The file system namespace hierarchy is similar to most other existing file systems; one can create and 
-      remove files, move a file from one directory to another, or rename a file. HDFS does not yet implement user quotas. HDFS 
-      does not support hard links or soft links. However, the HDFS architecture does not preclude implementing these features.
+      remove files, move a file from one directory to another, or rename a file. HDFS implements user quotas for number of names and 
+      amount of data stored in a particular directory (See 
+      <a href="http://hadoop.apache.org/hdfs/docs/current/hdfs_quota_admin_guide.html">HDFS Quota Admin Guide</a>). In addition, HDFS
+      supports <a href="http://hadoop.apache.org/common/docs/current/api/org/apache/hadoop/fs/FileContext.html#createSymlink(org.apache.hadoop.fs.Path, org.apache.hadoop.fs.Path, boolean)">symbolic links</a>.
       </p>
       </p>
       <p>
       <p>
       The NameNode maintains the file system namespace. Any change to the file system namespace or its properties is 
       The NameNode maintains the file system namespace. Any change to the file system namespace or its properties is 
@@ -163,8 +169,8 @@
       HDFS is designed to reliably store very large files across machines in a large cluster. It stores each file as a sequence 
       HDFS is designed to reliably store very large files across machines in a large cluster. It stores each file as a sequence 
       of blocks; all blocks in a file except the last block are the same size. The blocks of a file are replicated for fault tolerance. 
       of blocks; all blocks in a file except the last block are the same size. The blocks of a file are replicated for fault tolerance. 
       The block size and replication factor are configurable per file. An application can specify the number of replicas of a file. 
       The block size and replication factor are configurable per file. An application can specify the number of replicas of a file. 
-      The replication factor can be specified at file creation time and can be changed later. Files in HDFS are write-once and 
-      have strictly one writer at any time. 
+      The replication factor can be specified at file creation time and can be changed later. Files in HDFS are strictly one writer at any 
+      time. 
       </p>
       </p>
       <p>
       <p>
       The NameNode makes all decisions regarding replication of blocks. It periodically receives a Heartbeat and a Blockreport 
       The NameNode makes all decisions regarding replication of blocks. It periodically receives a Heartbeat and a Blockreport 
@@ -208,7 +214,8 @@
         data reliability or read performance.
         data reliability or read performance.
         </p>
         </p>
         <p>
         <p>
-        The current, default replica placement policy described here is a work in progress.
+        In addition to the default placement policy described above, HDFS also provides a pluggable interface for block placement. See
+        <a href="http://hadoop.apache.org/hdfs/docs/current/api/org/apache/hadoop/hdfs/server/namenode/BlockPlacementPolicy.html">BlockPlacementPolicy</a>.
         </p>
         </p>
       </section>
       </section>
 
 
@@ -217,7 +224,7 @@
         <p>
         <p>
         To minimize global bandwidth consumption and read latency, HDFS tries to satisfy a read request from a replica 
         To minimize global bandwidth consumption and read latency, HDFS tries to satisfy a read request from a replica 
         that is closest to the reader. If there exists a replica on the same rack as the reader node, then that replica is 
         that is closest to the reader. If there exists a replica on the same rack as the reader node, then that replica is 
-        preferred to satisfy the read request. If angg/ HDFS cluster spans multiple data centers, then a replica that is 
+        preferred to satisfy the read request. If an HDFS cluster spans multiple data centers, then a replica that is 
         resident in the local data center is preferred over any remote replica.
         resident in the local data center is preferred over any remote replica.
         </p>
         </p>
       </section>
       </section>
@@ -255,9 +262,12 @@
         huge number of files and directories. When the NameNode starts up, it reads the FsImage and EditLog from 
         huge number of files and directories. When the NameNode starts up, it reads the FsImage and EditLog from 
         disk, applies all the transactions from the EditLog to the in-memory representation of the FsImage, and flushes 
         disk, applies all the transactions from the EditLog to the in-memory representation of the FsImage, and flushes 
         out this new version into a new FsImage on disk. It can then truncate the old EditLog because its transactions 
         out this new version into a new FsImage on disk. It can then truncate the old EditLog because its transactions 
-        have been applied to the persistent FsImage. This process is called a checkpoint. In the current implementation, 
-        a checkpoint only occurs when the NameNode starts up. Work is in progress to support periodic checkpointing 
-        in the near future.
+        have been applied to the persistent FsImage. This process is called a checkpoint. The 
+        <a href="http://hadoop.apache.org/hdfs/docs/current/hdfs_user_guide.html#Checkpoint+Node">Checkpoint Node</a> is a 
+        separate daemon that can be configured to periodically build checkpoints from the FsImage and EditLog which are 
+        uploaded to the NameNode.  The 
+        <a href="http://hadoop.apache.org/hdfs/docs/current/hdfs_user_guide.html#Backup+Node">Backup Node</a> builds 
+        checkpoints like the Checkpoint Node and also maintains an up-to-date copy of the FsImage in memory.
         </p>
         </p>
         <p>
         <p>
         The DataNode stores HDFS data in files in its local file system. The DataNode has no knowledge about HDFS files. 
         The DataNode stores HDFS data in files in its local file system. The DataNode has no knowledge about HDFS files. 

+ 23 - 2
src/java/org/apache/hadoop/hdfs/server/namenode/BlockPlacementPolicy.java

@@ -17,6 +17,11 @@
  */
  */
 package org.apache.hadoop.hdfs.server.namenode;
 package org.apache.hadoop.hdfs.server.namenode;
 
 
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.Block;
@@ -24,7 +29,6 @@ import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.net.NetworkTopology;
 import org.apache.hadoop.net.NetworkTopology;
 import org.apache.hadoop.net.Node;
 import org.apache.hadoop.net.Node;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.ReflectionUtils;
-import java.util.*;
 
 
 /** 
 /** 
  * This interface is used for choosing the desired number of targets
  * This interface is used for choosing the desired number of targets
@@ -60,6 +64,21 @@ public abstract class BlockPlacementPolicy {
                                              List<DatanodeDescriptor> chosenNodes,
                                              List<DatanodeDescriptor> chosenNodes,
                                              long blocksize);
                                              long blocksize);
 
 
+  /**
+   * Same as
+   * {{@link #chooseTarget(String, int, DatanodeDescriptor, List, boolean, HashMap, long)}
+   * with returnChosenNodes equal to false.
+   */
+  final DatanodeDescriptor[] chooseTarget(String srcPath,
+                                          int numOfReplicas,
+                                          DatanodeDescriptor writer,
+                                          List<DatanodeDescriptor> chosenNodes,
+                                          HashMap<Node, Node> excludedNodes,
+                                          long blocksize) {
+    return chooseTarget(srcPath, numOfReplicas, writer, chosenNodes, false,
+        excludedNodes, blocksize);
+  }
+
   /**
   /**
    * choose <i>numOfReplicas</i> data nodes for <i>writer</i> 
    * choose <i>numOfReplicas</i> data nodes for <i>writer</i> 
    * to re-replicate a block with size <i>blocksize</i> 
    * to re-replicate a block with size <i>blocksize</i> 
@@ -69,7 +88,8 @@ public abstract class BlockPlacementPolicy {
    * @param numOfReplicas additional number of replicas wanted.
    * @param numOfReplicas additional number of replicas wanted.
    * @param writer the writer's machine, null if not in the cluster.
    * @param writer the writer's machine, null if not in the cluster.
    * @param chosenNodes datanodes that have been chosen as targets.
    * @param chosenNodes datanodes that have been chosen as targets.
-   * @param excludedNodes: datanodes that should not be considered as targets.
+   * @param returnChosenNodes decide if the chosenNodes are returned.
+   * @param excludedNodes datanodes that should not be considered as targets.
    * @param blocksize size of the data to be written.
    * @param blocksize size of the data to be written.
    * @return array of DatanodeDescriptor instances chosen as target
    * @return array of DatanodeDescriptor instances chosen as target
    * and sorted as a pipeline.
    * and sorted as a pipeline.
@@ -78,6 +98,7 @@ public abstract class BlockPlacementPolicy {
                                              int numOfReplicas,
                                              int numOfReplicas,
                                              DatanodeDescriptor writer,
                                              DatanodeDescriptor writer,
                                              List<DatanodeDescriptor> chosenNodes,
                                              List<DatanodeDescriptor> chosenNodes,
+                                             boolean returnChosenNodes,
                                              HashMap<Node, Node> excludedNodes,
                                              HashMap<Node, Node> excludedNodes,
                                              long blocksize);
                                              long blocksize);
 
 

+ 24 - 14
src/java/org/apache/hadoop/hdfs/server/namenode/BlockPlacementPolicyDefault.java

@@ -17,9 +17,17 @@
  */
  */
 package org.apache.hadoop.hdfs.server.namenode;
 package org.apache.hadoop.hdfs.server.namenode;
 
 
-import org.apache.commons.logging.*;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import java.util.TreeSet;
+
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.FSConstants;
 import org.apache.hadoop.hdfs.protocol.FSConstants;
@@ -27,8 +35,6 @@ import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.net.NetworkTopology;
 import org.apache.hadoop.net.NetworkTopology;
 import org.apache.hadoop.net.Node;
 import org.apache.hadoop.net.Node;
 import org.apache.hadoop.net.NodeBase;
 import org.apache.hadoop.net.NodeBase;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
-import java.util.*;
 
 
 /** The class is responsible for choosing the desired number of targets
 /** The class is responsible for choosing the desired number of targets
  * for placing block replicas.
  * for placing block replicas.
@@ -66,20 +72,23 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
                                     DatanodeDescriptor writer,
                                     DatanodeDescriptor writer,
                                     List<DatanodeDescriptor> chosenNodes,
                                     List<DatanodeDescriptor> chosenNodes,
                                     long blocksize) {
                                     long blocksize) {
-    return chooseTarget(numOfReplicas, writer, chosenNodes, null, blocksize);
+    return chooseTarget(numOfReplicas, writer, chosenNodes, false,
+        null, blocksize);
   }
   }
 
 
   /** {@inheritDoc} */
   /** {@inheritDoc} */
-  public DatanodeDescriptor[] chooseTarget(String srcPath,
+  @Override
+  DatanodeDescriptor[] chooseTarget(String srcPath,
                                     int numOfReplicas,
                                     int numOfReplicas,
                                     DatanodeDescriptor writer,
                                     DatanodeDescriptor writer,
                                     List<DatanodeDescriptor> chosenNodes,
                                     List<DatanodeDescriptor> chosenNodes,
+                                    boolean returnChosenNodes,
                                     HashMap<Node, Node> excludedNodes,
                                     HashMap<Node, Node> excludedNodes,
                                     long blocksize) {
                                     long blocksize) {
-    return chooseTarget(numOfReplicas, writer, chosenNodes, excludedNodes, blocksize);
+    return chooseTarget(numOfReplicas, writer, chosenNodes, returnChosenNodes,
+        excludedNodes, blocksize);
   }
   }
 
 
-
   /** {@inheritDoc} */
   /** {@inheritDoc} */
   @Override
   @Override
   public DatanodeDescriptor[] chooseTarget(FSInodeInfo srcInode,
   public DatanodeDescriptor[] chooseTarget(FSInodeInfo srcInode,
@@ -87,15 +96,15 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
                                     DatanodeDescriptor writer,
                                     DatanodeDescriptor writer,
                                     List<DatanodeDescriptor> chosenNodes,
                                     List<DatanodeDescriptor> chosenNodes,
                                     long blocksize) {
                                     long blocksize) {
-    return chooseTarget(numOfReplicas, writer, chosenNodes, null, blocksize);
+    return chooseTarget(numOfReplicas, writer, chosenNodes, false,
+        null, blocksize);
   }
   }
-    
-  /**
-   * This is not part of the public API but is used by the unit tests.
-   */
+
+  /** This is the implementation. */
   DatanodeDescriptor[] chooseTarget(int numOfReplicas,
   DatanodeDescriptor[] chooseTarget(int numOfReplicas,
                                     DatanodeDescriptor writer,
                                     DatanodeDescriptor writer,
                                     List<DatanodeDescriptor> chosenNodes,
                                     List<DatanodeDescriptor> chosenNodes,
+                                    boolean returnChosenNodes,
                                     HashMap<Node, Node> excludedNodes,
                                     HashMap<Node, Node> excludedNodes,
                                     long blocksize) {
                                     long blocksize) {
     if (numOfReplicas == 0 || clusterMap.getNumOfLeaves()==0) {
     if (numOfReplicas == 0 || clusterMap.getNumOfLeaves()==0) {
@@ -128,8 +137,9 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
       
       
     DatanodeDescriptor localNode = chooseTarget(numOfReplicas, writer, 
     DatanodeDescriptor localNode = chooseTarget(numOfReplicas, writer, 
                                                 excludedNodes, blocksize, maxNodesPerRack, results);
                                                 excludedNodes, blocksize, maxNodesPerRack, results);
-      
-    results.removeAll(chosenNodes);
+    if (!returnChosenNodes) {  
+      results.removeAll(chosenNodes);
+    }
       
       
     // sorting nodes to form a pipeline
     // sorting nodes to form a pipeline
     return getPipeline((writer==null)?localNode:writer,
     return getPipeline((writer==null)?localNode:writer,

+ 35 - 10
src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestReplicationPolicy.java

@@ -20,6 +20,7 @@ package org.apache.hadoop.hdfs.server.namenode;
 
 
 import java.io.IOException;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.HashMap;
 import java.util.HashMap;
 import java.util.List;
 import java.util.List;
 
 
@@ -130,6 +131,17 @@ public class TestReplicationPolicy extends TestCase {
         FSConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, 0); 
         FSConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, 0); 
   }
   }
 
 
+  private static DatanodeDescriptor[] chooseTarget(
+      BlockPlacementPolicyDefault policy,
+      int numOfReplicas,
+      DatanodeDescriptor writer,
+      List<DatanodeDescriptor> chosenNodes,
+      HashMap<Node, Node> excludedNodes,
+      long blocksize) {
+    return policy.chooseTarget(numOfReplicas, writer, chosenNodes, false,
+        excludedNodes, blocksize);
+  }
+
   /**
   /**
    * In this testcase, client is dataNodes[0], but the dataNodes[1] is
    * In this testcase, client is dataNodes[0], but the dataNodes[1] is
    * not allowed to be chosen. So the 1st replica should be
    * not allowed to be chosen. So the 1st replica should be
@@ -146,23 +158,23 @@ public class TestReplicationPolicy extends TestCase {
     
     
     excludedNodes = new HashMap<Node, Node>();
     excludedNodes = new HashMap<Node, Node>();
     excludedNodes.put(dataNodes[1], dataNodes[1]); 
     excludedNodes.put(dataNodes[1], dataNodes[1]); 
-    targets = repl.chooseTarget(
-                                0, dataNodes[0], chosenNodes, excludedNodes, BLOCK_SIZE);
+    targets = chooseTarget(repl, 0, dataNodes[0], chosenNodes, excludedNodes,
+        BLOCK_SIZE);
     assertEquals(targets.length, 0);
     assertEquals(targets.length, 0);
     
     
     excludedNodes.clear();
     excludedNodes.clear();
     chosenNodes.clear();
     chosenNodes.clear();
     excludedNodes.put(dataNodes[1], dataNodes[1]); 
     excludedNodes.put(dataNodes[1], dataNodes[1]); 
-    targets = repl.chooseTarget(
-                                1, dataNodes[0], chosenNodes, excludedNodes, BLOCK_SIZE);
+    targets = chooseTarget(repl, 1, dataNodes[0], chosenNodes, excludedNodes,
+        BLOCK_SIZE);
     assertEquals(targets.length, 1);
     assertEquals(targets.length, 1);
     assertEquals(targets[0], dataNodes[0]);
     assertEquals(targets[0], dataNodes[0]);
     
     
     excludedNodes.clear();
     excludedNodes.clear();
     chosenNodes.clear();
     chosenNodes.clear();
     excludedNodes.put(dataNodes[1], dataNodes[1]); 
     excludedNodes.put(dataNodes[1], dataNodes[1]); 
-    targets = repl.chooseTarget(
-                                2, dataNodes[0], chosenNodes, excludedNodes, BLOCK_SIZE);
+    targets = chooseTarget(repl, 2, dataNodes[0], chosenNodes, excludedNodes,
+        BLOCK_SIZE);
     assertEquals(targets.length, 2);
     assertEquals(targets.length, 2);
     assertEquals(targets[0], dataNodes[0]);
     assertEquals(targets[0], dataNodes[0]);
     assertFalse(cluster.isOnSameRack(targets[0], targets[1]));
     assertFalse(cluster.isOnSameRack(targets[0], targets[1]));
@@ -170,8 +182,8 @@ public class TestReplicationPolicy extends TestCase {
     excludedNodes.clear();
     excludedNodes.clear();
     chosenNodes.clear();
     chosenNodes.clear();
     excludedNodes.put(dataNodes[1], dataNodes[1]); 
     excludedNodes.put(dataNodes[1], dataNodes[1]); 
-    targets = repl.chooseTarget(
-                                3, dataNodes[0], chosenNodes, excludedNodes, BLOCK_SIZE);
+    targets = chooseTarget(repl, 3, dataNodes[0], chosenNodes, excludedNodes,
+        BLOCK_SIZE);
     assertEquals(targets.length, 3);
     assertEquals(targets.length, 3);
     assertEquals(targets[0], dataNodes[0]);
     assertEquals(targets[0], dataNodes[0]);
     assertFalse(cluster.isOnSameRack(targets[0], targets[1]));
     assertFalse(cluster.isOnSameRack(targets[0], targets[1]));
@@ -180,8 +192,8 @@ public class TestReplicationPolicy extends TestCase {
     excludedNodes.clear();
     excludedNodes.clear();
     chosenNodes.clear();
     chosenNodes.clear();
     excludedNodes.put(dataNodes[1], dataNodes[1]); 
     excludedNodes.put(dataNodes[1], dataNodes[1]); 
-    targets = repl.chooseTarget(
-                                4, dataNodes[0], chosenNodes, excludedNodes, BLOCK_SIZE);
+    targets = chooseTarget(repl, 4, dataNodes[0], chosenNodes, excludedNodes,
+        BLOCK_SIZE);
     assertEquals(targets.length, 4);
     assertEquals(targets.length, 4);
     assertEquals(targets[0], dataNodes[0]);
     assertEquals(targets[0], dataNodes[0]);
     for(int i=1; i<4; i++) {
     for(int i=1; i<4; i++) {
@@ -190,6 +202,19 @@ public class TestReplicationPolicy extends TestCase {
     assertTrue(cluster.isOnSameRack(targets[1], targets[2]) ||
     assertTrue(cluster.isOnSameRack(targets[1], targets[2]) ||
                cluster.isOnSameRack(targets[2], targets[3]));
                cluster.isOnSameRack(targets[2], targets[3]));
     assertFalse(cluster.isOnSameRack(targets[1], targets[3]));
     assertFalse(cluster.isOnSameRack(targets[1], targets[3]));
+
+    excludedNodes.clear();
+    chosenNodes.clear();
+    excludedNodes.put(dataNodes[1], dataNodes[1]); 
+    chosenNodes.add(dataNodes[2]);
+    targets = repl.chooseTarget(1, dataNodes[0], chosenNodes, true,
+        excludedNodes, BLOCK_SIZE);
+    System.out.println("targets=" + Arrays.asList(targets));
+    assertEquals(2, targets.length);
+    //make sure that the chosen node is in the target.
+    int i = 0;
+    for(; i < targets.length && !dataNodes[2].equals(targets[i]); i++);
+    assertTrue(i < targets.length);
   }
   }
 
 
   /**
   /**