Преглед на файлове

Merge r1335791 through r1337002 from trunk.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-3092@1337003 13f79535-47bb-0310-9956-ffa450edef68
Tsz-wo Sze преди 13 години
родител
ревизия
ac6b105e60
променени са 59 файла, в които са добавени 826 реда и са изтрити 633 реда
  1. 29 4
      dev-support/test-patch.sh
  2. 23 0
      hadoop-common-project/hadoop-common/CHANGES.txt
  3. 2 67
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/BlockLocation.java
  4. 8 8
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileStatus.java
  5. 4 4
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/permission/PermissionStatus.java
  6. 29 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/Text.java
  7. 7 12
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/NetUtils.java
  8. 32 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/VersionUtil.java
  9. 3 3
      hadoop-common-project/hadoop-common/src/main/native/acinclude.m4
  10. 12 2
      hadoop-common-project/hadoop-common/src/main/native/configure.ac
  11. 0 78
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestBlockLocation.java
  12. 35 1
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/TestText.java
  13. 23 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPC.java
  14. 22 2
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestNetUtils.java
  15. 6 6
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestTableMapping.java
  16. 9 1
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestVersionUtil.java
  17. 27 0
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
  18. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/fuse-dfs/src/Makefile.am
  19. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
  20. 4 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
  21. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/block/BlockTokenIdentifier.java
  22. 10 10
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java
  23. 68 70
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
  24. 6 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicy.java
  25. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyDefault.java
  26. 8 8
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlocksMap.java
  27. 1 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
  28. 3 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
  29. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
  30. 29 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java
  31. 17 20
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
  32. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
  33. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFileUnderConstruction.java
  34. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeJspHelper.java
  35. 14 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java
  36. 0 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java
  37. 51 4
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
  38. 2 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientRetries.java
  39. 42 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend.java
  40. 1 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestGetBlocks.java
  41. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReplaceDatanodeOnFailure.java
  42. 13 15
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocolPB/TestPBHelper.java
  43. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/security/token/block/TestBlockToken.java
  44. 36 30
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java
  45. 37 35
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestHost2NodesMap.java
  46. 5 6
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestPendingDataNodeMessages.java
  47. 0 104
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestRBWBlockInvalidation.java
  48. 42 37
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java
  49. 0 5
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/DataNodeTestUtils.java
  50. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBPOfferService.java
  51. 4 4
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockRecovery.java
  52. 0 6
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetTestUtil.java
  53. 1 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestInterDatanodeProtocol.java
  54. 72 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java
  55. 36 22
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/net/TestNetworkTopology.java
  56. 3 0
      hadoop-mapreduce-project/CHANGES.txt
  57. 7 2
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeStatusUpdater.java
  58. 14 14
      hadoop-mapreduce-project/src/contrib/raid/src/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyRaid.java
  59. 13 13
      hadoop-mapreduce-project/src/contrib/raid/src/test/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockPlacementPolicyRaid.java

+ 29 - 4
dev-support/test-patch.sh

@@ -423,8 +423,8 @@ checkJavacWarnings () {
   if [[ $? != 0 ]] ; then
   if [[ $? != 0 ]] ; then
     JIRA_COMMENT="$JIRA_COMMENT
     JIRA_COMMENT="$JIRA_COMMENT
 
 
-    -1 javac.  The patch appears to cause tar ant target to fail."
-    return 1
+    -1 javac.  The patch appears to cause the build to fail."
+    return 2
   fi
   fi
   ### Compare trunk and patch javac warning numbers
   ### Compare trunk and patch javac warning numbers
   if [[ -f $PATCH_DIR/patchJavacWarnings.txt ]] ; then
   if [[ -f $PATCH_DIR/patchJavacWarnings.txt ]] ; then
@@ -528,6 +528,24 @@ $JIRA_COMMENT_FOOTER"
   return 0
   return 0
 }
 }
 
 
+###############################################################################
+### Install the new jars so tests and findbugs can find all of the updated jars 
+buildAndInstall () {
+  echo ""
+  echo ""
+  echo "======================================================================"
+  echo "======================================================================"
+  echo "    Installing all of the jars"
+  echo "======================================================================"
+  echo "======================================================================"
+  echo ""
+  echo ""
+  echo "$MVN install -Dmaven.javadoc.skip=true -DskipTests -D${PROJECT_NAME}PatchProcess"
+  $MVN install -Dmaven.javadoc.skip=true -DskipTests -D${PROJECT_NAME}PatchProcess
+  return $?
+}
+
+
 ###############################################################################
 ###############################################################################
 ### Check there are no changes in the number of Findbugs warnings
 ### Check there are no changes in the number of Findbugs warnings
 checkFindbugsWarnings () {
 checkFindbugsWarnings () {
@@ -882,15 +900,22 @@ if [[ $? != 0 ]] ; then
   submitJiraComment 1
   submitJiraComment 1
   cleanupAndExit 1
   cleanupAndExit 1
 fi
 fi
-checkJavadocWarnings
-(( RESULT = RESULT + $? ))
 checkJavacWarnings
 checkJavacWarnings
+JAVAC_RET=$?
+#2 is returned if the code could not compile
+if [[ $JAVAC_RET == 2 ]] ; then
+  submitJiraComment 1
+  cleanupAndExit 1
+fi
+(( RESULT = RESULT + $JAVAC_RET ))
+checkJavadocWarnings
 (( RESULT = RESULT + $? ))
 (( RESULT = RESULT + $? ))
 checkEclipseGeneration
 checkEclipseGeneration
 (( RESULT = RESULT + $? ))
 (( RESULT = RESULT + $? ))
 ### Checkstyle not implemented yet
 ### Checkstyle not implemented yet
 #checkStyle
 #checkStyle
 #(( RESULT = RESULT + $? ))
 #(( RESULT = RESULT + $? ))
+buildAndInstall
 checkFindbugsWarnings
 checkFindbugsWarnings
 (( RESULT = RESULT + $? ))
 (( RESULT = RESULT + $? ))
 checkReleaseAuditWarnings
 checkReleaseAuditWarnings

+ 23 - 0
hadoop-common-project/hadoop-common/CHANGES.txt

@@ -129,6 +129,12 @@ Trunk (unreleased changes)
     HADOOP-8339. jenkins complaining about 16 javadoc warnings 
     HADOOP-8339. jenkins complaining about 16 javadoc warnings 
     (Tom White and Robert Evans via tgraves)
     (Tom White and Robert Evans via tgraves)
 
 
+    HADOOP-8354. test-patch findbugs may fail if a dependent module is changed
+    (Tom White and Robert Evans)
+
+    HADOOP-8375. test-patch should stop immediately once it has found
+    compilation errors (bobby)
+
   OPTIMIZATIONS
   OPTIMIZATIONS
 
 
     HADOOP-7761. Improve the performance of raw comparisons. (todd)
     HADOOP-7761. Improve the performance of raw comparisons. (todd)
@@ -139,6 +145,9 @@ Release 2.0.0 - UNRELEASED
 
 
     HADOOP-7920. Remove Avro Rpc. (suresh)
     HADOOP-7920. Remove Avro Rpc. (suresh)
 
 
+    HADOOP-8388. Remove unused BlockLocation serialization.
+    (Colin Patrick McCabe via eli)
+
   NEW FEATURES
   NEW FEATURES
 
 
     HADOOP-7773. Add support for protocol buffer based RPC engine.
     HADOOP-7773. Add support for protocol buffer based RPC engine.
@@ -296,6 +305,12 @@ Release 2.0.0 - UNRELEASED
     HADOOP-8356. FileSystem service loading mechanism should print the FileSystem 
     HADOOP-8356. FileSystem service loading mechanism should print the FileSystem 
     impl it is failing to load (tucu)
     impl it is failing to load (tucu)
 
 
+    HADOOP-8340. SNAPSHOT build versions should compare as less than their eventual
+    final release. (todd)
+
+    HADOOP-8361. Avoid out-of-memory problems when deserializing strings.
+    (Colin Patrick McCabe via eli)
+
   OPTIMIZATIONS
   OPTIMIZATIONS
 
 
   BUG FIXES
   BUG FIXES
@@ -429,6 +444,9 @@ Release 2.0.0 - UNRELEASED
     HADOOP-8359. Fix javadoc warnings in Configuration.  (Anupam Seth via
     HADOOP-8359. Fix javadoc warnings in Configuration.  (Anupam Seth via
     szetszwo)
     szetszwo)
 
 
+    HADOOP-8372. NetUtils.normalizeHostName() incorrectly handles hostname
+    starting with a numeric character. (Junping Du via suresh)
+
   BREAKDOWN OF HADOOP-7454 SUBTASKS
   BREAKDOWN OF HADOOP-7454 SUBTASKS
 
 
     HADOOP-7455. HA: Introduce HA Service Protocol Interface. (suresh)
     HADOOP-7455. HA: Introduce HA Service Protocol Interface. (suresh)
@@ -487,6 +505,9 @@ Release 2.0.0 - UNRELEASED
     HADOOP-8172. Configuration no longer sets all keys in a deprecated key 
     HADOOP-8172. Configuration no longer sets all keys in a deprecated key 
     list. (Anupam Seth via bobby)
     list. (Anupam Seth via bobby)
 
 
+    HADOOP-7868. Hadoop native fails to compile when default linker
+    option is -Wl,--as-needed. (Trevor Robinson via eli)
+
 Release 0.23.3 - UNRELEASED
 Release 0.23.3 - UNRELEASED
 
 
   INCOMPATIBLE CHANGES
   INCOMPATIBLE CHANGES
@@ -547,6 +568,8 @@ Release 0.23.3 - UNRELEASED
 
 
     HADOOP-8341. Fix or filter findbugs issues in hadoop-tools (bobby)
     HADOOP-8341. Fix or filter findbugs issues in hadoop-tools (bobby)
 
 
+    HADOOP-8373. Port RPC.getServerAddress to 0.23 (Daryn Sharp via bobby)
+
 Release 0.23.2 - UNRELEASED 
 Release 0.23.2 - UNRELEASED 
 
 
   INCOMPATIBLE CHANGES
   INCOMPATIBLE CHANGES

+ 2 - 67
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/BlockLocation.java

@@ -35,16 +35,7 @@ import org.apache.hadoop.io.WritableFactory;
  */
  */
 @InterfaceAudience.Public
 @InterfaceAudience.Public
 @InterfaceStability.Stable
 @InterfaceStability.Stable
-public class BlockLocation implements Writable {
-
-  static {               // register a ctor
-    WritableFactories.setFactory
-      (BlockLocation.class,
-       new WritableFactory() {
-         public Writable newInstance() { return new BlockLocation(); }
-       });
-  }
-
+public class BlockLocation {
   private String[] hosts; //hostnames of datanodes
   private String[] hosts; //hostnames of datanodes
   private String[] names; //hostname:portNumber of datanodes
   private String[] names; //hostname:portNumber of datanodes
   private String[] topologyPaths; // full path name in network topology
   private String[] topologyPaths; // full path name in network topology
@@ -219,62 +210,6 @@ public class BlockLocation implements Writable {
     }
     }
   }
   }
 
 
-  /**
-   * Implement write of Writable
-   */
-  public void write(DataOutput out) throws IOException {
-    out.writeLong(offset);
-    out.writeLong(length);
-    out.writeBoolean(corrupt);
-    out.writeInt(names.length);
-    for (int i=0; i < names.length; i++) {
-      Text name = new Text(names[i]);
-      name.write(out);
-    }
-    out.writeInt(hosts.length);
-    for (int i=0; i < hosts.length; i++) {
-      Text host = new Text(hosts[i]);
-      host.write(out);
-    }
-    out.writeInt(topologyPaths.length);
-    for (int i=0; i < topologyPaths.length; i++) {
-      Text host = new Text(topologyPaths[i]);
-      host.write(out);
-    }
-  }
-  
-  /**
-   * Implement readFields of Writable
-   */
-  public void readFields(DataInput in) throws IOException {
-    this.offset = in.readLong();
-    this.length = in.readLong();
-    this.corrupt = in.readBoolean();
-    int numNames = in.readInt();
-    this.names = new String[numNames];
-    for (int i = 0; i < numNames; i++) {
-      Text name = new Text();
-      name.readFields(in);
-      names[i] = name.toString();
-    }
-    
-    int numHosts = in.readInt();
-    this.hosts = new String[numHosts];
-    for (int i = 0; i < numHosts; i++) {
-      Text host = new Text();
-      host.readFields(in);
-      hosts[i] = host.toString();
-    }
-    
-    int numTops = in.readInt();
-    topologyPaths = new String[numTops];
-    for (int i = 0; i < numTops; i++) {
-      Text path = new Text();
-      path.readFields(in);
-      topologyPaths[i] = path.toString();
-    }
-  }
-  
   public String toString() {
   public String toString() {
     StringBuilder result = new StringBuilder();
     StringBuilder result = new StringBuilder();
     result.append(offset);
     result.append(offset);
@@ -289,4 +224,4 @@ public class BlockLocation implements Writable {
     }
     }
     return result.toString();
     return result.toString();
   }
   }
-}
+}

+ 8 - 8
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileStatus.java

@@ -254,7 +254,7 @@ public class FileStatus implements Writable, Comparable {
   // Writable
   // Writable
   //////////////////////////////////////////////////
   //////////////////////////////////////////////////
   public void write(DataOutput out) throws IOException {
   public void write(DataOutput out) throws IOException {
-    Text.writeString(out, getPath().toString());
+    Text.writeString(out, getPath().toString(), Text.ONE_MEGABYTE);
     out.writeLong(getLen());
     out.writeLong(getLen());
     out.writeBoolean(isDirectory());
     out.writeBoolean(isDirectory());
     out.writeShort(getReplication());
     out.writeShort(getReplication());
@@ -262,16 +262,16 @@ public class FileStatus implements Writable, Comparable {
     out.writeLong(getModificationTime());
     out.writeLong(getModificationTime());
     out.writeLong(getAccessTime());
     out.writeLong(getAccessTime());
     getPermission().write(out);
     getPermission().write(out);
-    Text.writeString(out, getOwner());
-    Text.writeString(out, getGroup());
+    Text.writeString(out, getOwner(), Text.ONE_MEGABYTE);
+    Text.writeString(out, getGroup(), Text.ONE_MEGABYTE);
     out.writeBoolean(isSymlink());
     out.writeBoolean(isSymlink());
     if (isSymlink()) {
     if (isSymlink()) {
-      Text.writeString(out, getSymlink().toString());
+      Text.writeString(out, getSymlink().toString(), Text.ONE_MEGABYTE);
     }
     }
   }
   }
 
 
   public void readFields(DataInput in) throws IOException {
   public void readFields(DataInput in) throws IOException {
-    String strPath = Text.readString(in);
+    String strPath = Text.readString(in, Text.ONE_MEGABYTE);
     this.path = new Path(strPath);
     this.path = new Path(strPath);
     this.length = in.readLong();
     this.length = in.readLong();
     this.isdir = in.readBoolean();
     this.isdir = in.readBoolean();
@@ -280,10 +280,10 @@ public class FileStatus implements Writable, Comparable {
     modification_time = in.readLong();
     modification_time = in.readLong();
     access_time = in.readLong();
     access_time = in.readLong();
     permission.readFields(in);
     permission.readFields(in);
-    owner = Text.readString(in);
-    group = Text.readString(in);
+    owner = Text.readString(in, Text.ONE_MEGABYTE);
+    group = Text.readString(in, Text.ONE_MEGABYTE);
     if (in.readBoolean()) {
     if (in.readBoolean()) {
-      this.symlink = new Path(Text.readString(in));
+      this.symlink = new Path(Text.readString(in, Text.ONE_MEGABYTE));
     } else {
     } else {
       this.symlink = null;
       this.symlink = null;
     }
     }

+ 4 - 4
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/permission/PermissionStatus.java

@@ -84,8 +84,8 @@ public class PermissionStatus implements Writable {
 
 
   /** {@inheritDoc} */
   /** {@inheritDoc} */
   public void readFields(DataInput in) throws IOException {
   public void readFields(DataInput in) throws IOException {
-    username = Text.readString(in);
-    groupname = Text.readString(in);
+    username = Text.readString(in, Text.ONE_MEGABYTE);
+    groupname = Text.readString(in, Text.ONE_MEGABYTE);
     permission = FsPermission.read(in);
     permission = FsPermission.read(in);
   }
   }
 
 
@@ -110,8 +110,8 @@ public class PermissionStatus implements Writable {
                            String username, 
                            String username, 
                            String groupname,
                            String groupname,
                            FsPermission permission) throws IOException {
                            FsPermission permission) throws IOException {
-    Text.writeString(out, username);
-    Text.writeString(out, groupname);
+    Text.writeString(out, username, Text.ONE_MEGABYTE);
+    Text.writeString(out, groupname, Text.ONE_MEGABYTE);
     permission.write(out);
     permission.write(out);
   }
   }
 
 

+ 29 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/Text.java

@@ -412,6 +412,8 @@ public class Text extends BinaryComparable
     return bytes;
     return bytes;
   }
   }
 
 
+  static final public int ONE_MEGABYTE = 1024 * 1024;
+
   /** Read a UTF8 encoded string from in
   /** Read a UTF8 encoded string from in
    */
    */
   public static String readString(DataInput in) throws IOException {
   public static String readString(DataInput in) throws IOException {
@@ -420,7 +422,17 @@ public class Text extends BinaryComparable
     in.readFully(bytes, 0, length);
     in.readFully(bytes, 0, length);
     return decode(bytes);
     return decode(bytes);
   }
   }
-
+  
+  /** Read a UTF8 encoded string with a maximum size
+   */
+  public static String readString(DataInput in, int maxLength)
+      throws IOException {
+    int length = WritableUtils.readVIntInRange(in, 0, maxLength - 1);
+    byte [] bytes = new byte[length];
+    in.readFully(bytes, 0, length);
+    return decode(bytes);
+  }
+  
   /** Write a UTF8 encoded string to out
   /** Write a UTF8 encoded string to out
    */
    */
   public static int writeString(DataOutput out, String s) throws IOException {
   public static int writeString(DataOutput out, String s) throws IOException {
@@ -431,6 +443,22 @@ public class Text extends BinaryComparable
     return length;
     return length;
   }
   }
 
 
+  /** Write a UTF8 encoded string with a maximum size to out
+   */
+  public static int writeString(DataOutput out, String s, int maxLength)
+      throws IOException {
+    ByteBuffer bytes = encode(s);
+    int length = bytes.limit();
+    if (length >= maxLength) {
+      throw new IOException("string was too long to write!  Expected " +
+          "less than " + maxLength + " bytes, but got " +
+          length + " bytes.");
+    }
+    WritableUtils.writeVInt(out, length);
+    out.write(bytes.array(), 0, length);
+    return length;
+  }
+
   ////// states for validateUTF8
   ////// states for validateUTF8
   
   
   private static final int LEAD_BYTE = 0;
   private static final int LEAD_BYTE = 0;

+ 7 - 12
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/NetUtils.java

@@ -140,7 +140,7 @@ public class NetUtils {
 
 
   /**
   /**
    * Util method to build socket addr from either:
    * Util method to build socket addr from either:
-   *   <host>:<post>
+   *   <host>:<port>
    *   <fs>://<host>:<port>/<path>
    *   <fs>://<host>:<port>/<path>
    */
    */
   public static InetSocketAddress createSocketAddr(String target) {
   public static InetSocketAddress createSocketAddr(String target) {
@@ -150,7 +150,7 @@ public class NetUtils {
   /**
   /**
    * Util method to build socket addr from either:
    * Util method to build socket addr from either:
    *   <host>
    *   <host>
-   *   <host>:<post>
+   *   <host>:<port>
    *   <fs>://<host>:<port>/<path>
    *   <fs>://<host>:<port>/<path>
    */
    */
   public static InetSocketAddress createSocketAddr(String target,
   public static InetSocketAddress createSocketAddr(String target,
@@ -494,7 +494,7 @@ public class NetUtils {
    * also takes a local address and port to bind the socket to. 
    * also takes a local address and port to bind the socket to. 
    * 
    * 
    * @param socket
    * @param socket
-   * @param address the remote address
+   * @param endpoint the remote address
    * @param localAddr the local address to bind the socket to
    * @param localAddr the local address to bind the socket to
    * @param timeout timeout in milliseconds
    * @param timeout timeout in milliseconds
    */
    */
@@ -549,16 +549,11 @@ public class NetUtils {
    * @return its IP address in the string format
    * @return its IP address in the string format
    */
    */
   public static String normalizeHostName(String name) {
   public static String normalizeHostName(String name) {
-    if (Character.digit(name.charAt(0), 10) != -1) { // it is an IP
+    try {
+      return InetAddress.getByName(name).getHostAddress();
+    } catch (UnknownHostException e) {
       return name;
       return name;
-    } else {
-      try {
-        InetAddress ipAddress = InetAddress.getByName(name);
-        return ipAddress.getHostAddress();
-      } catch (UnknownHostException e) {
-        return name;
-      }
-    }
+    }    
   }
   }
   
   
   /** 
   /** 

+ 32 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/VersionUtil.java

@@ -22,11 +22,20 @@ import java.util.regex.Pattern;
 
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
 
 
+import com.google.common.collect.ComparisonChain;
+
 @InterfaceAudience.Private
 @InterfaceAudience.Private
 public abstract class VersionUtil {
 public abstract class VersionUtil {
   
   
   private static final Pattern COMPONENT_GROUPS = Pattern.compile("(\\d+)|(\\D+)");
   private static final Pattern COMPONENT_GROUPS = Pattern.compile("(\\d+)|(\\D+)");
 
 
+  /**
+   * Suffix added by maven for nightly builds and other snapshot releases.
+   * These releases are considered to precede the non-SNAPSHOT version
+   * with the same version number.
+   */
+  private static final String SNAPSHOT_SUFFIX = "-SNAPSHOT";
+
   /**
   /**
    * This function splits the two versions on &quot;.&quot; and performs a
    * This function splits the two versions on &quot;.&quot; and performs a
    * naturally-ordered comparison of the resulting components. For example, the
    * naturally-ordered comparison of the resulting components. For example, the
@@ -48,6 +57,11 @@ public abstract class VersionUtil {
    * between the two versions, then the version with fewer components is
    * between the two versions, then the version with fewer components is
    * considered to precede the version with more components.
    * considered to precede the version with more components.
    * 
    * 
+   * In addition to the above rules, there is one special case: maven SNAPSHOT
+   * releases are considered to precede a non-SNAPSHOT release with an
+   * otherwise identical version number. For example, 2.0-SNAPSHOT precedes
+   * 2.0.
+   * 
    * This function returns a negative integer if version1 precedes version2, a
    * This function returns a negative integer if version1 precedes version2, a
    * positive integer if version2 precedes version1, and 0 if and only if the
    * positive integer if version2 precedes version1, and 0 if and only if the
    * two versions' components are identical in value and cardinality.
    * two versions' components are identical in value and cardinality.
@@ -61,6 +75,11 @@ public abstract class VersionUtil {
    *         versions are equal.
    *         versions are equal.
    */
    */
   public static int compareVersions(String version1, String version2) {
   public static int compareVersions(String version1, String version2) {
+    boolean isSnapshot1 = version1.endsWith(SNAPSHOT_SUFFIX);
+    boolean isSnapshot2 = version2.endsWith(SNAPSHOT_SUFFIX);
+    version1 = stripSnapshotSuffix(version1);
+    version2 = stripSnapshotSuffix(version2);
+    
     String[] version1Parts = version1.split("\\.");
     String[] version1Parts = version1.split("\\.");
     String[] version2Parts = version2.split("\\.");
     String[] version2Parts = version2.split("\\.");
     
     
@@ -87,9 +106,21 @@ public abstract class VersionUtil {
         return component1.length() - component2.length();
         return component1.length() - component2.length();
       }
       }
     }
     }
-    return version1Parts.length - version2Parts.length;
+    
+    return ComparisonChain.start()
+      .compare(version1Parts.length, version2Parts.length)
+      .compare(isSnapshot2, isSnapshot1)
+      .result();
   }
   }
   
   
+  private static String stripSnapshotSuffix(String version) {
+    if (version.endsWith(SNAPSHOT_SUFFIX)) {
+      return version.substring(0, version.length() - SNAPSHOT_SUFFIX.length());
+    } else {
+      return version;
+    }
+  }
+
   private static boolean isNumeric(String s) {
   private static boolean isNumeric(String s) {
     try {
     try {
       Integer.parseInt(s);
       Integer.parseInt(s);

+ 3 - 3
hadoop-common-project/hadoop-common/src/main/native/acinclude.m4

@@ -1,4 +1,4 @@
-# AC_COMPUTE_NEEDED_DSO(LIBRARY, PREPROC_SYMBOL)
+# AC_COMPUTE_NEEDED_DSO(LIBRARY, TEST_PROGRAM, PREPROC_SYMBOL)
 # --------------------------------------------------
 # --------------------------------------------------
 # Compute the 'actual' dynamic-library used 
 # Compute the 'actual' dynamic-library used 
 # for LIBRARY and set it to PREPROC_SYMBOL
 # for LIBRARY and set it to PREPROC_SYMBOL
@@ -6,7 +6,7 @@ AC_DEFUN([AC_COMPUTE_NEEDED_DSO],
 [
 [
 AC_CACHE_CHECK([Checking for the 'actual' dynamic-library for '-l$1'], ac_cv_libname_$1,
 AC_CACHE_CHECK([Checking for the 'actual' dynamic-library for '-l$1'], ac_cv_libname_$1,
   [
   [
-  echo 'int main(int argc, char **argv){return 0;}' > conftest.c
+  echo '$2' > conftest.c
   if test -z "`${CC} ${LDFLAGS} -o conftest conftest.c -l$1 2>&1`"; then
   if test -z "`${CC} ${LDFLAGS} -o conftest conftest.c -l$1 2>&1`"; then
     dnl Try objdump and ldd in that order to get the dynamic library
     dnl Try objdump and ldd in that order to get the dynamic library
     if test ! -z "`which objdump | grep -v 'no objdump'`"; then
     if test ! -z "`which objdump | grep -v 'no objdump'`"; then
@@ -24,5 +24,5 @@ AC_CACHE_CHECK([Checking for the 'actual' dynamic-library for '-l$1'], ac_cv_lib
   rm -f conftest*
   rm -f conftest*
   ]
   ]
 )
 )
-AC_DEFINE_UNQUOTED($2, ${ac_cv_libname_$1}, [The 'actual' dynamic-library for '-l$1'])
+AC_DEFINE_UNQUOTED($3, ${ac_cv_libname_$1}, [The 'actual' dynamic-library for '-l$1'])
 ])# AC_COMPUTE_NEEDED_DSO
 ])# AC_COMPUTE_NEEDED_DSO

+ 12 - 2
hadoop-common-project/hadoop-common/src/main/native/configure.ac

@@ -87,10 +87,20 @@ CPPFLAGS=$cppflags_bak
 AC_SUBST([JNI_CPPFLAGS])
 AC_SUBST([JNI_CPPFLAGS])
 
 
 dnl Check for zlib headers
 dnl Check for zlib headers
-AC_CHECK_HEADERS([zlib.h zconf.h], AC_COMPUTE_NEEDED_DSO(z,HADOOP_ZLIB_LIBRARY), AC_MSG_ERROR(Zlib headers were not found... native-hadoop library needs zlib to build. Please install the requisite zlib development package.))
+AC_CHECK_HEADERS([zlib.h zconf.h],
+  AC_COMPUTE_NEEDED_DSO(z,
+    [#include "zlib.h"
+    int main(int argc, char **argv){zlibVersion();return 0;}],
+    HADOOP_ZLIB_LIBRARY),
+  AC_MSG_ERROR(Zlib headers were not found... native-hadoop library needs zlib to build. Please install the requisite zlib development package.))
 
 
 dnl Check for snappy headers
 dnl Check for snappy headers
-AC_CHECK_HEADERS([snappy-c.h], AC_COMPUTE_NEEDED_DSO(snappy,HADOOP_SNAPPY_LIBRARY), AC_MSG_WARN(Snappy headers were not found... building without snappy.))
+AC_CHECK_HEADERS([snappy-c.h],
+  AC_COMPUTE_NEEDED_DSO(snappy,
+    [#include "snappy-c.h"
+    int main(int argc, char **argv){snappy_compress(0,0,0,0);return 0;}],
+    HADOOP_SNAPPY_LIBRARY),
+  AC_MSG_WARN(Snappy headers were not found... building without snappy.))
 
 
 dnl Check for headers needed by the native Group resolution implementation
 dnl Check for headers needed by the native Group resolution implementation
 AC_CHECK_HEADERS([fcntl.h stdlib.h string.h unistd.h], [], AC_MSG_ERROR(Some system headers not found... please ensure their presence on your platform.))
 AC_CHECK_HEADERS([fcntl.h stdlib.h string.h unistd.h], [], AC_MSG_ERROR(Some system headers not found... please ensure their presence on your platform.))

+ 0 - 78
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestBlockLocation.java

@@ -1,78 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.fs;
-
-import java.io.ByteArrayInputStream;
-import java.io.DataInput;
-import java.io.DataInputStream;
-import java.io.IOException;
-
-import junit.framework.TestCase;
-
-import org.apache.hadoop.io.DataOutputBuffer;
-
-public class TestBlockLocation extends TestCase {
-  // Verify fix of bug identified in HADOOP-6004
-  public void testDeserialization() throws IOException {
-    // Create a test BlockLocation
-    String[] names = {"one", "two" };
-    String[] hosts = {"three", "four" };
-    String[] topologyPaths = {"five", "six"};
-    long offset = 25l;
-    long length = 55l;
-    
-    BlockLocation bl = new BlockLocation(names, hosts, topologyPaths, 
-                                         offset, length);
-    
-    DataOutputBuffer dob = new DataOutputBuffer();
-    
-    // Serialize it
-    try {
-      bl.write(dob);
-    } catch (IOException e) {
-      fail("Unable to serialize data: " + e.getMessage());
-    }
-
-    byte[] bytes = dob.getData();
-    DataInput da = new DataInputStream(new ByteArrayInputStream(bytes));
-
-    // Try to re-create the BlockLocation the same way as is done during
-    // deserialization
-    BlockLocation bl2 = new BlockLocation();
-    
-    try {
-      bl2.readFields(da);
-    } catch (IOException e) {
-      fail("Unable to deserialize BlockLocation: " + e.getMessage());
-    }
-    
-    // Check that we got back what we started with
-    verifyDeserialization(bl2.getHosts(), hosts);
-    verifyDeserialization(bl2.getNames(), names);
-    verifyDeserialization(bl2.getTopologyPaths(), topologyPaths);
-    assertEquals(bl2.getOffset(), offset);
-    assertEquals(bl2.getLength(), length);
-  }
-
-  private void verifyDeserialization(String[] ar1, String[] ar2) {
-    assertEquals(ar1.length, ar2.length);
-    
-    for(int i = 0; i < ar1.length; i++)
-      assertEquals(ar1[i], ar2[i]);
-  }
-}

+ 35 - 1
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/TestText.java

@@ -20,6 +20,7 @@ package org.apache.hadoop.io;
 
 
 import junit.framework.TestCase;
 import junit.framework.TestCase;
 
 
+import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.nio.ByteBuffer;
 import java.nio.charset.CharacterCodingException;
 import java.nio.charset.CharacterCodingException;
 import java.util.Random;
 import java.util.Random;
@@ -107,7 +108,6 @@ public class TestText extends TestCase {
     }
     }
   }
   }
   
   
-  
   public void testIO() throws Exception {
   public void testIO() throws Exception {
     DataOutputBuffer out = new DataOutputBuffer();
     DataOutputBuffer out = new DataOutputBuffer();
     DataInputBuffer in = new DataInputBuffer();
     DataInputBuffer in = new DataInputBuffer();
@@ -136,6 +136,40 @@ public class TestText extends TestCase {
       assertTrue(before.equals(after2));
       assertTrue(before.equals(after2));
     }
     }
   }
   }
+  
+  public void doTestLimitedIO(String str, int strLen) throws IOException {
+    DataOutputBuffer out = new DataOutputBuffer();
+    DataInputBuffer in = new DataInputBuffer();
+
+    out.reset();
+    try {
+      Text.writeString(out, str, strLen);
+      fail("expected writeString to fail when told to write a string " +
+          "that was too long!  The string was '" + str + "'");
+    } catch (IOException e) {
+    }
+    Text.writeString(out, str, strLen + 1);
+
+    // test that it reads correctly
+    in.reset(out.getData(), out.getLength());
+    in.mark(strLen);
+    String after;
+    try {
+      after = Text.readString(in, strLen);
+      fail("expected readString to fail when told to read a string " +
+          "that was too long!  The string was '" + str + "'");
+    } catch (IOException e) {
+    }
+    in.reset();
+    after = Text.readString(in, strLen + 1);
+    assertTrue(str.equals(after));
+  }
+  
+  public void testLimitedIO() throws Exception {
+    doTestLimitedIO("abcd", 4);
+    doTestLimitedIO("", 0);
+    doTestLimitedIO("1", 1);
+  }
 
 
   public void testCompare() throws Exception {
   public void testCompare() throws Exception {
     DataOutputBuffer out1 = new DataOutputBuffer();
     DataOutputBuffer out1 = new DataOutputBuffer();

+ 23 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPC.java

@@ -322,6 +322,29 @@ public class TestRPC {
     server.stop();    
     server.stop();    
   }
   }
 
 
+  @Test
+  public void testProxyAddress() throws Exception {
+    Server server = RPC.getServer(TestProtocol.class,
+                                  new TestImpl(), ADDRESS, 0, conf);
+    TestProtocol proxy = null;
+    
+    try {
+      server.start();
+      InetSocketAddress addr = NetUtils.getConnectAddress(server);
+
+      // create a client
+      proxy = (TestProtocol)RPC.getProxy(
+          TestProtocol.class, TestProtocol.versionID, addr, conf);
+      
+      assertEquals(addr, RPC.getServerAddress(proxy));
+    } finally {
+      server.stop();
+      if (proxy != null) {
+        RPC.stopProxy(proxy);
+      }
+    }
+  }
+
   @Test
   @Test
   public void testSlowRpc() throws Exception {
   public void testSlowRpc() throws Exception {
     System.out.println("Testing Slow RPC");
     System.out.println("Testing Slow RPC");

+ 22 - 2
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestNetUtils.java

@@ -31,7 +31,9 @@ import java.net.SocketException;
 import java.net.SocketTimeoutException;
 import java.net.SocketTimeoutException;
 import java.net.URI;
 import java.net.URI;
 import java.net.UnknownHostException;
 import java.net.UnknownHostException;
+import java.util.Arrays;
 import java.util.Enumeration;
 import java.util.Enumeration;
+import java.util.List;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeUnit;
 
 
 import junit.framework.AssertionFailedError;
 import junit.framework.AssertionFailedError;
@@ -42,8 +44,6 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.security.NetUtilsTestResolver;
 import org.apache.hadoop.security.NetUtilsTestResolver;
-import org.apache.hadoop.test.MultithreadedTestUtil.TestContext;
-import org.apache.hadoop.test.MultithreadedTestUtil.TestingThread;
 import org.junit.Assume;
 import org.junit.Assume;
 import org.junit.Before;
 import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.BeforeClass;
@@ -599,6 +599,26 @@ public class TestNetUtils {
     assertEquals("scheme://host.a.b/path", uri.toString());
     assertEquals("scheme://host.a.b/path", uri.toString());
   }
   }
   
   
+  /** 
+   * Test for {@link NetUtils#normalizeHostNames}
+   */
+  @Test
+  public void testNormalizeHostName() {	
+    List<String> hosts = Arrays.asList(new String[] {"127.0.0.1",
+        "localhost", "3w.org", "UnknownHost"});
+    List<String> normalizedHosts = NetUtils.normalizeHostNames(hosts);
+    // when ipaddress is normalized, same address is expected in return
+    assertEquals(normalizedHosts.get(0), hosts.get(0));
+    // for normalizing a resolvable hostname, resolved ipaddress is expected in return
+    assertFalse(normalizedHosts.get(1).equals(hosts.get(1)));
+    assertEquals(normalizedHosts.get(1), hosts.get(0));
+    // this address HADOOP-8372: when normalizing a valid resolvable hostname start with numeric, 
+    // its ipaddress is expected to return
+    assertFalse(normalizedHosts.get(2).equals(hosts.get(2)));
+    // return the same hostname after normalizing a irresolvable hostname.
+    assertEquals(normalizedHosts.get(3), hosts.get(3));
+  }
+  
   @Test
   @Test
   public void testGetHostNameOfIP() {
   public void testGetHostNameOfIP() {
     assertNull(NetUtils.getHostNameOfIP(null));
     assertNull(NetUtils.getHostNameOfIP(null));

+ 6 - 6
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestTableMapping.java

@@ -41,7 +41,7 @@ public class TestTableMapping {
   public void setUp() throws IOException {
   public void setUp() throws IOException {
     mappingFile = File.createTempFile(getClass().getSimpleName(), ".txt");
     mappingFile = File.createTempFile(getClass().getSimpleName(), ".txt");
     Files.write("a.b.c /rack1\n" +
     Files.write("a.b.c /rack1\n" +
-                "1.2.3\t/rack2\n", mappingFile, Charsets.UTF_8);
+                "1.2.3.4\t/rack2\n", mappingFile, Charsets.UTF_8);
     mappingFile.deleteOnExit();
     mappingFile.deleteOnExit();
   }
   }
 
 
@@ -55,7 +55,7 @@ public class TestTableMapping {
 
 
     List<String> names = new ArrayList<String>();
     List<String> names = new ArrayList<String>();
     names.add("a.b.c");
     names.add("a.b.c");
-    names.add("1.2.3");
+    names.add("1.2.3.4");
 
 
     List<String> result = mapping.resolve(names);
     List<String> result = mapping.resolve(names);
     assertEquals(names.size(), result.size());
     assertEquals(names.size(), result.size());
@@ -73,7 +73,7 @@ public class TestTableMapping {
 
 
     List<String> names = new ArrayList<String>();
     List<String> names = new ArrayList<String>();
     names.add("a.b.c");
     names.add("a.b.c");
-    names.add("1.2.3");
+    names.add("1.2.3.4");
 
 
     List<String> result1 = mapping.resolve(names);
     List<String> result1 = mapping.resolve(names);
     assertEquals(names.size(), result1.size());
     assertEquals(names.size(), result1.size());
@@ -96,7 +96,7 @@ public class TestTableMapping {
 
 
     List<String> names = new ArrayList<String>();
     List<String> names = new ArrayList<String>();
     names.add("a.b.c");
     names.add("a.b.c");
-    names.add("1.2.3");
+    names.add("1.2.3.4");
 
 
     List<String> result = mapping.resolve(names);
     List<String> result = mapping.resolve(names);
     assertEquals(names.size(), result.size());
     assertEquals(names.size(), result.size());
@@ -114,7 +114,7 @@ public class TestTableMapping {
 
 
     List<String> names = new ArrayList<String>();
     List<String> names = new ArrayList<String>();
     names.add("a.b.c");
     names.add("a.b.c");
-    names.add("1.2.3");
+    names.add("1.2.3.4");
 
 
     List<String> result = mapping.resolve(names);
     List<String> result = mapping.resolve(names);
     assertEquals(names.size(), result.size());
     assertEquals(names.size(), result.size());
@@ -134,7 +134,7 @@ public class TestTableMapping {
 
 
     List<String> names = new ArrayList<String>();
     List<String> names = new ArrayList<String>();
     names.add("a.b.c");
     names.add("a.b.c");
-    names.add("1.2.3");
+    names.add("1.2.3.4");
 
 
     List<String> result = mapping.resolve(names);
     List<String> result = mapping.resolve(names);
     assertEquals(names.size(), result.size());
     assertEquals(names.size(), result.size());

+ 9 - 1
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestVersionUtil.java

@@ -19,7 +19,6 @@ package org.apache.hadoop.util;
 
 
 import static org.junit.Assert.*;
 import static org.junit.Assert.*;
 
 
-import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.Test;
 import org.junit.Test;
 
 
 public class TestVersionUtil {
 public class TestVersionUtil {
@@ -30,6 +29,8 @@ public class TestVersionUtil {
     assertEquals(0, VersionUtil.compareVersions("2.0.0", "2.0.0"));
     assertEquals(0, VersionUtil.compareVersions("2.0.0", "2.0.0"));
     assertEquals(0, VersionUtil.compareVersions("2.0.0a", "2.0.0a"));
     assertEquals(0, VersionUtil.compareVersions("2.0.0a", "2.0.0a"));
     assertEquals(0, VersionUtil.compareVersions("1", "1"));
     assertEquals(0, VersionUtil.compareVersions("1", "1"));
+    assertEquals(0, VersionUtil.compareVersions(
+        "2.0.0-SNAPSHOT", "2.0.0-SNAPSHOT"));
     
     
     // Assert that lower versions are lower, and higher versions are higher.
     // Assert that lower versions are lower, and higher versions are higher.
     assertExpectedValues("1", "2.0.0");
     assertExpectedValues("1", "2.0.0");
@@ -52,6 +53,13 @@ public class TestVersionUtil {
     assertExpectedValues("1.0.0a2", "1.0.0a10");
     assertExpectedValues("1.0.0a2", "1.0.0a10");
     assertExpectedValues("1.0", "1.a");
     assertExpectedValues("1.0", "1.a");
     assertExpectedValues("1.0", "1.a0");
     assertExpectedValues("1.0", "1.a0");
+    
+    // Snapshot builds precede their eventual releases.
+    assertExpectedValues("1.0-SNAPSHOT", "1.0");
+    assertExpectedValues("1.0", "1.0.0-SNAPSHOT");
+    assertExpectedValues("1.0.0-SNAPSHOT", "1.0.0");
+    assertExpectedValues("1.0.0", "1.0.1-SNAPSHOT");
+    assertExpectedValues("1.0.1-SNAPSHOT", "1.0.1");
   }
   }
   
   
   private static void assertExpectedValues(String lower, String higher) {
   private static void assertExpectedValues(String lower, String higher) {

+ 27 - 0
hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt

@@ -435,6 +435,22 @@ Release 2.0.0 - UNRELEASED
     so that INodeFile and INodeFileUnderConstruction do not have to be used in
     so that INodeFile and INodeFileUnderConstruction do not have to be used in
     block management.  (John George via szetszwo)
     block management.  (John George via szetszwo)
 
 
+    HDFS-3390. DFSAdmin should print full stack traces of errors when DEBUG
+    logging is enabled. (atm)
+
+    HDFS-3341. Change minimum RPC versions to respective SNAPSHOTs instead of
+    final releases. (todd)
+
+    HDFS-3369. Rename {get|set|add}INode(..) methods in BlockManager and
+    BlocksMap to {get|set|add}BlockCollection(..).  (John George via szetszwo)
+
+    HDFS-3134. harden edit log loader against malformed or malicious input.
+    (Colin Patrick McCabe via eli)
+
+    HDFS-3230. Cleanup DatanodeID creation in the tests. (eli)
+
+    HDFS-3401. Cleanup DatanodeDescriptor creation in the tests. (eli)
+
   OPTIMIZATIONS
   OPTIMIZATIONS
 
 
     HDFS-3024. Improve performance of stringification in addStoredBlock (todd)
     HDFS-3024. Improve performance of stringification in addStoredBlock (todd)
@@ -621,6 +637,17 @@ Release 2.0.0 - UNRELEASED
     HDFS-3376. DFSClient fails to make connection to DN if there are many
     HDFS-3376. DFSClient fails to make connection to DN if there are many
     unusable cached sockets (todd)
     unusable cached sockets (todd)
 
 
+    HDFS-3328. NPE in DataNode.getIpcPort. (eli)
+
+    HDFS-3396. FUSE build fails on Ubuntu 12.04. (Colin Patrick McCabe via eli)
+
+    HDFS-3395. NN doesn't start with HA+security enabled and HTTP address
+    set to 0.0.0.0. (atm)
+
+    HDFS-3385. The last block of INodeFileUnderConstruction is not
+    necessarily a BlockInfoUnderConstruction, so do not cast it in
+    FSNamesystem.recoverLeaseInternal(..).  (szetszwo)
+
   BREAKDOWN OF HDFS-1623 SUBTASKS
   BREAKDOWN OF HDFS-1623 SUBTASKS
 
 
     HDFS-2179. Add fencing framework and mechanisms for NameNode HA. (todd)
     HDFS-2179. Add fencing framework and mechanisms for NameNode HA. (todd)

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs/src/contrib/fuse-dfs/src/Makefile.am

@@ -18,4 +18,5 @@ bin_PROGRAMS = fuse_dfs
 fuse_dfs_SOURCES = fuse_dfs.c fuse_options.c fuse_trash.c fuse_stat_struct.c fuse_users.c fuse_init.c fuse_connect.c fuse_impls_access.c fuse_impls_chmod.c  fuse_impls_chown.c  fuse_impls_create.c  fuse_impls_flush.c fuse_impls_getattr.c  fuse_impls_mkdir.c  fuse_impls_mknod.c  fuse_impls_open.c fuse_impls_read.c fuse_impls_release.c fuse_impls_readdir.c fuse_impls_rename.c fuse_impls_rmdir.c fuse_impls_statfs.c fuse_impls_symlink.c fuse_impls_truncate.c fuse_impls_utimens.c  fuse_impls_unlink.c fuse_impls_write.c
 fuse_dfs_SOURCES = fuse_dfs.c fuse_options.c fuse_trash.c fuse_stat_struct.c fuse_users.c fuse_init.c fuse_connect.c fuse_impls_access.c fuse_impls_chmod.c  fuse_impls_chown.c  fuse_impls_create.c  fuse_impls_flush.c fuse_impls_getattr.c  fuse_impls_mkdir.c  fuse_impls_mknod.c  fuse_impls_open.c fuse_impls_read.c fuse_impls_release.c fuse_impls_readdir.c fuse_impls_rename.c fuse_impls_rmdir.c fuse_impls_statfs.c fuse_impls_symlink.c fuse_impls_truncate.c fuse_impls_utimens.c  fuse_impls_unlink.c fuse_impls_write.c
 AM_CFLAGS= -Wall -g
 AM_CFLAGS= -Wall -g
 AM_CPPFLAGS= -DPERMS=$(PERMS) -D_FILE_OFFSET_BITS=64 -I$(JAVA_HOME)/include -I$(HADOOP_PREFIX)/../../src/main/native -I$(JAVA_HOME)/include/linux -D_FUSE_DFS_VERSION=\"$(PACKAGE_VERSION)\" -DPROTECTED_PATHS=\"$(PROTECTED_PATHS)\" -I$(FUSE_HOME)/include
 AM_CPPFLAGS= -DPERMS=$(PERMS) -D_FILE_OFFSET_BITS=64 -I$(JAVA_HOME)/include -I$(HADOOP_PREFIX)/../../src/main/native -I$(JAVA_HOME)/include/linux -D_FUSE_DFS_VERSION=\"$(PACKAGE_VERSION)\" -DPROTECTED_PATHS=\"$(PROTECTED_PATHS)\" -I$(FUSE_HOME)/include
-AM_LDFLAGS= -L$(HADOOP_PREFIX)/../../target/native/target/usr/local/lib -lhdfs -L$(FUSE_HOME)/lib -lfuse -L$(JAVA_HOME)/jre/lib/$(OS_ARCH)/server -ljvm -lm
+AM_LDFLAGS= -L$(HADOOP_PREFIX)/../../target/native/target/usr/local/lib64 -L$(HADOOP_PREFIX)/../../target/native/target/usr/local/lib -L$(FUSE_HOME)/lib -L$(JAVA_HOME)/jre/lib/$(OS_ARCH)/server
+fuse_dfs_LDADD=-lfuse -lhdfs -ljvm -lm

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java

@@ -145,7 +145,7 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final String  DFS_NAMENODE_NUM_EXTRA_EDITS_RETAINED_KEY = "dfs.namenode.num.extra.edits.retained";
   public static final String  DFS_NAMENODE_NUM_EXTRA_EDITS_RETAINED_KEY = "dfs.namenode.num.extra.edits.retained";
   public static final int     DFS_NAMENODE_NUM_EXTRA_EDITS_RETAINED_DEFAULT = 1000000; //1M
   public static final int     DFS_NAMENODE_NUM_EXTRA_EDITS_RETAINED_DEFAULT = 1000000; //1M
   public static final String  DFS_NAMENODE_MIN_SUPPORTED_DATANODE_VERSION_KEY = "dfs.namenode.min.supported.datanode.version";
   public static final String  DFS_NAMENODE_MIN_SUPPORTED_DATANODE_VERSION_KEY = "dfs.namenode.min.supported.datanode.version";
-  public static final String  DFS_NAMENODE_MIN_SUPPORTED_DATANODE_VERSION_DEFAULT = "3.0.0";
+  public static final String  DFS_NAMENODE_MIN_SUPPORTED_DATANODE_VERSION_DEFAULT = "3.0.0-SNAPSHOT";
 
 
   public static final String  DFS_NAMENODE_EDITS_DIR_MINIMUM_KEY = "dfs.namenode.edits.dir.minimum";
   public static final String  DFS_NAMENODE_EDITS_DIR_MINIMUM_KEY = "dfs.namenode.edits.dir.minimum";
   public static final int     DFS_NAMENODE_EDITS_DIR_MINIMUM_DEFAULT = 1;
   public static final int     DFS_NAMENODE_EDITS_DIR_MINIMUM_DEFAULT = 1;
@@ -273,7 +273,7 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final int     DFS_DATANODE_IPC_DEFAULT_PORT = 50020;
   public static final int     DFS_DATANODE_IPC_DEFAULT_PORT = 50020;
   public static final String  DFS_DATANODE_IPC_ADDRESS_DEFAULT = "0.0.0.0" + DFS_DATANODE_IPC_DEFAULT_PORT;
   public static final String  DFS_DATANODE_IPC_ADDRESS_DEFAULT = "0.0.0.0" + DFS_DATANODE_IPC_DEFAULT_PORT;
   public static final String  DFS_DATANODE_MIN_SUPPORTED_NAMENODE_VERSION_KEY = "dfs.datanode.min.supported.namenode.version";
   public static final String  DFS_DATANODE_MIN_SUPPORTED_NAMENODE_VERSION_KEY = "dfs.datanode.min.supported.namenode.version";
-  public static final String  DFS_DATANODE_MIN_SUPPORTED_NAMENODE_VERSION_DEFAULT = "3.0.0";
+  public static final String  DFS_DATANODE_MIN_SUPPORTED_NAMENODE_VERSION_DEFAULT = "3.0.0-SNAPSHOT";
 
 
   public static final String  DFS_BLOCK_ACCESS_TOKEN_ENABLE_KEY = "dfs.block.access.token.enable";
   public static final String  DFS_BLOCK_ACCESS_TOKEN_ENABLE_KEY = "dfs.block.access.token.enable";
   public static final boolean DFS_BLOCK_ACCESS_TOKEN_ENABLE_DEFAULT = false;
   public static final boolean DFS_BLOCK_ACCESS_TOKEN_ENABLE_DEFAULT = false;

+ 4 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java

@@ -715,8 +715,11 @@ public class DFSUtil {
   public static String substituteForWildcardAddress(String configuredAddress,
   public static String substituteForWildcardAddress(String configuredAddress,
       String defaultHost) throws IOException {
       String defaultHost) throws IOException {
     InetSocketAddress sockAddr = NetUtils.createSocketAddr(configuredAddress);
     InetSocketAddress sockAddr = NetUtils.createSocketAddr(configuredAddress);
+    InetSocketAddress defaultSockAddr = NetUtils.createSocketAddr(defaultHost
+        + ":0");
     if (sockAddr.getAddress().isAnyLocalAddress()) {
     if (sockAddr.getAddress().isAnyLocalAddress()) {
-      if(UserGroupInformation.isSecurityEnabled()) {
+      if (UserGroupInformation.isSecurityEnabled() &&
+          defaultSockAddr.getAddress().isAnyLocalAddress()) {
         throw new IOException("Cannot use a wildcard address with security. " +
         throw new IOException("Cannot use a wildcard address with security. " +
                               "Must explicitly set bind address for Kerberos");
                               "Must explicitly set bind address for Kerberos");
       }
       }

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/block/BlockTokenIdentifier.java

@@ -148,7 +148,8 @@ public class BlockTokenIdentifier extends TokenIdentifier {
     userId = WritableUtils.readString(in);
     userId = WritableUtils.readString(in);
     blockPoolId = WritableUtils.readString(in);
     blockPoolId = WritableUtils.readString(in);
     blockId = WritableUtils.readVLong(in);
     blockId = WritableUtils.readVLong(in);
-    int length = WritableUtils.readVInt(in);
+    int length = WritableUtils.readVIntInRange(in, 0,
+        AccessMode.class.getEnumConstants().length);
     for (int i = 0; i < length; i++) {
     for (int i = 0; i < length; i++) {
       modes.add(WritableUtils.readEnum(in, AccessMode.class));
       modes.add(WritableUtils.readEnum(in, AccessMode.class));
     }
     }

+ 10 - 10
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java

@@ -32,7 +32,7 @@ import org.apache.hadoop.hdfs.util.LightWeightGSet;
 @InterfaceAudience.Private
 @InterfaceAudience.Private
 public class BlockInfo extends Block implements
 public class BlockInfo extends Block implements
     LightWeightGSet.LinkedElement {
     LightWeightGSet.LinkedElement {
-  private BlockCollection inode;
+  private BlockCollection bc;
 
 
   /** For implementing {@link LightWeightGSet.LinkedElement} interface */
   /** For implementing {@link LightWeightGSet.LinkedElement} interface */
   private LightWeightGSet.LinkedElement nextLinkedElement;
   private LightWeightGSet.LinkedElement nextLinkedElement;
@@ -57,13 +57,13 @@ public class BlockInfo extends Block implements
    */
    */
   public BlockInfo(int replication) {
   public BlockInfo(int replication) {
     this.triplets = new Object[3*replication];
     this.triplets = new Object[3*replication];
-    this.inode = null;
+    this.bc = null;
   }
   }
   
   
   public BlockInfo(Block blk, int replication) {
   public BlockInfo(Block blk, int replication) {
     super(blk);
     super(blk);
     this.triplets = new Object[3*replication];
     this.triplets = new Object[3*replication];
-    this.inode = null;
+    this.bc = null;
   }
   }
 
 
   /**
   /**
@@ -72,16 +72,16 @@ public class BlockInfo extends Block implements
    * @param from BlockInfo to copy from.
    * @param from BlockInfo to copy from.
    */
    */
   protected BlockInfo(BlockInfo from) {
   protected BlockInfo(BlockInfo from) {
-    this(from, from.inode.getReplication());
-    this.inode = from.inode;
+    this(from, from.bc.getReplication());
+    this.bc = from.bc;
   }
   }
 
 
-  public BlockCollection getINode() {
-    return inode;
+  public BlockCollection getBlockCollection() {
+    return bc;
   }
   }
 
 
-  public void setINode(BlockCollection inode) {
-    this.inode = inode;
+  public void setBlockCollection(BlockCollection bc) {
+    this.bc = bc;
   }
   }
 
 
   DatanodeDescriptor getDatanode(int index) {
   DatanodeDescriptor getDatanode(int index) {
@@ -334,7 +334,7 @@ public class BlockInfo extends Block implements
       BlockUCState s, DatanodeDescriptor[] targets) {
       BlockUCState s, DatanodeDescriptor[] targets) {
     if(isComplete()) {
     if(isComplete()) {
       return new BlockInfoUnderConstruction(
       return new BlockInfoUnderConstruction(
-          this, getINode().getReplication(), s, targets);
+          this, getBlockCollection().getReplication(), s, targets);
     }
     }
     // the block is already under construction
     // the block is already under construction
     BlockInfoUnderConstruction ucBlock = (BlockInfoUnderConstruction)this;
     BlockInfoUnderConstruction ucBlock = (BlockInfoUnderConstruction)this;

+ 68 - 70
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java

@@ -140,7 +140,7 @@ public class BlockManager {
   private final long replicationRecheckInterval;
   private final long replicationRecheckInterval;
   
   
   /**
   /**
-   * Mapping: Block -> { INode, datanodes, self ref }
+   * Mapping: Block -> { BlockCollection, datanodes, self ref }
    * Updated only in response to client-sent information.
    * Updated only in response to client-sent information.
    */
    */
   final BlocksMap blocksMap;
   final BlocksMap blocksMap;
@@ -190,7 +190,7 @@ public class BlockManager {
   public final short minReplication;
   public final short minReplication;
   /** Default number of replicas */
   /** Default number of replicas */
   public final int defaultReplication;
   public final int defaultReplication;
-  /** The maximum number of entries returned by getCorruptInodes() */
+  /** value returned by MAX_CORRUPT_FILES_RETURNED */
   final int maxCorruptFilesReturned;
   final int maxCorruptFilesReturned;
 
 
   /** variable to enable check for enough racks */
   /** variable to enable check for enough racks */
@@ -382,7 +382,7 @@ public class BlockManager {
                          numReplicas.decommissionedReplicas();
                          numReplicas.decommissionedReplicas();
     
     
     if (block instanceof BlockInfo) {
     if (block instanceof BlockInfo) {
-      String fileName = ((BlockInfo)block).getINode().getName();
+      String fileName = ((BlockInfo)block).getBlockCollection().getName();
       out.print(fileName + ": ");
       out.print(fileName + ": ");
     }
     }
     // l: == live:, d: == decommissioned c: == corrupt e: == excess
     // l: == live:, d: == decommissioned c: == corrupt e: == excess
@@ -452,17 +452,17 @@ public class BlockManager {
    * Commit the last block of the file and mark it as complete if it has
    * Commit the last block of the file and mark it as complete if it has
    * meets the minimum replication requirement
    * meets the minimum replication requirement
    * 
    * 
-   * @param fileINode file inode
+   * @param bc block collection
    * @param commitBlock - contains client reported block length and generation
    * @param commitBlock - contains client reported block length and generation
    * @return true if the last block is changed to committed state.
    * @return true if the last block is changed to committed state.
    * @throws IOException if the block does not have at least a minimal number
    * @throws IOException if the block does not have at least a minimal number
    * of replicas reported from data-nodes.
    * of replicas reported from data-nodes.
    */
    */
-  public boolean commitOrCompleteLastBlock(MutableBlockCollection fileINode, 
+  public boolean commitOrCompleteLastBlock(MutableBlockCollection bc, 
       Block commitBlock) throws IOException {
       Block commitBlock) throws IOException {
     if(commitBlock == null)
     if(commitBlock == null)
       return false; // not committing, this is a block allocation retry
       return false; // not committing, this is a block allocation retry
-    BlockInfo lastBlock = fileINode.getLastBlock();
+    BlockInfo lastBlock = bc.getLastBlock();
     if(lastBlock == null)
     if(lastBlock == null)
       return false; // no blocks in file yet
       return false; // no blocks in file yet
     if(lastBlock.isComplete())
     if(lastBlock.isComplete())
@@ -470,22 +470,22 @@ public class BlockManager {
     
     
     final boolean b = commitBlock((BlockInfoUnderConstruction)lastBlock, commitBlock);
     final boolean b = commitBlock((BlockInfoUnderConstruction)lastBlock, commitBlock);
     if(countNodes(lastBlock).liveReplicas() >= minReplication)
     if(countNodes(lastBlock).liveReplicas() >= minReplication)
-      completeBlock(fileINode, fileINode.numBlocks()-1, false);
+      completeBlock(bc, bc.numBlocks()-1, false);
     return b;
     return b;
   }
   }
 
 
   /**
   /**
    * Convert a specified block of the file to a complete block.
    * Convert a specified block of the file to a complete block.
-   * @param fileINode file
+   * @param bc file
    * @param blkIndex  block index in the file
    * @param blkIndex  block index in the file
    * @throws IOException if the block does not have at least a minimal number
    * @throws IOException if the block does not have at least a minimal number
    * of replicas reported from data-nodes.
    * of replicas reported from data-nodes.
    */
    */
-  private BlockInfo completeBlock(final MutableBlockCollection fileINode,
+  private BlockInfo completeBlock(final MutableBlockCollection bc,
       final int blkIndex, boolean force) throws IOException {
       final int blkIndex, boolean force) throws IOException {
     if(blkIndex < 0)
     if(blkIndex < 0)
       return null;
       return null;
-    BlockInfo curBlock = fileINode.getBlocks()[blkIndex];
+    BlockInfo curBlock = bc.getBlocks()[blkIndex];
     if(curBlock.isComplete())
     if(curBlock.isComplete())
       return curBlock;
       return curBlock;
     BlockInfoUnderConstruction ucBlock = (BlockInfoUnderConstruction)curBlock;
     BlockInfoUnderConstruction ucBlock = (BlockInfoUnderConstruction)curBlock;
@@ -498,7 +498,7 @@ public class BlockManager {
           "Cannot complete block: block has not been COMMITTED by the client");
           "Cannot complete block: block has not been COMMITTED by the client");
     BlockInfo completeBlock = ucBlock.convertToCompleteBlock();
     BlockInfo completeBlock = ucBlock.convertToCompleteBlock();
     // replace penultimate block in file
     // replace penultimate block in file
-    fileINode.setBlock(blkIndex, completeBlock);
+    bc.setBlock(blkIndex, completeBlock);
     
     
     // Since safe-mode only counts complete blocks, and we now have
     // Since safe-mode only counts complete blocks, and we now have
     // one more complete block, we need to adjust the total up, and
     // one more complete block, we need to adjust the total up, and
@@ -514,12 +514,12 @@ public class BlockManager {
     return blocksMap.replaceBlock(completeBlock);
     return blocksMap.replaceBlock(completeBlock);
   }
   }
 
 
-  private BlockInfo completeBlock(final MutableBlockCollection fileINode,
+  private BlockInfo completeBlock(final MutableBlockCollection bc,
       final BlockInfo block, boolean force) throws IOException {
       final BlockInfo block, boolean force) throws IOException {
-    BlockInfo[] fileBlocks = fileINode.getBlocks();
+    BlockInfo[] fileBlocks = bc.getBlocks();
     for(int idx = 0; idx < fileBlocks.length; idx++)
     for(int idx = 0; idx < fileBlocks.length; idx++)
       if(fileBlocks[idx] == block) {
       if(fileBlocks[idx] == block) {
-        return completeBlock(fileINode, idx, force);
+        return completeBlock(bc, idx, force);
       }
       }
     return block;
     return block;
   }
   }
@@ -529,10 +529,10 @@ public class BlockManager {
    * regardless of whether enough replicas are present. This is necessary
    * regardless of whether enough replicas are present. This is necessary
    * when tailing edit logs as a Standby.
    * when tailing edit logs as a Standby.
    */
    */
-  public BlockInfo forceCompleteBlock(final MutableBlockCollection fileINode,
+  public BlockInfo forceCompleteBlock(final MutableBlockCollection bc,
       final BlockInfoUnderConstruction block) throws IOException {
       final BlockInfoUnderConstruction block) throws IOException {
     block.commitBlock(block);
     block.commitBlock(block);
-    return completeBlock(fileINode, block, true);
+    return completeBlock(bc, block, true);
   }
   }
 
 
   
   
@@ -546,14 +546,14 @@ public class BlockManager {
    * The methods returns null if there is no partial block at the end.
    * The methods returns null if there is no partial block at the end.
    * The client is supposed to allocate a new block with the next call.
    * The client is supposed to allocate a new block with the next call.
    *
    *
-   * @param fileINode file
+   * @param bc file
    * @return the last block locations if the block is partial or null otherwise
    * @return the last block locations if the block is partial or null otherwise
    */
    */
   public LocatedBlock convertLastBlockToUnderConstruction(
   public LocatedBlock convertLastBlockToUnderConstruction(
-      MutableBlockCollection fileINode) throws IOException {
-    BlockInfo oldBlock = fileINode.getLastBlock();
+      MutableBlockCollection bc) throws IOException {
+    BlockInfo oldBlock = bc.getLastBlock();
     if(oldBlock == null ||
     if(oldBlock == null ||
-        fileINode.getPreferredBlockSize() == oldBlock.getNumBytes())
+        bc.getPreferredBlockSize() == oldBlock.getNumBytes())
       return null;
       return null;
     assert oldBlock == getStoredBlock(oldBlock) :
     assert oldBlock == getStoredBlock(oldBlock) :
       "last block of the file is not in blocksMap";
       "last block of the file is not in blocksMap";
@@ -561,7 +561,7 @@ public class BlockManager {
     DatanodeDescriptor[] targets = getNodes(oldBlock);
     DatanodeDescriptor[] targets = getNodes(oldBlock);
 
 
     BlockInfoUnderConstruction ucBlock =
     BlockInfoUnderConstruction ucBlock =
-      fileINode.setLastBlock(oldBlock, targets);
+      bc.setLastBlock(oldBlock, targets);
     blocksMap.replaceBlock(ucBlock);
     blocksMap.replaceBlock(ucBlock);
 
 
     // Remove block from replication queue.
     // Remove block from replication queue.
@@ -581,7 +581,7 @@ public class BlockManager {
         // always decrement total blocks
         // always decrement total blocks
         -1);
         -1);
 
 
-    final long fileLength = fileINode.computeContentSummary().getLength();
+    final long fileLength = bc.computeContentSummary().getLength();
     final long pos = fileLength - ucBlock.getNumBytes();
     final long pos = fileLength - ucBlock.getNumBytes();
     return createLocatedBlock(ucBlock, pos, AccessMode.WRITE);
     return createLocatedBlock(ucBlock, pos, AccessMode.WRITE);
   }
   }
@@ -921,8 +921,8 @@ public class BlockManager {
                             " does not exist. ");
                             " does not exist. ");
     }
     }
 
 
-    BlockCollection inode = storedBlock.getINode();
-    if (inode == null) {
+    BlockCollection bc = storedBlock.getBlockCollection();
+    if (bc == null) {
       NameNode.stateChangeLog.info("BLOCK markBlockAsCorrupt: " +
       NameNode.stateChangeLog.info("BLOCK markBlockAsCorrupt: " +
                                    "block " + storedBlock +
                                    "block " + storedBlock +
                                    " could not be marked as corrupt as it" +
                                    " could not be marked as corrupt as it" +
@@ -936,7 +936,7 @@ public class BlockManager {
 
 
     // Add this replica to corruptReplicas Map
     // Add this replica to corruptReplicas Map
     corruptReplicas.addToCorruptReplicasMap(storedBlock, node, reason);
     corruptReplicas.addToCorruptReplicasMap(storedBlock, node, reason);
-    if (countNodes(storedBlock).liveReplicas() >= inode.getReplication()) {
+    if (countNodes(storedBlock).liveReplicas() >= bc.getReplication()) {
       // the block is over-replicated so invalidate the replicas immediately
       // the block is over-replicated so invalidate the replicas immediately
       invalidateBlock(storedBlock, node);
       invalidateBlock(storedBlock, node);
     } else if (namesystem.isPopulatingReplQueues()) {
     } else if (namesystem.isPopulatingReplQueues()) {
@@ -1049,7 +1049,7 @@ public class BlockManager {
     int requiredReplication, numEffectiveReplicas;
     int requiredReplication, numEffectiveReplicas;
     List<DatanodeDescriptor> containingNodes, liveReplicaNodes;
     List<DatanodeDescriptor> containingNodes, liveReplicaNodes;
     DatanodeDescriptor srcNode;
     DatanodeDescriptor srcNode;
-    BlockCollection fileINode = null;
+    BlockCollection bc = null;
     int additionalReplRequired;
     int additionalReplRequired;
 
 
     int scheduledWork = 0;
     int scheduledWork = 0;
@@ -1061,15 +1061,15 @@ public class BlockManager {
         for (int priority = 0; priority < blocksToReplicate.size(); priority++) {
         for (int priority = 0; priority < blocksToReplicate.size(); priority++) {
           for (Block block : blocksToReplicate.get(priority)) {
           for (Block block : blocksToReplicate.get(priority)) {
             // block should belong to a file
             // block should belong to a file
-            fileINode = blocksMap.getINode(block);
+            bc = blocksMap.getBlockCollection(block);
             // abandoned block or block reopened for append
             // abandoned block or block reopened for append
-            if(fileINode == null || fileINode instanceof MutableBlockCollection) {
+            if(bc == null || bc instanceof MutableBlockCollection) {
               neededReplications.remove(block, priority); // remove from neededReplications
               neededReplications.remove(block, priority); // remove from neededReplications
               neededReplications.decrementReplicationIndex(priority);
               neededReplications.decrementReplicationIndex(priority);
               continue;
               continue;
             }
             }
 
 
-            requiredReplication = fileINode.getReplication();
+            requiredReplication = bc.getReplication();
 
 
             // get a source data-node
             // get a source data-node
             containingNodes = new ArrayList<DatanodeDescriptor>();
             containingNodes = new ArrayList<DatanodeDescriptor>();
@@ -1105,7 +1105,7 @@ public class BlockManager {
             } else {
             } else {
               additionalReplRequired = 1; // Needed on a new rack
               additionalReplRequired = 1; // Needed on a new rack
             }
             }
-            work.add(new ReplicationWork(block, fileINode, srcNode,
+            work.add(new ReplicationWork(block, bc, srcNode,
                 containingNodes, liveReplicaNodes, additionalReplRequired,
                 containingNodes, liveReplicaNodes, additionalReplRequired,
                 priority));
                 priority));
           }
           }
@@ -1127,8 +1127,8 @@ public class BlockManager {
 
 
       // choose replication targets: NOT HOLDING THE GLOBAL LOCK
       // choose replication targets: NOT HOLDING THE GLOBAL LOCK
       // It is costly to extract the filename for which chooseTargets is called,
       // It is costly to extract the filename for which chooseTargets is called,
-      // so for now we pass in the Inode itself.
-      rw.targets = blockplacement.chooseTarget(rw.fileINode,
+      // so for now we pass in the block collection itself.
+      rw.targets = blockplacement.chooseTarget(rw.bc,
           rw.additionalReplRequired, rw.srcNode, rw.liveReplicaNodes,
           rw.additionalReplRequired, rw.srcNode, rw.liveReplicaNodes,
           excludedNodes, rw.block.getNumBytes());
           excludedNodes, rw.block.getNumBytes());
     }
     }
@@ -1147,15 +1147,15 @@ public class BlockManager {
           int priority = rw.priority;
           int priority = rw.priority;
           // Recheck since global lock was released
           // Recheck since global lock was released
           // block should belong to a file
           // block should belong to a file
-          fileINode = blocksMap.getINode(block);
+          bc = blocksMap.getBlockCollection(block);
           // abandoned block or block reopened for append
           // abandoned block or block reopened for append
-          if(fileINode == null || fileINode instanceof MutableBlockCollection) {
+          if(bc == null || bc instanceof MutableBlockCollection) {
             neededReplications.remove(block, priority); // remove from neededReplications
             neededReplications.remove(block, priority); // remove from neededReplications
             rw.targets = null;
             rw.targets = null;
             neededReplications.decrementReplicationIndex(priority);
             neededReplications.decrementReplicationIndex(priority);
             continue;
             continue;
           }
           }
-          requiredReplication = fileINode.getReplication();
+          requiredReplication = bc.getReplication();
 
 
           // do not schedule more if enough replicas is already pending
           // do not schedule more if enough replicas is already pending
           NumberReplicas numReplicas = countNodes(block);
           NumberReplicas numReplicas = countNodes(block);
@@ -1802,8 +1802,7 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
       case COMPLETE:
       case COMPLETE:
       case COMMITTED:
       case COMMITTED:
         if (storedBlock.getGenerationStamp() != iblk.getGenerationStamp()) {
         if (storedBlock.getGenerationStamp() != iblk.getGenerationStamp()) {
-          return new BlockToMarkCorrupt(new BlockInfo(iblk, storedBlock
-              .getINode().getReplication()),
+          return new BlockToMarkCorrupt(storedBlock,
               "block is " + ucState + " and reported genstamp " +
               "block is " + ucState + " and reported genstamp " +
               iblk.getGenerationStamp() + " does not match " +
               iblk.getGenerationStamp() + " does not match " +
               "genstamp in block map " + storedBlock.getGenerationStamp());
               "genstamp in block map " + storedBlock.getGenerationStamp());
@@ -1823,8 +1822,7 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
       if (!storedBlock.isComplete()) {
       if (!storedBlock.isComplete()) {
         return null; // not corrupt
         return null; // not corrupt
       } else if (storedBlock.getGenerationStamp() != iblk.getGenerationStamp()) {
       } else if (storedBlock.getGenerationStamp() != iblk.getGenerationStamp()) {
-        return new BlockToMarkCorrupt(new BlockInfo(iblk, storedBlock
-            .getINode().getReplication()),
+        return new BlockToMarkCorrupt(storedBlock,
             "reported " + reportedState + " replica with genstamp " +
             "reported " + reportedState + " replica with genstamp " +
             iblk.getGenerationStamp() + " does not match COMPLETE block's " +
             iblk.getGenerationStamp() + " does not match COMPLETE block's " +
             "genstamp in block map " + storedBlock.getGenerationStamp());
             "genstamp in block map " + storedBlock.getGenerationStamp());
@@ -1916,7 +1914,7 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
     int numCurrentReplica = countLiveNodes(storedBlock);
     int numCurrentReplica = countLiveNodes(storedBlock);
     if (storedBlock.getBlockUCState() == BlockUCState.COMMITTED
     if (storedBlock.getBlockUCState() == BlockUCState.COMMITTED
         && numCurrentReplica >= minReplication) {
         && numCurrentReplica >= minReplication) {
-      completeBlock((MutableBlockCollection)storedBlock.getINode(), storedBlock, false);
+      completeBlock((MutableBlockCollection)storedBlock.getBlockCollection(), storedBlock, false);
     } else if (storedBlock.isComplete()) {
     } else if (storedBlock.isComplete()) {
       // check whether safe replication is reached for the block
       // check whether safe replication is reached for the block
       // only complete blocks are counted towards that.
       // only complete blocks are counted towards that.
@@ -1944,7 +1942,7 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
     } else {
     } else {
       storedBlock = block;
       storedBlock = block;
     }
     }
-    if (storedBlock == null || storedBlock.getINode() == null) {
+    if (storedBlock == null || storedBlock.getBlockCollection() == null) {
       // If this block does not belong to anyfile, then we are done.
       // If this block does not belong to anyfile, then we are done.
       NameNode.stateChangeLog.info("BLOCK* addStoredBlock: " + block + " on "
       NameNode.stateChangeLog.info("BLOCK* addStoredBlock: " + block + " on "
           + node + " size " + block.getNumBytes()
           + node + " size " + block.getNumBytes()
@@ -1954,8 +1952,8 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
       return block;
       return block;
     }
     }
     assert storedBlock != null : "Block must be stored by now";
     assert storedBlock != null : "Block must be stored by now";
-    BlockCollection fileINode = storedBlock.getINode();
-    assert fileINode != null : "Block must belong to a file";
+    BlockCollection bc = storedBlock.getBlockCollection();
+    assert bc != null : "Block must belong to a file";
 
 
     // add block to the datanode
     // add block to the datanode
     boolean added = node.addBlock(storedBlock);
     boolean added = node.addBlock(storedBlock);
@@ -1981,7 +1979,7 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
 
 
     if(storedBlock.getBlockUCState() == BlockUCState.COMMITTED &&
     if(storedBlock.getBlockUCState() == BlockUCState.COMMITTED &&
         numLiveReplicas >= minReplication) {
         numLiveReplicas >= minReplication) {
-      storedBlock = completeBlock((MutableBlockCollection)fileINode, storedBlock, false);
+      storedBlock = completeBlock((MutableBlockCollection)bc, storedBlock, false);
     } else if (storedBlock.isComplete()) {
     } else if (storedBlock.isComplete()) {
       // check whether safe replication is reached for the block
       // check whether safe replication is reached for the block
       // only complete blocks are counted towards that
       // only complete blocks are counted towards that
@@ -1992,7 +1990,7 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
     }
     }
     
     
     // if file is under construction, then done for now
     // if file is under construction, then done for now
-    if (fileINode instanceof MutableBlockCollection) {
+    if (bc instanceof MutableBlockCollection) {
       return storedBlock;
       return storedBlock;
     }
     }
 
 
@@ -2002,7 +2000,7 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
     }
     }
 
 
     // handle underReplication/overReplication
     // handle underReplication/overReplication
-    short fileReplication = fileINode.getReplication();
+    short fileReplication = bc.getReplication();
     if (!isNeededReplication(storedBlock, fileReplication, numCurrentReplica)) {
     if (!isNeededReplication(storedBlock, fileReplication, numCurrentReplica)) {
       neededReplications.remove(storedBlock, numCurrentReplica,
       neededReplications.remove(storedBlock, numCurrentReplica,
           num.decommissionedReplicas(), fileReplication);
           num.decommissionedReplicas(), fileReplication);
@@ -2129,8 +2127,8 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
    * what happened with it.
    * what happened with it.
    */
    */
   private MisReplicationResult processMisReplicatedBlock(BlockInfo block) {
   private MisReplicationResult processMisReplicatedBlock(BlockInfo block) {
-    BlockCollection fileINode = block.getINode();
-    if (fileINode == null) {
+    BlockCollection bc = block.getBlockCollection();
+    if (bc == null) {
       // block does not belong to any file
       // block does not belong to any file
       addToInvalidates(block);
       addToInvalidates(block);
       return MisReplicationResult.INVALID;
       return MisReplicationResult.INVALID;
@@ -2141,7 +2139,7 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
       return MisReplicationResult.UNDER_CONSTRUCTION;
       return MisReplicationResult.UNDER_CONSTRUCTION;
     }
     }
     // calculate current replication
     // calculate current replication
-    short expectedReplication = fileINode.getReplication();
+    short expectedReplication = bc.getReplication();
     NumberReplicas num = countNodes(block);
     NumberReplicas num = countNodes(block);
     int numCurrentReplica = num.liveReplicas();
     int numCurrentReplica = num.liveReplicas();
     // add to under-replicated queue if need to be
     // add to under-replicated queue if need to be
@@ -2258,7 +2256,7 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
                               BlockPlacementPolicy replicator) {
                               BlockPlacementPolicy replicator) {
     assert namesystem.hasWriteLock();
     assert namesystem.hasWriteLock();
     // first form a rack to datanodes map and
     // first form a rack to datanodes map and
-    BlockCollection inode = getINode(b);
+    BlockCollection bc = getBlockCollection(b);
     final Map<String, List<DatanodeDescriptor>> rackMap
     final Map<String, List<DatanodeDescriptor>> rackMap
         = new HashMap<String, List<DatanodeDescriptor>>();
         = new HashMap<String, List<DatanodeDescriptor>>();
     for(final Iterator<DatanodeDescriptor> iter = nonExcess.iterator();
     for(final Iterator<DatanodeDescriptor> iter = nonExcess.iterator();
@@ -2298,7 +2296,7 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
               || (addedNode != null && !priSet.contains(addedNode))) ) {
               || (addedNode != null && !priSet.contains(addedNode))) ) {
         cur = delNodeHint;
         cur = delNodeHint;
       } else { // regular excessive replica removal
       } else { // regular excessive replica removal
-        cur = replicator.chooseReplicaToDelete(inode, b, replication,
+        cur = replicator.chooseReplicaToDelete(bc, b, replication,
             priSet, remains);
             priSet, remains);
       }
       }
       firstOne = false;
       firstOne = false;
@@ -2379,8 +2377,8 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
       // necessary. In that case, put block on a possibly-will-
       // necessary. In that case, put block on a possibly-will-
       // be-replicated list.
       // be-replicated list.
       //
       //
-      BlockCollection fileINode = blocksMap.getINode(block);
-      if (fileINode != null) {
+      BlockCollection bc = blocksMap.getBlockCollection(block);
+      if (bc != null) {
         namesystem.decrementSafeBlockCount(block);
         namesystem.decrementSafeBlockCount(block);
         updateNeededReplications(block, -1, 0);
         updateNeededReplications(block, -1, 0);
       }
       }
@@ -2611,7 +2609,7 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
       NumberReplicas num) {
       NumberReplicas num) {
     int curReplicas = num.liveReplicas();
     int curReplicas = num.liveReplicas();
     int curExpectedReplicas = getReplication(block);
     int curExpectedReplicas = getReplication(block);
-    BlockCollection fileINode = blocksMap.getINode(block);
+    BlockCollection bc = blocksMap.getBlockCollection(block);
     Iterator<DatanodeDescriptor> nodeIter = blocksMap.nodeIterator(block);
     Iterator<DatanodeDescriptor> nodeIter = blocksMap.nodeIterator(block);
     StringBuilder nodeList = new StringBuilder();
     StringBuilder nodeList = new StringBuilder();
     while (nodeIter.hasNext()) {
     while (nodeIter.hasNext()) {
@@ -2624,7 +2622,7 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
         + ", corrupt replicas: " + num.corruptReplicas()
         + ", corrupt replicas: " + num.corruptReplicas()
         + ", decommissioned replicas: " + num.decommissionedReplicas()
         + ", decommissioned replicas: " + num.decommissionedReplicas()
         + ", excess replicas: " + num.excessReplicas()
         + ", excess replicas: " + num.excessReplicas()
-        + ", Is Open File: " + (fileINode instanceof MutableBlockCollection)
+        + ", Is Open File: " + (bc instanceof MutableBlockCollection)
         + ", Datanodes having this block: " + nodeList + ", Current Datanode: "
         + ", Datanodes having this block: " + nodeList + ", Current Datanode: "
         + srcNode + ", Is current datanode decommissioning: "
         + srcNode + ", Is current datanode decommissioning: "
         + srcNode.isDecommissionInProgress());
         + srcNode.isDecommissionInProgress());
@@ -2639,8 +2637,8 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
     final Iterator<? extends Block> it = srcNode.getBlockIterator();
     final Iterator<? extends Block> it = srcNode.getBlockIterator();
     while(it.hasNext()) {
     while(it.hasNext()) {
       final Block block = it.next();
       final Block block = it.next();
-      BlockCollection fileINode = blocksMap.getINode(block);
-      short expectedReplication = fileINode.getReplication();
+      BlockCollection bc = blocksMap.getBlockCollection(block);
+      short expectedReplication = bc.getReplication();
       NumberReplicas num = countNodes(block);
       NumberReplicas num = countNodes(block);
       int numCurrentReplica = num.liveReplicas();
       int numCurrentReplica = num.liveReplicas();
       if (numCurrentReplica > expectedReplication) {
       if (numCurrentReplica > expectedReplication) {
@@ -2662,9 +2660,9 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
     final Iterator<? extends Block> it = srcNode.getBlockIterator();
     final Iterator<? extends Block> it = srcNode.getBlockIterator();
     while(it.hasNext()) {
     while(it.hasNext()) {
       final Block block = it.next();
       final Block block = it.next();
-      BlockCollection fileINode = blocksMap.getINode(block);
+      BlockCollection bc = blocksMap.getBlockCollection(block);
 
 
-      if (fileINode != null) {
+      if (bc != null) {
         NumberReplicas num = countNodes(block);
         NumberReplicas num = countNodes(block);
         int curReplicas = num.liveReplicas();
         int curReplicas = num.liveReplicas();
         int curExpectedReplicas = getReplication(block);
         int curExpectedReplicas = getReplication(block);
@@ -2679,7 +2677,7 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
             if ((curReplicas == 0) && (num.decommissionedReplicas() > 0)) {
             if ((curReplicas == 0) && (num.decommissionedReplicas() > 0)) {
               decommissionOnlyReplicas++;
               decommissionOnlyReplicas++;
             }
             }
-            if (fileINode instanceof MutableBlockCollection) {
+            if (bc instanceof MutableBlockCollection) {
               underReplicatedInOpenFiles++;
               underReplicatedInOpenFiles++;
             }
             }
           }
           }
@@ -2782,11 +2780,11 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
 
 
   /* get replication factor of a block */
   /* get replication factor of a block */
   private int getReplication(Block block) {
   private int getReplication(Block block) {
-    BlockCollection fileINode = blocksMap.getINode(block);
-    if (fileINode == null) { // block does not belong to any file
+    BlockCollection bc = blocksMap.getBlockCollection(block);
+    if (bc == null) { // block does not belong to any file
       return 0;
       return 0;
     }
     }
-    return fileINode.getReplication();
+    return bc.getReplication();
   }
   }
 
 
 
 
@@ -2858,12 +2856,12 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
     return this.neededReplications.getCorruptBlockSize();
     return this.neededReplications.getCorruptBlockSize();
   }
   }
 
 
-  public BlockInfo addINode(BlockInfo block, BlockCollection iNode) {
-    return blocksMap.addINode(block, iNode);
+  public BlockInfo addBlockCollection(BlockInfo block, BlockCollection bc) {
+    return blocksMap.addBlockCollection(block, bc);
   }
   }
 
 
-  public BlockCollection getINode(Block b) {
-    return blocksMap.getINode(b);
+  public BlockCollection getBlockCollection(Block b) {
+    return blocksMap.getBlockCollection(b);
   }
   }
 
 
   /** @return an iterator of the datanodes. */
   /** @return an iterator of the datanodes. */
@@ -3002,7 +3000,7 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
   private static class ReplicationWork {
   private static class ReplicationWork {
 
 
     private Block block;
     private Block block;
-    private BlockCollection fileINode;
+    private BlockCollection bc;
 
 
     private DatanodeDescriptor srcNode;
     private DatanodeDescriptor srcNode;
     private List<DatanodeDescriptor> containingNodes;
     private List<DatanodeDescriptor> containingNodes;
@@ -3013,14 +3011,14 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
     private int priority;
     private int priority;
 
 
     public ReplicationWork(Block block,
     public ReplicationWork(Block block,
-        BlockCollection fileINode,
+        BlockCollection bc,
         DatanodeDescriptor srcNode,
         DatanodeDescriptor srcNode,
         List<DatanodeDescriptor> containingNodes,
         List<DatanodeDescriptor> containingNodes,
         List<DatanodeDescriptor> liveReplicaNodes,
         List<DatanodeDescriptor> liveReplicaNodes,
         int additionalReplRequired,
         int additionalReplRequired,
         int priority) {
         int priority) {
       this.block = block;
       this.block = block;
-      this.fileINode = fileINode;
+      this.bc = bc;
       this.srcNode = srcNode;
       this.srcNode = srcNode;
       this.containingNodes = containingNodes;
       this.containingNodes = containingNodes;
       this.liveReplicaNodes = liveReplicaNodes;
       this.liveReplicaNodes = liveReplicaNodes;

+ 6 - 6
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicy.java

@@ -110,11 +110,11 @@ public abstract class BlockPlacementPolicy {
    * choose <i>numOfReplicas</i> data nodes for <i>writer</i>
    * choose <i>numOfReplicas</i> data nodes for <i>writer</i>
    * If not, return as many as we can.
    * If not, return as many as we can.
    * The base implemenatation extracts the pathname of the file from the
    * The base implemenatation extracts the pathname of the file from the
-   * specified srcInode, but this could be a costly operation depending on the
+   * specified srcBC, but this could be a costly operation depending on the
    * file system implementation. Concrete implementations of this class should
    * file system implementation. Concrete implementations of this class should
    * override this method to avoid this overhead.
    * override this method to avoid this overhead.
    * 
    * 
-   * @param srcInode The inode of the file for which chooseTarget is being invoked.
+   * @param srcBC block collection of file for which chooseTarget is invoked.
    * @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.
@@ -122,13 +122,13 @@ public abstract class BlockPlacementPolicy {
    * @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.
    */
    */
-  DatanodeDescriptor[] chooseTarget(BlockCollection srcInode,
+  DatanodeDescriptor[] chooseTarget(BlockCollection srcBC,
                                     int numOfReplicas,
                                     int numOfReplicas,
                                     DatanodeDescriptor writer,
                                     DatanodeDescriptor writer,
                                     List<DatanodeDescriptor> chosenNodes,
                                     List<DatanodeDescriptor> chosenNodes,
                                     HashMap<Node, Node> excludedNodes,
                                     HashMap<Node, Node> excludedNodes,
                                     long blocksize) {
                                     long blocksize) {
-    return chooseTarget(srcInode.getName(), numOfReplicas, writer,
+    return chooseTarget(srcBC.getName(), numOfReplicas, writer,
                         chosenNodes, excludedNodes, blocksize);
                         chosenNodes, excludedNodes, blocksize);
   }
   }
 
 
@@ -149,7 +149,7 @@ public abstract class BlockPlacementPolicy {
    * Decide whether deleting the specified replica of the block still makes 
    * Decide whether deleting the specified replica of the block still makes 
    * the block conform to the configured block placement policy.
    * the block conform to the configured block placement policy.
    * 
    * 
-   * @param srcInode The inode of the file to which the block-to-be-deleted belongs
+   * @param srcBC block collection of file to which block-to-be-deleted belongs
    * @param block The block to be deleted
    * @param block The block to be deleted
    * @param replicationFactor The required number of replicas for this block
    * @param replicationFactor The required number of replicas for this block
    * @param existingReplicas The replica locations of this block that are present
    * @param existingReplicas The replica locations of this block that are present
@@ -158,7 +158,7 @@ public abstract class BlockPlacementPolicy {
                    listed in the previous parameter.
                    listed in the previous parameter.
    * @return the replica that is the best candidate for deletion
    * @return the replica that is the best candidate for deletion
    */
    */
-  abstract public DatanodeDescriptor chooseReplicaToDelete(BlockCollection srcInode,
+  abstract public DatanodeDescriptor chooseReplicaToDelete(BlockCollection srcBC,
                                       Block block, 
                                       Block block, 
                                       short replicationFactor,
                                       short replicationFactor,
                                       Collection<DatanodeDescriptor> existingReplicas,
                                       Collection<DatanodeDescriptor> existingReplicas,

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyDefault.java

@@ -546,7 +546,7 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
   }
   }
 
 
   @Override
   @Override
-  public DatanodeDescriptor chooseReplicaToDelete(BlockCollection inode,
+  public DatanodeDescriptor chooseReplicaToDelete(BlockCollection bc,
                                                  Block block,
                                                  Block block,
                                                  short replicationFactor,
                                                  short replicationFactor,
                                                  Collection<DatanodeDescriptor> first, 
                                                  Collection<DatanodeDescriptor> first, 

+ 8 - 8
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlocksMap.java

@@ -25,7 +25,7 @@ import org.apache.hadoop.hdfs.util.LightWeightGSet;
 
 
 /**
 /**
  * This class maintains the map from a block to its metadata.
  * This class maintains the map from a block to its metadata.
- * block's metadata currently includes INode it belongs to and
+ * block's metadata currently includes blockCollection it belongs to and
  * the datanodes that store the block.
  * the datanodes that store the block.
  */
  */
 class BlocksMap {
 class BlocksMap {
@@ -92,21 +92,21 @@ class BlocksMap {
     blocks = null;
     blocks = null;
   }
   }
 
 
-  BlockCollection getINode(Block b) {
+  BlockCollection getBlockCollection(Block b) {
     BlockInfo info = blocks.get(b);
     BlockInfo info = blocks.get(b);
-    return (info != null) ? info.getINode() : null;
+    return (info != null) ? info.getBlockCollection() : null;
   }
   }
 
 
   /**
   /**
-   * Add block b belonging to the specified file inode to the map.
+   * Add block b belonging to the specified block collection to the map.
    */
    */
-  BlockInfo addINode(BlockInfo b, BlockCollection iNode) {
+  BlockInfo addBlockCollection(BlockInfo b, BlockCollection bc) {
     BlockInfo info = blocks.get(b);
     BlockInfo info = blocks.get(b);
     if (info != b) {
     if (info != b) {
       info = b;
       info = b;
       blocks.put(info);
       blocks.put(info);
     }
     }
-    info.setINode(iNode);
+    info.setBlockCollection(bc);
     return info;
     return info;
   }
   }
 
 
@@ -120,7 +120,7 @@ class BlocksMap {
     if (blockInfo == null)
     if (blockInfo == null)
       return;
       return;
 
 
-    blockInfo.setINode(null);
+    blockInfo.setBlockCollection(null);
     for(int idx = blockInfo.numNodes()-1; idx >= 0; idx--) {
     for(int idx = blockInfo.numNodes()-1; idx >= 0; idx--) {
       DatanodeDescriptor dn = blockInfo.getDatanode(idx);
       DatanodeDescriptor dn = blockInfo.getDatanode(idx);
       dn.removeBlock(blockInfo); // remove from the list and wipe the location
       dn.removeBlock(blockInfo); // remove from the list and wipe the location
@@ -168,7 +168,7 @@ class BlocksMap {
     boolean removed = node.removeBlock(info);
     boolean removed = node.removeBlock(info);
 
 
     if (info.getDatanode(0) == null     // no datanodes left
     if (info.getDatanode(0) == null     // no datanodes left
-              && info.getINode() == null) {  // does not belong to a file
+              && info.getBlockCollection() == null) {  // does not belong to a file
       blocks.remove(b);  // remove block from the map
       blocks.remove(b);  // remove block from the map
     }
     }
     return removed;
     return removed;

+ 1 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java

@@ -860,7 +860,7 @@ public class DataNode extends Configured
    */
    */
   public String getDisplayName() {
   public String getDisplayName() {
     // NB: our DatanodeID may not be set yet
     // NB: our DatanodeID may not be set yet
-    return hostName + ":" + getIpcPort();
+    return hostName + ":" + getXferPort();
   }
   }
 
 
   /**
   /**
@@ -877,7 +877,6 @@ public class DataNode extends Configured
   /**
   /**
    * @return the datanode's IPC port
    * @return the datanode's IPC port
    */
    */
-  @VisibleForTesting
   public int getIpcPort() {
   public int getIpcPort() {
     return ipcServer.getListenerAddress().getPort();
     return ipcServer.getListenerAddress().getPort();
   }
   }

+ 3 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java

@@ -309,7 +309,7 @@ public class FSDirectory implements Closeable {
         INodeFile newF = (INodeFile)newNode;
         INodeFile newF = (INodeFile)newNode;
         BlockInfo[] blocks = newF.getBlocks();
         BlockInfo[] blocks = newF.getBlocks();
         for (int i = 0; i < blocks.length; i++) {
         for (int i = 0; i < blocks.length; i++) {
-          newF.setBlock(i, getBlockManager().addINode(blocks[i], newF));
+          newF.setBlock(i, getBlockManager().addBlockCollection(blocks[i], newF));
         }
         }
       }
       }
     } finally {
     } finally {
@@ -346,7 +346,7 @@ public class FSDirectory implements Closeable {
             fileINode.getReplication(),
             fileINode.getReplication(),
             BlockUCState.UNDER_CONSTRUCTION,
             BlockUCState.UNDER_CONSTRUCTION,
             targets);
             targets);
-      getBlockManager().addINode(blockInfo, fileINode);
+      getBlockManager().addBlockCollection(blockInfo, fileINode);
       fileINode.addBlock(blockInfo);
       fileINode.addBlock(blockInfo);
 
 
       if(NameNode.stateChangeLog.isDebugEnabled()) {
       if(NameNode.stateChangeLog.isDebugEnabled()) {
@@ -1127,7 +1127,7 @@ public class FSDirectory implements Closeable {
 
 
       int index = 0;
       int index = 0;
       for (BlockInfo b : newnode.getBlocks()) {
       for (BlockInfo b : newnode.getBlocks()) {
-        BlockInfo info = getBlockManager().addINode(b, newnode);
+        BlockInfo info = getBlockManager().addBlockCollection(b, newnode);
         newnode.setBlock(index, info); // inode refers to the block in BlocksMap
         newnode.setBlock(index, info); // inode refers to the block in BlocksMap
         index++;
         index++;
       }
       }

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java

@@ -601,7 +601,7 @@ public class FSEditLogLoader {
           // OP_ADD operations as each block is allocated.
           // OP_ADD operations as each block is allocated.
           newBI = new BlockInfo(newBlock, file.getReplication());
           newBI = new BlockInfo(newBlock, file.getReplication());
         }
         }
-        fsNamesys.getBlockManager().addINode(newBI, file);
+        fsNamesys.getBlockManager().addBlockCollection(newBI, file);
         file.addBlock(newBI);
         file.addBlock(newBI);
         fsNamesys.getBlockManager().processQueuedMessagesForBlock(newBlock);
         fsNamesys.getBlockManager().processQueuedMessagesForBlock(newBlock);
       }
       }

+ 29 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java

@@ -203,6 +203,10 @@ public abstract class FSEditLogOp {
     }
     }
 
 
     <T extends AddCloseOp> T setBlocks(Block[] blocks) {
     <T extends AddCloseOp> T setBlocks(Block[] blocks) {
+      if (blocks.length > MAX_BLOCKS) {
+        throw new RuntimeException("Can't have more than " + MAX_BLOCKS +
+            " in an AddCloseOp.");
+      }
       this.blocks = blocks;
       this.blocks = blocks;
       return (T)this;
       return (T)this;
     }
     }
@@ -296,10 +300,18 @@ public abstract class FSEditLogOp {
       }
       }
     }
     }
 
 
+    static final public int MAX_BLOCKS = 1024 * 1024 * 64;
+    
     private static Block[] readBlocks(
     private static Block[] readBlocks(
         DataInputStream in,
         DataInputStream in,
         int logVersion) throws IOException {
         int logVersion) throws IOException {
       int numBlocks = in.readInt();
       int numBlocks = in.readInt();
+      if (numBlocks < 0) {
+        throw new IOException("invalid negative number of blocks");
+      } else if (numBlocks > MAX_BLOCKS) {
+        throw new IOException("invalid number of blocks: " + numBlocks +
+            ".  The maximum number of blocks per file is " + MAX_BLOCKS);
+      }
       Block[] blocks = new Block[numBlocks];
       Block[] blocks = new Block[numBlocks];
       for (int i = 0; i < numBlocks; i++) {
       for (int i = 0; i < numBlocks; i++) {
         Block blk = new Block();
         Block blk = new Block();
@@ -579,6 +591,7 @@ public abstract class FSEditLogOp {
     String trg;
     String trg;
     String[] srcs;
     String[] srcs;
     long timestamp;
     long timestamp;
+    final static public int MAX_CONCAT_SRC = 1024 * 1024;
 
 
     private ConcatDeleteOp() {
     private ConcatDeleteOp() {
       super(OP_CONCAT_DELETE);
       super(OP_CONCAT_DELETE);
@@ -594,7 +607,12 @@ public abstract class FSEditLogOp {
     }
     }
 
 
     ConcatDeleteOp setSources(String[] srcs) {
     ConcatDeleteOp setSources(String[] srcs) {
+      if (srcs.length > MAX_CONCAT_SRC) {
+        throw new RuntimeException("ConcatDeleteOp can only have " +
+            MAX_CONCAT_SRC + " sources at most.");
+      }
       this.srcs = srcs;
       this.srcs = srcs;
+
       return this;
       return this;
     }
     }
 
 
@@ -624,8 +642,8 @@ public abstract class FSEditLogOp {
       if (!LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
       if (!LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
         this.length = in.readInt();
         this.length = in.readInt();
         if (length < 3) { // trg, srcs.., timestamp
         if (length < 3) { // trg, srcs.., timestamp
-          throw new IOException("Incorrect data format. "
-              + "Concat delete operation.");
+          throw new IOException("Incorrect data format " +
+              "for ConcatDeleteOp.");
         }
         }
       }
       }
       this.trg = FSImageSerialization.readString(in);
       this.trg = FSImageSerialization.readString(in);
@@ -635,6 +653,15 @@ public abstract class FSEditLogOp {
       } else {
       } else {
         srcSize = this.length - 1 - 1; // trg and timestamp
         srcSize = this.length - 1 - 1; // trg and timestamp
       }
       }
+      if (srcSize < 0) {
+          throw new IOException("Incorrect data format. "
+              + "ConcatDeleteOp cannot have a negative number of data " +
+              " sources.");
+      } else if (srcSize > MAX_CONCAT_SRC) {
+          throw new IOException("Incorrect data format. "
+              + "ConcatDeleteOp can have at most " + MAX_CONCAT_SRC +
+              " sources, but we tried to have " + (length - 3) + " sources.");
+      }
       this.srcs = new String [srcSize];
       this.srcs = new String [srcSize];
       for(int i=0; i<srcSize;i++) {
       for(int i=0; i<srcSize;i++) {
         srcs[i]= FSImageSerialization.readString(in);
         srcs[i]= FSImageSerialization.readString(in);

+ 17 - 20
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

@@ -1783,24 +1783,21 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
                 "Failed to close file " + src +
                 "Failed to close file " + src +
                 ". Lease recovery is in progress. Try again later.");
                 ". Lease recovery is in progress. Try again later.");
         } else {
         } else {
-          BlockInfoUnderConstruction lastBlock=pendingFile.getLastBlock();
-          if(lastBlock != null && lastBlock.getBlockUCState() ==
-            BlockUCState.UNDER_RECOVERY) {
-            throw new RecoveryInProgressException(
-              "Recovery in progress, file [" + src + "], " +
-              "lease owner [" + lease.getHolder() + "]");
-            } else {
-              throw new AlreadyBeingCreatedException(
-                "Failed to create file [" + src + "] for [" + holder +
-                "] on client [" + clientMachine +
-                "], because this file is already being created by [" +
-                pendingFile.getClientName() + "] on [" +
-                pendingFile.getClientMachine() + "]");
-            }
-         }
+          final BlockInfo lastBlock = pendingFile.getLastBlock();
+          if (lastBlock != null
+              && lastBlock.getBlockUCState() == BlockUCState.UNDER_RECOVERY) {
+            throw new RecoveryInProgressException("Recovery in progress, file ["
+                + src + "], " + "lease owner [" + lease.getHolder() + "]");
+          } else {
+            throw new AlreadyBeingCreatedException("Failed to create file ["
+                + src + "] for [" + holder + "] on client [" + clientMachine
+                + "], because this file is already being created by ["
+                + pendingFile.getClientName() + "] on ["
+                + pendingFile.getClientMachine() + "]");
+          }
+        }
       }
       }
     }
     }
-
   }
   }
 
 
   /**
   /**
@@ -2840,7 +2837,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       if (storedBlock == null) {
       if (storedBlock == null) {
         throw new IOException("Block (=" + lastblock + ") not found");
         throw new IOException("Block (=" + lastblock + ") not found");
       }
       }
-      INodeFile iFile = (INodeFile) storedBlock.getINode();
+      INodeFile iFile = (INodeFile) storedBlock.getBlockCollection();
       if (!iFile.isUnderConstruction() || storedBlock.isComplete()) {
       if (!iFile.isUnderConstruction() || storedBlock.isComplete()) {
         throw new IOException("Unexpected block (=" + lastblock
         throw new IOException("Unexpected block (=" + lastblock
                               + ") since the file (=" + iFile.getLocalName()
                               + ") since the file (=" + iFile.getLocalName()
@@ -4135,7 +4132,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
    * Returns whether the given block is one pointed-to by a file.
    * Returns whether the given block is one pointed-to by a file.
    */
    */
   private boolean isValidBlock(Block b) {
   private boolean isValidBlock(Block b) {
-    return (blockManager.getINode(b) != null);
+    return (blockManager.getBlockCollection(b) != null);
   }
   }
 
 
   // Distributed upgrade manager
   // Distributed upgrade manager
@@ -4394,7 +4391,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     }
     }
     
     
     // check file inode
     // check file inode
-    INodeFile file = (INodeFile) storedBlock.getINode();
+    INodeFile file = (INodeFile) storedBlock.getBlockCollection();
     if (file==null || !file.isUnderConstruction()) {
     if (file==null || !file.isUnderConstruction()) {
       throw new IOException("The file " + storedBlock + 
       throw new IOException("The file " + storedBlock + 
           " belonged to does not exist or it is not under construction.");
           " belonged to does not exist or it is not under construction.");
@@ -4706,7 +4703,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
 
 
       while (blkIterator.hasNext()) {
       while (blkIterator.hasNext()) {
         Block blk = blkIterator.next();
         Block blk = blkIterator.next();
-        INode inode = (INodeFile) blockManager.getINode(blk);
+        INode inode = (INodeFile) blockManager.getBlockCollection(blk);
         skip++;
         skip++;
         if (inode != null && blockManager.countNodes(blk).liveReplicas() == 0) {
         if (inode != null && blockManager.countNodes(blk).liveReplicas() == 0) {
           String src = FSDirectory.getFullPathName(inode);
           String src = FSDirectory.getFullPathName(inode);

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java

@@ -131,7 +131,7 @@ public class INodeFile extends INode implements BlockCollection {
     }
     }
     
     
     for(BlockInfo bi: newlist) {
     for(BlockInfo bi: newlist) {
-      bi.setINode(this);
+      bi.setBlockCollection(this);
     }
     }
     this.blocks = newlist;
     this.blocks = newlist;
   }
   }
@@ -164,7 +164,7 @@ public class INodeFile extends INode implements BlockCollection {
     if(blocks != null && v != null) {
     if(blocks != null && v != null) {
       for (BlockInfo blk : blocks) {
       for (BlockInfo blk : blocks) {
         v.add(blk);
         v.add(blk);
-        blk.setINode(null);
+        blk.setBlockCollection(null);
       }
       }
     }
     }
     blocks = null;
     blocks = null;

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFileUnderConstruction.java

@@ -156,7 +156,7 @@ public class INodeFileUnderConstruction extends INodeFile
     BlockInfoUnderConstruction ucBlock =
     BlockInfoUnderConstruction ucBlock =
       lastBlock.convertToBlockUnderConstruction(
       lastBlock.convertToBlockUnderConstruction(
           BlockUCState.UNDER_CONSTRUCTION, targets);
           BlockUCState.UNDER_CONSTRUCTION, targets);
-    ucBlock.setINode(this);
+    ucBlock.setBlockCollection(this);
     setBlock(numBlocks()-1, ucBlock);
     setBlock(numBlocks()-1, ucBlock);
     return ucBlock;
     return ucBlock;
   }
   }

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeJspHelper.java

@@ -734,7 +734,7 @@ class NamenodeJspHelper {
         this.inode = null;
         this.inode = null;
       } else {
       } else {
         this.block = new Block(blockId);
         this.block = new Block(blockId);
-        this.inode = (INodeFile) blockManager.getINode(block);
+        this.inode = (INodeFile) blockManager.getBlockCollection(block);
       }
       }
     }
     }
 
 

+ 14 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java

@@ -26,6 +26,8 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.List;
 import java.util.TreeSet;
 import java.util.TreeSet;
 
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 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.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
@@ -64,9 +66,11 @@ import org.apache.hadoop.util.ToolRunner;
 @InterfaceAudience.Private
 @InterfaceAudience.Private
 public class DFSAdmin extends FsShell {
 public class DFSAdmin extends FsShell {
 
 
-  static{
+  static {
     HdfsConfiguration.init();
     HdfsConfiguration.init();
   }
   }
+  
+  private static final Log LOG = LogFactory.getLog(DFSAdmin.class);
 
 
   /**
   /**
    * An abstract class for the execution of a file system command
    * An abstract class for the execution of a file system command
@@ -1089,6 +1093,7 @@ public class DFSAdmin extends FsShell {
       return exitCode;
       return exitCode;
     }
     }
 
 
+    Exception debugException = null;
     exitCode = 0;
     exitCode = 0;
     try {
     try {
       if ("-report".equals(cmd)) {
       if ("-report".equals(cmd)) {
@@ -1143,6 +1148,7 @@ public class DFSAdmin extends FsShell {
         printUsage("");
         printUsage("");
       }
       }
     } catch (IllegalArgumentException arge) {
     } catch (IllegalArgumentException arge) {
+      debugException = arge;
       exitCode = -1;
       exitCode = -1;
       System.err.println(cmd.substring(1) + ": " + arge.getLocalizedMessage());
       System.err.println(cmd.substring(1) + ": " + arge.getLocalizedMessage());
       printUsage(cmd);
       printUsage(cmd);
@@ -1151,6 +1157,7 @@ public class DFSAdmin extends FsShell {
       // This is a error returned by hadoop server. Print
       // This is a error returned by hadoop server. Print
       // out the first line of the error message, ignore the stack trace.
       // out the first line of the error message, ignore the stack trace.
       exitCode = -1;
       exitCode = -1;
+      debugException = e;
       try {
       try {
         String[] content;
         String[] content;
         content = e.getLocalizedMessage().split("\n");
         content = e.getLocalizedMessage().split("\n");
@@ -1159,12 +1166,17 @@ public class DFSAdmin extends FsShell {
       } catch (Exception ex) {
       } catch (Exception ex) {
         System.err.println(cmd.substring(1) + ": "
         System.err.println(cmd.substring(1) + ": "
                            + ex.getLocalizedMessage());
                            + ex.getLocalizedMessage());
+        debugException = ex;
       }
       }
     } catch (Exception e) {
     } catch (Exception e) {
       exitCode = -1;
       exitCode = -1;
+      debugException = e;
       System.err.println(cmd.substring(1) + ": "
       System.err.println(cmd.substring(1) + ": "
                          + e.getLocalizedMessage());
                          + e.getLocalizedMessage());
-    } 
+    }
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Exception encountered:", debugException);
+    }
     return exitCode;
     return exitCode;
   }
   }
 
 

+ 0 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java

@@ -40,7 +40,6 @@ import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
-import org.apache.hadoop.io.MD5Hash;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.TokenIdentifier;
 import org.apache.hadoop.security.token.TokenIdentifier;

+ 51 - 4
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java

@@ -59,6 +59,7 @@ import org.apache.hadoop.hdfs.MiniDFSCluster.NameNodeInfo;
 import org.apache.hadoop.hdfs.client.HdfsDataInputStream;
 import org.apache.hadoop.hdfs.client.HdfsDataInputStream;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo.AdminStates;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
@@ -706,13 +707,59 @@ public class DFSTestUtil {
         .join(nameservices));
         .join(nameservices));
   }
   }
   
   
+  private static DatanodeID getDatanodeID(String ipAddr) {
+    return new DatanodeID(ipAddr, "localhost",
+        DFSConfigKeys.DFS_DATANODE_DEFAULT_PORT);
+  }
+
+  public static DatanodeID getLocalDatanodeID() {
+    return new DatanodeID("127.0.0.1", "localhost",
+        DFSConfigKeys.DFS_DATANODE_DEFAULT_PORT);
+  }
+
+  public static DatanodeID getLocalDatanodeID(int port) {
+    return new DatanodeID("127.0.0.1", "localhost", "",
+        port, port, port);
+  }
+
   public static DatanodeDescriptor getLocalDatanodeDescriptor() {
   public static DatanodeDescriptor getLocalDatanodeDescriptor() {
-    return new DatanodeDescriptor(
-        new DatanodeID("127.0.0.1", DFSConfigKeys.DFS_DATANODE_DEFAULT_PORT));
+    return new DatanodeDescriptor(getLocalDatanodeID());
   }
   }
 
 
   public static DatanodeInfo getLocalDatanodeInfo() {
   public static DatanodeInfo getLocalDatanodeInfo() {
-    return new DatanodeInfo(
-        new DatanodeID("127.0.0.1", DFSConfigKeys.DFS_DATANODE_DEFAULT_PORT));
+    return new DatanodeInfo(getLocalDatanodeID());
+  }
+
+  public static DatanodeInfo getDatanodeInfo(String ipAddr) {
+    return new DatanodeInfo(getDatanodeID(ipAddr));
+  }
+  
+  public static DatanodeInfo getLocalDatanodeInfo(int port) {
+    return new DatanodeInfo(getLocalDatanodeID(port));
+  }
+
+  public static DatanodeInfo getDatanodeInfo(String ipAddr, 
+      String host, int port) {
+    return new DatanodeInfo(new DatanodeID(ipAddr, host, port));
+  }
+
+  public static DatanodeInfo getLocalDatanodeInfo(String ipAddr,
+      String hostname, AdminStates adminState) {
+    return new DatanodeInfo(ipAddr, hostname, "storage",
+        DFSConfigKeys.DFS_DATANODE_DEFAULT_PORT,
+        DFSConfigKeys.DFS_DATANODE_HTTP_DEFAULT_PORT,
+        DFSConfigKeys.DFS_DATANODE_IPC_DEFAULT_PORT,
+        1, 2, 3, 4, 5, 6, "local", adminState);
+  }
+
+  public static DatanodeDescriptor getDatanodeDescriptor(String ipAddr,
+      String rackLocation) {
+    return getDatanodeDescriptor(ipAddr, DFSConfigKeys.DFS_DATANODE_DEFAULT_PORT,
+        rackLocation);
+  }
+
+  public static DatanodeDescriptor getDatanodeDescriptor(String ipAddr,
+      int port, String rackLocation) {
+    return new DatanodeDescriptor(new DatanodeID(ipAddr, port), rackLocation);
   }
   }
 }
 }

+ 2 - 3
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientRetries.java

@@ -333,7 +333,7 @@ public class TestDFSClientRetries extends TestCase {
       LocatedBlock badLocatedBlock = new LocatedBlock(
       LocatedBlock badLocatedBlock = new LocatedBlock(
         goodLocatedBlock.getBlock(),
         goodLocatedBlock.getBlock(),
         new DatanodeInfo[] {
         new DatanodeInfo[] {
-          new DatanodeInfo(new DatanodeID("255.255.255.255", 234))
+          DFSTestUtil.getDatanodeInfo("1.2.3.4", "bogus", 1234)
         },
         },
         goodLocatedBlock.getStartOffset(),
         goodLocatedBlock.getStartOffset(),
         false);
         false);
@@ -627,8 +627,7 @@ public class TestDFSClientRetries extends TestCase {
     server.start();
     server.start();
 
 
     final InetSocketAddress addr = NetUtils.getConnectAddress(server);
     final InetSocketAddress addr = NetUtils.getConnectAddress(server);
-    DatanodeID fakeDnId = new DatanodeID(
-        "localhost", "localhost", "fake-storage", addr.getPort(), 0, addr.getPort());
+    DatanodeID fakeDnId = DFSTestUtil.getLocalDatanodeID(addr.getPort());
     
     
     ExtendedBlock b = new ExtendedBlock("fake-pool", new Block(12345L));
     ExtendedBlock b = new ExtendedBlock("fake-pool", new Block(12345L));
     LocatedBlock fakeBlock = new LocatedBlock(b, new DatanodeInfo[0]);
     LocatedBlock fakeBlock = new LocatedBlock(b, new DatanodeInfo[0]);

+ 42 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend.java

@@ -31,12 +31,15 @@ import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.HardLink;
 import org.apache.hadoop.fs.HardLink;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
 import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
 import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset;
 import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset;
+import org.apache.hadoop.ipc.RemoteException;
+import org.junit.Assert;
 import org.junit.Test;
 import org.junit.Test;
 
 
 /**
 /**
@@ -295,4 +298,43 @@ public class TestFileAppend{
       cluster.shutdown();
       cluster.shutdown();
     }
     }
   }
   }
+
+  /** Test two consecutive appends on a file with a full block. */
+  @Test
+  public void testAppendTwice() throws Exception {
+    Configuration conf = new HdfsConfiguration();
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build();
+    final FileSystem fs1 = cluster.getFileSystem();
+    final FileSystem fs2 = AppendTestUtil.createHdfsWithDifferentUsername(conf);
+    try {
+  
+      final Path p = new Path("/testAppendTwice/foo");
+      final int len = 1 << 16;
+      final byte[] fileContents = AppendTestUtil.initBuffer(len);
+
+      {
+        // create a new file with a full block.
+        FSDataOutputStream out = fs2.create(p, true, 4096, (short)1, len);
+        out.write(fileContents, 0, len);
+        out.close();
+      }
+  
+      //1st append does not add any data so that the last block remains full
+      //and the last block in INodeFileUnderConstruction is a BlockInfo
+      //but not BlockInfoUnderConstruction. 
+      fs2.append(p);
+      
+      //2nd append should get AlreadyBeingCreatedException
+      fs1.append(p);
+      Assert.fail();
+    } catch(RemoteException re) {
+      AppendTestUtil.LOG.info("Got an exception:", re);
+      Assert.assertEquals(AlreadyBeingCreatedException.class.getName(),
+          re.getClassName());
+    } finally {
+      fs2.close();
+      fs1.close();
+      cluster.shutdown();
+    }
+  }
 }
 }

+ 1 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestGetBlocks.java

@@ -121,8 +121,7 @@ public class TestGetBlocks extends TestCase {
       getBlocksWithException(namenode, dataNodes[0], -1);
       getBlocksWithException(namenode, dataNodes[0], -1);
 
 
       // get blocks of size BlockSize from a non-existent datanode
       // get blocks of size BlockSize from a non-existent datanode
-      DatanodeInfo info = DFSTestUtil.getLocalDatanodeInfo();
-      info.setIpAddr("1.2.3.4");
+      DatanodeInfo info = DFSTestUtil.getDatanodeInfo("1.2.3.4");
       getBlocksWithException(namenode, info, 2);
       getBlocksWithException(namenode, info, 2);
     } finally {
     } finally {
       cluster.shutdown();
       cluster.shutdown();

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReplaceDatanodeOnFailure.java

@@ -62,7 +62,7 @@ public class TestReplaceDatanodeOnFailure {
     final DatanodeInfo[][] datanodes = new DatanodeInfo[infos.length + 1][];
     final DatanodeInfo[][] datanodes = new DatanodeInfo[infos.length + 1][];
     datanodes[0] = new DatanodeInfo[0];
     datanodes[0] = new DatanodeInfo[0];
     for(int i = 0; i < infos.length; ) {
     for(int i = 0; i < infos.length; ) {
-      infos[i] = new DatanodeInfo(new DatanodeID("dn" + i, 100));
+      infos[i] = DFSTestUtil.getLocalDatanodeInfo(50020 + i);
       i++;
       i++;
       datanodes[i] = new DatanodeInfo[i];
       datanodes[i] = new DatanodeInfo[i];
       System.arraycopy(infos, 0, datanodes[i], 0, datanodes[i].length);
       System.arraycopy(infos, 0, datanodes[i], 0, datanodes[i].length);

+ 13 - 15
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocolPB/TestPBHelper.java

@@ -131,7 +131,7 @@ public class TestPBHelper {
 
 
   @Test
   @Test
   public void testConvertDatanodeID() {
   public void testConvertDatanodeID() {
-    DatanodeID dn = new DatanodeID("node", "node", "sid", 1, 2, 3);
+    DatanodeID dn = DFSTestUtil.getLocalDatanodeID();
     DatanodeIDProto dnProto = PBHelper.convert(dn);
     DatanodeIDProto dnProto = PBHelper.convert(dn);
     DatanodeID dn2 = PBHelper.convert(dnProto);
     DatanodeID dn2 = PBHelper.convert(dnProto);
     compare(dn, dn2);
     compare(dn, dn2);
@@ -280,10 +280,6 @@ public class TestPBHelper {
     return new ExtendedBlock("bpid", blkid, 100, 2);
     return new ExtendedBlock("bpid", blkid, 100, 2);
   }
   }
   
   
-  private DatanodeInfo getDNInfo() {
-    return new DatanodeInfo(new DatanodeID("node", "node", "sid", 0, 1, 2));
-  }
-  
   private void compare(DatanodeInfo dn1, DatanodeInfo dn2) {
   private void compare(DatanodeInfo dn1, DatanodeInfo dn2) {
       assertEquals(dn1.getAdminState(), dn2.getAdminState());
       assertEquals(dn1.getAdminState(), dn2.getAdminState());
       assertEquals(dn1.getBlockPoolUsed(), dn2.getBlockPoolUsed());
       assertEquals(dn1.getBlockPoolUsed(), dn2.getBlockPoolUsed());
@@ -316,7 +312,9 @@ public class TestPBHelper {
   
   
   @Test
   @Test
   public void testConvertRecoveringBlock() {
   public void testConvertRecoveringBlock() {
-    DatanodeInfo[] dnInfo = new DatanodeInfo[] { getDNInfo(), getDNInfo() };
+    DatanodeInfo di1 = DFSTestUtil.getLocalDatanodeInfo();
+    DatanodeInfo di2 = DFSTestUtil.getLocalDatanodeInfo();
+    DatanodeInfo[] dnInfo = new DatanodeInfo[] { di1, di2 };
     RecoveringBlock b = new RecoveringBlock(getExtendedBlock(), dnInfo, 3);
     RecoveringBlock b = new RecoveringBlock(getExtendedBlock(), dnInfo, 3);
     RecoveringBlockProto bProto = PBHelper.convert(b);
     RecoveringBlockProto bProto = PBHelper.convert(b);
     RecoveringBlock b1 = PBHelper.convert(bProto);
     RecoveringBlock b1 = PBHelper.convert(bProto);
@@ -330,7 +328,9 @@ public class TestPBHelper {
   
   
   @Test
   @Test
   public void testConvertBlockRecoveryCommand() {
   public void testConvertBlockRecoveryCommand() {
-    DatanodeInfo[] dnInfo = new DatanodeInfo[] { getDNInfo(), getDNInfo() };
+    DatanodeInfo di1 = DFSTestUtil.getLocalDatanodeInfo();
+    DatanodeInfo di2 = DFSTestUtil.getLocalDatanodeInfo();
+    DatanodeInfo[] dnInfo = new DatanodeInfo[] { di1, di2 };
 
 
     List<RecoveringBlock> blks = ImmutableList.of(
     List<RecoveringBlock> blks = ImmutableList.of(
       new RecoveringBlock(getExtendedBlock(1), dnInfo, 3),
       new RecoveringBlock(getExtendedBlock(1), dnInfo, 3),
@@ -400,13 +400,11 @@ public class TestPBHelper {
   
   
   @Test
   @Test
   public void testConvertLocatedBlock() {
   public void testConvertLocatedBlock() {
-    DatanodeInfo [] dnInfos = new DatanodeInfo[3];
-    dnInfos[0] = new DatanodeInfo("host0", "host0", "0", 5000, 5001, 5002, 20000, 10001, 9999,
-        59, 69, 32, "local", AdminStates.DECOMMISSION_INPROGRESS);
-    dnInfos[1] = new DatanodeInfo("host1", "host1", "1", 5000, 5001, 5002, 20000, 10001, 9999,
-        59, 69, 32, "local", AdminStates.DECOMMISSIONED);
-    dnInfos[2] = new DatanodeInfo("host2", "host2", "2", 5000, 5001, 5002, 20000, 10001, 9999,
-        59, 69, 32, "local", AdminStates.NORMAL);
+    DatanodeInfo [] dnInfos = {
+        DFSTestUtil.getLocalDatanodeInfo("1.1.1.1", "h1", AdminStates.DECOMMISSION_INPROGRESS),
+        DFSTestUtil.getLocalDatanodeInfo("2.2.2.2", "h2", AdminStates.DECOMMISSIONED),
+        DFSTestUtil.getLocalDatanodeInfo("3.3.3.3", "h3", AdminStates.NORMAL)
+    };
     LocatedBlock lb = new LocatedBlock(
     LocatedBlock lb = new LocatedBlock(
         new ExtendedBlock("bp12", 12345, 10, 53), dnInfos, 5, false);
         new ExtendedBlock("bp12", 12345, 10, 53), dnInfos, 5, false);
     LocatedBlockProto lbProto = PBHelper.convert(lb);
     LocatedBlockProto lbProto = PBHelper.convert(lb);
@@ -424,7 +422,7 @@ public class TestPBHelper {
   
   
   @Test
   @Test
   public void testConvertDatanodeRegistration() {
   public void testConvertDatanodeRegistration() {
-    DatanodeID dnId = new DatanodeID("host", "host", "xyz", 0, 1, 0);
+    DatanodeID dnId = DFSTestUtil.getLocalDatanodeID();
     BlockKey[] keys = new BlockKey[] { getBlockKey(2), getBlockKey(3) };
     BlockKey[] keys = new BlockKey[] { getBlockKey(2), getBlockKey(3) };
     ExportedBlockKeys expKeys = new ExportedBlockKeys(true, 9, 10,
     ExportedBlockKeys expKeys = new ExportedBlockKeys(true, 9, 10,
         getBlockKey(1), keys);
         getBlockKey(1), keys);

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/security/token/block/TestBlockToken.java

@@ -42,6 +42,7 @@ 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.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
@@ -279,8 +280,7 @@ public class TestBlockToken {
     server.start();
     server.start();
 
 
     final InetSocketAddress addr = NetUtils.getConnectAddress(server);
     final InetSocketAddress addr = NetUtils.getConnectAddress(server);
-    DatanodeID fakeDnId = new DatanodeID("localhost",
-        "localhost", "fake-storage", addr.getPort(), 0, addr.getPort());
+    DatanodeID fakeDnId = DFSTestUtil.getLocalDatanodeID(addr.getPort());
 
 
     ExtendedBlock b = new ExtendedBlock("fake-pool", new Block(12345L));
     ExtendedBlock b = new ExtendedBlock("fake-pool", new Block(12345L));
     LocatedBlock fakeBlock = new LocatedBlock(b, new DatanodeInfo[0]);
     LocatedBlock fakeBlock = new LocatedBlock(b, new DatanodeInfo[0]);

+ 36 - 30
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java

@@ -26,6 +26,7 @@ import java.util.Map.Entry;
 
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
@@ -47,17 +48,10 @@ import com.google.common.collect.LinkedListMultimap;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Lists;
 
 
 public class TestBlockManager {
 public class TestBlockManager {
-  private final List<DatanodeDescriptor> nodes = ImmutableList.of( 
-      new DatanodeDescriptor(new DatanodeID("h1", 5020), "/rackA"),
-      new DatanodeDescriptor(new DatanodeID("h2", 5020), "/rackA"),
-      new DatanodeDescriptor(new DatanodeID("h3", 5020), "/rackA"),
-      new DatanodeDescriptor(new DatanodeID("h4", 5020), "/rackB"),
-      new DatanodeDescriptor(new DatanodeID("h5", 5020), "/rackB"),
-      new DatanodeDescriptor(new DatanodeID("h6", 5020), "/rackB")
-    );
-  private final List<DatanodeDescriptor> rackA = nodes.subList(0, 3);
-  private final List<DatanodeDescriptor> rackB = nodes.subList(3, 6);
-  
+  private List<DatanodeDescriptor> nodes;
+  private List<DatanodeDescriptor> rackA;
+  private List<DatanodeDescriptor> rackB;
+
   /**
   /**
    * Some of these tests exercise code which has some randomness involved -
    * Some of these tests exercise code which has some randomness involved -
    * ie even if there's a bug, they may pass because the random node selection
    * ie even if there's a bug, they may pass because the random node selection
@@ -82,6 +76,16 @@ public class TestBlockManager {
     fsn = Mockito.mock(FSNamesystem.class);
     fsn = Mockito.mock(FSNamesystem.class);
     Mockito.doReturn(true).when(fsn).hasWriteLock();
     Mockito.doReturn(true).when(fsn).hasWriteLock();
     bm = new BlockManager(fsn, fsn, conf);
     bm = new BlockManager(fsn, fsn, conf);
+    nodes = ImmutableList.of(
+        DFSTestUtil.getDatanodeDescriptor("1.1.1.1", "/rackA"),
+        DFSTestUtil.getDatanodeDescriptor("2.2.2.2", "/rackA"),
+        DFSTestUtil.getDatanodeDescriptor("3.3.3.3", "/rackA"),
+        DFSTestUtil.getDatanodeDescriptor("4.4.4.4", "/rackB"),
+        DFSTestUtil.getDatanodeDescriptor("5.5.5.5", "/rackB"),
+        DFSTestUtil.getDatanodeDescriptor("6.6.6.6", "/rackB")
+      );
+    rackA = nodes.subList(0, 3);
+    rackB = nodes.subList(3, 6);
   }
   }
   
   
   private void addNodes(Iterable<DatanodeDescriptor> nodesToAdd) {
   private void addNodes(Iterable<DatanodeDescriptor> nodesToAdd) {
@@ -116,7 +120,7 @@ public class TestBlockManager {
   }
   }
   
   
   private void doBasicTest(int testIndex) {
   private void doBasicTest(int testIndex) {
-    List<DatanodeDescriptor> origNodes = nodes(0, 1);
+    List<DatanodeDescriptor> origNodes = getNodes(0, 1);
     BlockInfo blockInfo = addBlockOnNodes((long)testIndex, origNodes);
     BlockInfo blockInfo = addBlockOnNodes((long)testIndex, origNodes);
 
 
     DatanodeDescriptor[] pipeline = scheduleSingleReplication(blockInfo);
     DatanodeDescriptor[] pipeline = scheduleSingleReplication(blockInfo);
@@ -147,7 +151,7 @@ public class TestBlockManager {
   
   
   private void doTestTwoOfThreeNodesDecommissioned(int testIndex) throws Exception {
   private void doTestTwoOfThreeNodesDecommissioned(int testIndex) throws Exception {
     // Block originally on A1, A2, B1
     // Block originally on A1, A2, B1
-    List<DatanodeDescriptor> origNodes = nodes(0, 1, 3);
+    List<DatanodeDescriptor> origNodes = getNodes(0, 1, 3);
     BlockInfo blockInfo = addBlockOnNodes(testIndex, origNodes);
     BlockInfo blockInfo = addBlockOnNodes(testIndex, origNodes);
     
     
     // Decommission two of the nodes (A1, A2)
     // Decommission two of the nodes (A1, A2)
@@ -157,7 +161,7 @@ public class TestBlockManager {
     assertTrue("Source of replication should be one of the nodes the block " +
     assertTrue("Source of replication should be one of the nodes the block " +
         "was on. Was: " + pipeline[0],
         "was on. Was: " + pipeline[0],
         origNodes.contains(pipeline[0]));
         origNodes.contains(pipeline[0]));
-    assertEquals("Should have two targets", 3, pipeline.length);
+    assertEquals("Should have three targets", 3, pipeline.length);
     
     
     boolean foundOneOnRackA = false;
     boolean foundOneOnRackA = false;
     for (int i = 1; i < pipeline.length; i++) {
     for (int i = 1; i < pipeline.length; i++) {
@@ -190,7 +194,7 @@ public class TestBlockManager {
 
 
   private void doTestAllNodesHoldingReplicasDecommissioned(int testIndex) throws Exception {
   private void doTestAllNodesHoldingReplicasDecommissioned(int testIndex) throws Exception {
     // Block originally on A1, A2, B1
     // Block originally on A1, A2, B1
-    List<DatanodeDescriptor> origNodes = nodes(0, 1, 3);
+    List<DatanodeDescriptor> origNodes = getNodes(0, 1, 3);
     BlockInfo blockInfo = addBlockOnNodes(testIndex, origNodes);
     BlockInfo blockInfo = addBlockOnNodes(testIndex, origNodes);
     
     
     // Decommission all of the nodes
     // Decommission all of the nodes
@@ -242,7 +246,7 @@ public class TestBlockManager {
   
   
   private void doTestOneOfTwoRacksDecommissioned(int testIndex) throws Exception {
   private void doTestOneOfTwoRacksDecommissioned(int testIndex) throws Exception {
     // Block originally on A1, A2, B1
     // Block originally on A1, A2, B1
-    List<DatanodeDescriptor> origNodes = nodes(0, 1, 3);
+    List<DatanodeDescriptor> origNodes = getNodes(0, 1, 3);
     BlockInfo blockInfo = addBlockOnNodes(testIndex, origNodes);
     BlockInfo blockInfo = addBlockOnNodes(testIndex, origNodes);
     
     
     // Decommission all of the nodes in rack A
     // Decommission all of the nodes in rack A
@@ -252,7 +256,7 @@ public class TestBlockManager {
     assertTrue("Source of replication should be one of the nodes the block " +
     assertTrue("Source of replication should be one of the nodes the block " +
         "was on. Was: " + pipeline[0],
         "was on. Was: " + pipeline[0],
         origNodes.contains(pipeline[0]));
         origNodes.contains(pipeline[0]));
-    assertEquals("Should have 2 targets", 3, pipeline.length);
+    assertEquals("Should have three targets", 3, pipeline.length);
     
     
     boolean foundOneOnRackB = false;
     boolean foundOneOnRackB = false;
     for (int i = 1; i < pipeline.length; i++) {
     for (int i = 1; i < pipeline.length; i++) {
@@ -273,7 +277,8 @@ public class TestBlockManager {
 
 
     // the block is still under-replicated. Add a new node. This should allow
     // the block is still under-replicated. Add a new node. This should allow
     // the third off-rack replica.
     // the third off-rack replica.
-    DatanodeDescriptor rackCNode = new DatanodeDescriptor(new DatanodeID("h7", 100), "/rackC");
+    DatanodeDescriptor rackCNode =
+      DFSTestUtil.getDatanodeDescriptor("7.7.7.7", "/rackC");
     addNodes(ImmutableList.of(rackCNode));
     addNodes(ImmutableList.of(rackCNode));
     try {
     try {
       DatanodeDescriptor[] pipeline2 = scheduleSingleReplication(blockInfo);
       DatanodeDescriptor[] pipeline2 = scheduleSingleReplication(blockInfo);
@@ -313,13 +318,13 @@ public class TestBlockManager {
   
   
   @Test
   @Test
   public void testBlocksAreNotUnderreplicatedInSingleRack() throws Exception {
   public void testBlocksAreNotUnderreplicatedInSingleRack() throws Exception {
-    List<DatanodeDescriptor> nodes = ImmutableList.of( 
-        new DatanodeDescriptor(new DatanodeID("h1", 5020), "/rackA"),
-        new DatanodeDescriptor(new DatanodeID("h2", 5020), "/rackA"),
-        new DatanodeDescriptor(new DatanodeID("h3", 5020), "/rackA"),
-        new DatanodeDescriptor(new DatanodeID("h4", 5020), "/rackA"),
-        new DatanodeDescriptor(new DatanodeID("h5", 5020), "/rackA"),
-        new DatanodeDescriptor(new DatanodeID("h6", 5020), "/rackA")
+    List<DatanodeDescriptor> nodes = ImmutableList.of(
+        DFSTestUtil.getDatanodeDescriptor("1.1.1.1", "/rackA"),
+        DFSTestUtil.getDatanodeDescriptor("2.2.2.2", "/rackA"),
+        DFSTestUtil.getDatanodeDescriptor("3.3.3.3", "/rackA"),
+        DFSTestUtil.getDatanodeDescriptor("4.4.4.4", "/rackA"),
+        DFSTestUtil.getDatanodeDescriptor("5.5.5.5", "/rackA"),
+        DFSTestUtil.getDatanodeDescriptor("6.6.6.6", "/rackA")
       );
       );
     addNodes(nodes);
     addNodes(nodes);
     List<DatanodeDescriptor> origNodes = nodes.subList(0, 3);;
     List<DatanodeDescriptor> origNodes = nodes.subList(0, 3);;
@@ -359,7 +364,7 @@ public class TestBlockManager {
     return blockInfo;
     return blockInfo;
   }
   }
 
 
-  private List<DatanodeDescriptor> nodes(int ... indexes) {
+  private List<DatanodeDescriptor> getNodes(int ... indexes) {
     List<DatanodeDescriptor> ret = Lists.newArrayList();
     List<DatanodeDescriptor> ret = Lists.newArrayList();
     for (int idx : indexes) {
     for (int idx : indexes) {
       ret.add(nodes.get(idx));
       ret.add(nodes.get(idx));
@@ -368,7 +373,7 @@ public class TestBlockManager {
   }
   }
   
   
   private List<DatanodeDescriptor> startDecommission(int ... indexes) {
   private List<DatanodeDescriptor> startDecommission(int ... indexes) {
-    List<DatanodeDescriptor> nodes = nodes(indexes);
+    List<DatanodeDescriptor> nodes = getNodes(indexes);
     for (DatanodeDescriptor node : nodes) {
     for (DatanodeDescriptor node : nodes) {
       node.startDecommission();
       node.startDecommission();
     }
     }
@@ -380,7 +385,7 @@ public class TestBlockManager {
     Mockito.doReturn((short)3).when(iNode).getReplication();
     Mockito.doReturn((short)3).when(iNode).getReplication();
     BlockInfo blockInfo = blockOnNodes(blockId, nodes);
     BlockInfo blockInfo = blockOnNodes(blockId, nodes);
 
 
-    bm.blocksMap.addINode(blockInfo, iNode);
+    bm.blocksMap.addBlockCollection(blockInfo, iNode);
     return blockInfo;
     return blockInfo;
   }
   }
 
 
@@ -404,8 +409,9 @@ public class TestBlockManager {
 
 
     LinkedListMultimap<DatanodeDescriptor, BlockTargetPair> repls = getAllPendingReplications();
     LinkedListMultimap<DatanodeDescriptor, BlockTargetPair> repls = getAllPendingReplications();
     assertEquals(1, repls.size());
     assertEquals(1, repls.size());
-    Entry<DatanodeDescriptor, BlockTargetPair> repl = repls.entries()
-        .iterator().next();
+    Entry<DatanodeDescriptor, BlockTargetPair> repl =
+      repls.entries().iterator().next();
+        
     DatanodeDescriptor[] targets = repl.getValue().targets;
     DatanodeDescriptor[] targets = repl.getValue().targets;
 
 
     DatanodeDescriptor[] pipeline = new DatanodeDescriptor[1 + targets.length];
     DatanodeDescriptor[] pipeline = new DatanodeDescriptor[1 + targets.length];

+ 37 - 35
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestHost2NodesMap.java

@@ -18,73 +18,75 @@
 
 
 package org.apache.hadoop.hdfs.server.blockmanagement;
 package org.apache.hadoop.hdfs.server.blockmanagement;
 
 
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
+import org.apache.hadoop.hdfs.DFSTestUtil;
 
 
-import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.junit.Before;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.Test;
 
 
+import static org.junit.Assert.*;
+
 public class TestHost2NodesMap {
 public class TestHost2NodesMap {
   private Host2NodesMap map = new Host2NodesMap();
   private Host2NodesMap map = new Host2NodesMap();
-  private final DatanodeDescriptor dataNodes[] = new DatanodeDescriptor[] {
-    new DatanodeDescriptor(new DatanodeID("ip1", "h1", "", 5020, -1, -1), "/d1/r1"),
-    new DatanodeDescriptor(new DatanodeID("ip2", "h1", "", 5020, -1, -1), "/d1/r1"),
-    new DatanodeDescriptor(new DatanodeID("ip3", "h1", "", 5020, -1, -1), "/d1/r2"),
-    new DatanodeDescriptor(new DatanodeID("ip3", "h1", "", 5030, -1, -1), "/d1/r2"),
-  };
-  private final DatanodeDescriptor NULL_NODE = null; 
-  private final DatanodeDescriptor NODE = new DatanodeDescriptor(new DatanodeID("h3", 5040),
-      "/d1/r4");
-
+  private DatanodeDescriptor dataNodes[];
+  
   @Before
   @Before
   public void setup() {
   public void setup() {
-    for(DatanodeDescriptor node:dataNodes) {
+    dataNodes = new DatanodeDescriptor[] {
+        DFSTestUtil.getDatanodeDescriptor("1.1.1.1", "/d1/r1"),
+        DFSTestUtil.getDatanodeDescriptor("2.2.2.2", "/d1/r1"),
+        DFSTestUtil.getDatanodeDescriptor("3.3.3.3", "/d1/r2"),
+        DFSTestUtil.getDatanodeDescriptor("3.3.3.3", 5021, "/d1/r2"),
+    };
+    for (DatanodeDescriptor node : dataNodes) {
       map.add(node);
       map.add(node);
     }
     }
-    map.add(NULL_NODE);
+    map.add(null);
   }
   }
   
   
   @Test
   @Test
   public void testContains() throws Exception {
   public void testContains() throws Exception {
-    for(int i=0; i<dataNodes.length; i++) {
+    DatanodeDescriptor nodeNotInMap =
+      DFSTestUtil.getDatanodeDescriptor("3.3.3.3", "/d1/r4");
+    for (int i = 0; i < dataNodes.length; i++) {
       assertTrue(map.contains(dataNodes[i]));
       assertTrue(map.contains(dataNodes[i]));
     }
     }
-    assertFalse(map.contains(NULL_NODE));
-    assertFalse(map.contains(NODE));
+    assertFalse(map.contains(null));
+    assertFalse(map.contains(nodeNotInMap));
   }
   }
 
 
   @Test
   @Test
   public void testGetDatanodeByHost() throws Exception {
   public void testGetDatanodeByHost() throws Exception {
-    assertTrue(map.getDatanodeByHost("ip1")==dataNodes[0]);
-    assertTrue(map.getDatanodeByHost("ip2")==dataNodes[1]);
-    DatanodeDescriptor node = map.getDatanodeByHost("ip3");
-    assertTrue(node==dataNodes[2] || node==dataNodes[3]);
-    assertTrue(null==map.getDatanodeByHost("ip4"));
+    assertEquals(map.getDatanodeByHost("1.1.1.1"), dataNodes[0]);
+    assertEquals(map.getDatanodeByHost("2.2.2.2"), dataNodes[1]);
+    DatanodeDescriptor node = map.getDatanodeByHost("3.3.3.3");
+    assertTrue(node == dataNodes[2] || node == dataNodes[3]);
+    assertNull(map.getDatanodeByHost("4.4.4.4"));
   }
   }
 
 
   @Test
   @Test
   public void testRemove() throws Exception {
   public void testRemove() throws Exception {
-    assertFalse(map.remove(NODE));
+    DatanodeDescriptor nodeNotInMap =
+      DFSTestUtil.getDatanodeDescriptor("3.3.3.3", "/d1/r4");
+    assertFalse(map.remove(nodeNotInMap));
     
     
     assertTrue(map.remove(dataNodes[0]));
     assertTrue(map.remove(dataNodes[0]));
-    assertTrue(map.getDatanodeByHost("ip1")==null);
-    assertTrue(map.getDatanodeByHost("ip2")==dataNodes[1]);
-    DatanodeDescriptor node = map.getDatanodeByHost("ip3");
+    assertTrue(map.getDatanodeByHost("1.1.1.1.")==null);
+    assertTrue(map.getDatanodeByHost("2.2.2.2")==dataNodes[1]);
+    DatanodeDescriptor node = map.getDatanodeByHost("3.3.3.3");
     assertTrue(node==dataNodes[2] || node==dataNodes[3]);
     assertTrue(node==dataNodes[2] || node==dataNodes[3]);
-    assertTrue(null==map.getDatanodeByHost("ip4"));
+    assertNull(map.getDatanodeByHost("4.4.4.4"));
     
     
     assertTrue(map.remove(dataNodes[2]));
     assertTrue(map.remove(dataNodes[2]));
-    assertTrue(map.getDatanodeByHost("ip1")==null);
-    assertTrue(map.getDatanodeByHost("ip2")==dataNodes[1]);
-    assertTrue(map.getDatanodeByHost("ip3")==dataNodes[3]);
+    assertNull(map.getDatanodeByHost("1.1.1.1"));
+    assertEquals(map.getDatanodeByHost("2.2.2.2"), dataNodes[1]);
+    assertEquals(map.getDatanodeByHost("3.3.3.3"), dataNodes[3]);
     
     
     assertTrue(map.remove(dataNodes[3]));
     assertTrue(map.remove(dataNodes[3]));
-    assertTrue(map.getDatanodeByHost("ip1")==null);
-    assertTrue(map.getDatanodeByHost("ip2")==dataNodes[1]);
-    assertTrue(map.getDatanodeByHost("ip3")==null);
+    assertNull(map.getDatanodeByHost("1.1.1.1"));
+    assertEquals(map.getDatanodeByHost("2.2.2.2"), dataNodes[1]);
+    assertNull(map.getDatanodeByHost("3.3.3.3"));
     
     
-    assertFalse(map.remove(NULL_NODE));
+    assertFalse(map.remove(null));
     assertTrue(map.remove(dataNodes[1]));
     assertTrue(map.remove(dataNodes[1]));
     assertFalse(map.remove(dataNodes[1]));
     assertFalse(map.remove(dataNodes[1]));
   }
   }

+ 5 - 6
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestPendingDataNodeMessages.java

@@ -21,6 +21,7 @@ import static org.junit.Assert.*;
 
 
 import java.util.Queue;
 import java.util.Queue;
 
 
+import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.server.blockmanagement.PendingDataNodeMessages.ReportedBlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.PendingDataNodeMessages.ReportedBlockInfo;
@@ -38,12 +39,10 @@ public class TestPendingDataNodeMessages {
   private final Block block1Gs2DifferentInstance =
   private final Block block1Gs2DifferentInstance =
     new Block(1, 0, 2);
     new Block(1, 0, 2);
   private final Block block2Gs1 = new Block(2, 0, 1);
   private final Block block2Gs1 = new Block(2, 0, 1);
-  
-  private final DatanodeDescriptor fakeDN = new DatanodeDescriptor(
-      new DatanodeID("fake", 100));
-  
+
   @Test
   @Test
   public void testQueues() {
   public void testQueues() {
+    DatanodeDescriptor fakeDN = DFSTestUtil.getLocalDatanodeDescriptor();
     msgs.enqueueReportedBlock(fakeDN, block1Gs1, ReplicaState.FINALIZED);
     msgs.enqueueReportedBlock(fakeDN, block1Gs1, ReplicaState.FINALIZED);
     msgs.enqueueReportedBlock(fakeDN, block1Gs2, ReplicaState.FINALIZED);
     msgs.enqueueReportedBlock(fakeDN, block1Gs2, ReplicaState.FINALIZED);
 
 
@@ -56,8 +55,8 @@ public class TestPendingDataNodeMessages {
     Queue<ReportedBlockInfo> q =
     Queue<ReportedBlockInfo> q =
       msgs.takeBlockQueue(block1Gs2DifferentInstance);
       msgs.takeBlockQueue(block1Gs2DifferentInstance);
     assertEquals(
     assertEquals(
-        "ReportedBlockInfo [block=blk_1_1, dn=fake:100, reportedState=FINALIZED]," +
-        "ReportedBlockInfo [block=blk_1_2, dn=fake:100, reportedState=FINALIZED]",
+        "ReportedBlockInfo [block=blk_1_1, dn=127.0.0.1:50010, reportedState=FINALIZED]," +
+        "ReportedBlockInfo [block=blk_1_2, dn=127.0.0.1:50010, reportedState=FINALIZED]",
         Joiner.on(",").join(q));
         Joiner.on(",").join(q));
     assertEquals(0, msgs.count());
     assertEquals(0, msgs.count());
     
     

+ 0 - 104
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestRBWBlockInvalidation.java

@@ -1,104 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs.server.blockmanagement;
-
-import java.io.File;
-import java.io.IOException;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
-import org.apache.hadoop.hdfs.DFSTestUtil;
-import org.apache.hadoop.hdfs.HdfsConfiguration;
-import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.apache.hadoop.hdfs.protocol.Block;
-import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
-import org.apache.hadoop.hdfs.server.datanode.DataNode;
-import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
-import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
-
-import org.junit.Test;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-/**
- * Test when RBW block is removed. Invalidation of the corrupted block happens
- * and then the under replicated block gets replicated to the datanode.
- */
-public class TestRBWBlockInvalidation {
-  private static NumberReplicas countReplicas(final FSNamesystem namesystem,
-      ExtendedBlock block) {
-    return namesystem.getBlockManager().countNodes(block.getLocalBlock());
-  }
-
-  /**
-   * Test when a block's replica is removed from RBW folder in one of the
-   * datanode, namenode should ask to invalidate that corrupted block and
-   * schedule replication for one more replica for that under replicated block.
-   */
-  @Test
-  public void testBlockInvalidationWhenRBWReplicaMissedInDN()
-      throws IOException, InterruptedException {
-    Configuration conf = new HdfsConfiguration();
-    conf.setLong(DFSConfigKeys.DFS_BLOCKREPORT_INTERVAL_MSEC_KEY, 300);
-    conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_KEY, 1);
-    conf.setLong(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1);
-    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(3)
-        .build();
-    FSDataOutputStream out = null;
-    try {
-      final FSNamesystem namesystem = cluster.getNamesystem();
-      FileSystem fs = cluster.getFileSystem();
-      Path testPath = new Path(MiniDFSCluster.getBaseDirectory(), "foo1");
-      out = fs.create(testPath, (short) 3);
-      out.writeBytes("HDFS-3157: " + testPath);
-      out.hsync();
-      String bpid = namesystem.getBlockPoolId();
-      ExtendedBlock blk = DFSTestUtil.getFirstBlock(fs, testPath);
-      Block block = blk.getLocalBlock();
-      // Deleting partial block and its meta information from the RBW folder
-      // of first datanode.
-      DataNode dn = cluster.getDataNodes().get(0);
-      File blockFile = DataNodeTestUtils.getBlockFile(dn, bpid, block);
-      File metaFile = DataNodeTestUtils.getMetaFile(dn, bpid, block);
-      assertTrue("Could not delete the block file from the RBW folder",
-          blockFile.delete());
-      assertTrue("Could not delete the block meta file from the RBW folder",
-          metaFile.delete());
-      out.close();
-      assertEquals("The corrupt replica could not be invalidated", 0,
-          countReplicas(namesystem, blk).corruptReplicas());
-      /*
-       * Sleep for 3 seconds, for under replicated block to get replicated. As
-       * one second will be taken by ReplicationMonitor and one more second for
-       * invalidated block to get deleted from the datanode.
-       */
-      Thread.sleep(3000);
-      blk = DFSTestUtil.getFirstBlock(fs, testPath);
-      assertEquals("There should be three live replicas", 3,
-          countReplicas(namesystem, blk).liveReplicas());
-    } finally {
-      if (out != null) {
-        out.close();
-      }
-      cluster.shutdown();
-    }
-  }
-}

+ 42 - 37
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java

@@ -19,7 +19,7 @@ package org.apache.hadoop.hdfs.server.blockmanagement;
 
 
 import static org.junit.Assert.*;
 import static org.junit.Assert.*;
 
 
-import java.io.IOException;
+import java.io.File;
 import java.util.ArrayList;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Arrays;
 import java.util.HashMap;
 import java.util.HashMap;
@@ -39,54 +39,55 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 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.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.Test;
 
 
 public class TestReplicationPolicy {
 public class TestReplicationPolicy {
-  private Random random= DFSUtil.getRandom();
+  private Random random = DFSUtil.getRandom();
   private static final int BLOCK_SIZE = 1024;
   private static final int BLOCK_SIZE = 1024;
   private static final int NUM_OF_DATANODES = 6;
   private static final int NUM_OF_DATANODES = 6;
-  private static final Configuration CONF = new HdfsConfiguration();
-  private static final NetworkTopology cluster;
-  private static final NameNode namenode;
-  private static final BlockPlacementPolicy replicator;
+  private static NetworkTopology cluster;
+  private static NameNode namenode;
+  private static BlockPlacementPolicy replicator;
   private static final String filename = "/dummyfile.txt";
   private static final String filename = "/dummyfile.txt";
-  private static final DatanodeDescriptor dataNodes[] = 
-    new DatanodeDescriptor[] {
-      new DatanodeDescriptor(new DatanodeID("h1", 5020), "/d1/r1"),
-      new DatanodeDescriptor(new DatanodeID("h2", 5020), "/d1/r1"),
-      new DatanodeDescriptor(new DatanodeID("h3", 5020), "/d1/r2"),
-      new DatanodeDescriptor(new DatanodeID("h4", 5020), "/d1/r2"),
-      new DatanodeDescriptor(new DatanodeID("h5", 5020), "/d2/r3"),
-      new DatanodeDescriptor(new DatanodeID("h6", 5020), "/d2/r3")
-    };
-   
-  private final static DatanodeDescriptor NODE = 
-    new DatanodeDescriptor(new DatanodeID("h7", 5020), "/d2/r4");
-  
-  static {
-    try {
-      FileSystem.setDefaultUri(CONF, "hdfs://localhost:0");
-      CONF.set(DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY, "0.0.0.0:0");
-      DFSTestUtil.formatNameNode(CONF);
-      namenode = new NameNode(CONF);
-    } catch (IOException e) {
-      e.printStackTrace();
-      throw (RuntimeException)new RuntimeException().initCause(e);
-    }
+  private static DatanodeDescriptor dataNodes[];
+
+  @BeforeClass
+  public static void setupCluster() throws Exception {
+    Configuration conf = new HdfsConfiguration();
+    dataNodes = new DatanodeDescriptor[] {
+        DFSTestUtil.getDatanodeDescriptor("1.1.1.1", "/d1/r1"),
+        DFSTestUtil.getDatanodeDescriptor("2.2.2.2", "/d1/r1"),
+        DFSTestUtil.getDatanodeDescriptor("3.3.3.3", "/d1/r2"),
+        DFSTestUtil.getDatanodeDescriptor("4.4.4.4", "/d1/r2"),
+        DFSTestUtil.getDatanodeDescriptor("5.5.5.5", "/d2/r3"),
+        DFSTestUtil.getDatanodeDescriptor("6.6.6.6", "/d2/r3")        
+      };
+
+    FileSystem.setDefaultUri(conf, "hdfs://localhost:0");
+    conf.set(DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY, "0.0.0.0:0");
+    File baseDir = new File(System.getProperty(
+        "test.build.data", "build/test/data"), "dfs/");
+    conf.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY,
+        new File(baseDir, "name").getPath());
+
+    DFSTestUtil.formatNameNode(conf);
+    namenode = new NameNode(conf);
+
     final BlockManager bm = namenode.getNamesystem().getBlockManager();
     final BlockManager bm = namenode.getNamesystem().getBlockManager();
     replicator = bm.getBlockPlacementPolicy();
     replicator = bm.getBlockPlacementPolicy();
     cluster = bm.getDatanodeManager().getNetworkTopology();
     cluster = bm.getDatanodeManager().getNetworkTopology();
     // construct network topology
     // construct network topology
-    for(int i=0; i<NUM_OF_DATANODES; i++) {
+    for (int i=0; i < NUM_OF_DATANODES; i++) {
       cluster.add(dataNodes[i]);
       cluster.add(dataNodes[i]);
     }
     }
-    for(int i=0; i<NUM_OF_DATANODES; i++) {
+    for (int i=0; i < NUM_OF_DATANODES; i++) {
       dataNodes[i].updateHeartbeat(
       dataNodes[i].updateHeartbeat(
           2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L,
           2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L,
           2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, 0, 0);
           2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, 0, 0);
-    }
+    }    
   }
   }
-  
+
   /**
   /**
    * In this testcase, client is dataNodes[0]. So the 1st replica should be
    * In this testcase, client is dataNodes[0]. So the 1st replica should be
    * placed on dataNodes[0], the 2nd replica should be placed on 
    * placed on dataNodes[0], the 2nd replica should be placed on 
@@ -328,6 +329,7 @@ public class TestReplicationPolicy {
           HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, 0, 0);
           HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, 0, 0);
     }
     }
   }
   }
+
   /**
   /**
    * In this testcase, client is is a node outside of file system.
    * In this testcase, client is is a node outside of file system.
    * So the 1st replica can be placed on any node. 
    * So the 1st replica can be placed on any node. 
@@ -337,22 +339,25 @@ public class TestReplicationPolicy {
    */
    */
   @Test
   @Test
   public void testChooseTarget5() throws Exception {
   public void testChooseTarget5() throws Exception {
+    DatanodeDescriptor writerDesc =
+      DFSTestUtil.getDatanodeDescriptor("7.7.7.7", "/d2/r4");
+
     DatanodeDescriptor[] targets;
     DatanodeDescriptor[] targets;
     targets = replicator.chooseTarget(filename,
     targets = replicator.chooseTarget(filename,
-                                      0, NODE, BLOCK_SIZE);
+                                      0, writerDesc, BLOCK_SIZE);
     assertEquals(targets.length, 0);
     assertEquals(targets.length, 0);
     
     
     targets = replicator.chooseTarget(filename,
     targets = replicator.chooseTarget(filename,
-                                      1, NODE, BLOCK_SIZE);
+                                      1, writerDesc, BLOCK_SIZE);
     assertEquals(targets.length, 1);
     assertEquals(targets.length, 1);
     
     
     targets = replicator.chooseTarget(filename,
     targets = replicator.chooseTarget(filename,
-                                      2, NODE, BLOCK_SIZE);
+                                      2, writerDesc, BLOCK_SIZE);
     assertEquals(targets.length, 2);
     assertEquals(targets.length, 2);
     assertFalse(cluster.isOnSameRack(targets[0], targets[1]));
     assertFalse(cluster.isOnSameRack(targets[0], targets[1]));
     
     
     targets = replicator.chooseTarget(filename,
     targets = replicator.chooseTarget(filename,
-                                      3, NODE, BLOCK_SIZE);
+                                      3, writerDesc, BLOCK_SIZE);
     assertEquals(targets.length, 3);
     assertEquals(targets.length, 3);
     assertTrue(cluster.isOnSameRack(targets[1], targets[2]));
     assertTrue(cluster.isOnSameRack(targets[1], targets[2]));
     assertFalse(cluster.isOnSameRack(targets[0], targets[1]));    
     assertFalse(cluster.isOnSameRack(targets[0], targets[1]));    

+ 0 - 5
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/DataNodeTestUtils.java

@@ -136,11 +136,6 @@ public class DataNodeTestUtils {
       ) throws IOException {
       ) throws IOException {
     return FsDatasetTestUtil.getBlockFile(dn.getFSDataset(), bpid, b);
     return FsDatasetTestUtil.getBlockFile(dn.getFSDataset(), bpid, b);
   }
   }
-  
-  public static File getMetaFile(DataNode dn, String bpid, Block b)
-      throws IOException {
-    return FsDatasetTestUtil.getMetaFile(dn.getFSDataset(), bpid, b);
-  }
 
 
   public static boolean unlinkBlock(DataNode dn, ExtendedBlock bk, int numLinks
   public static boolean unlinkBlock(DataNode dn, ExtendedBlock bk, int numLinks
       ) throws IOException {
       ) throws IOException {

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBPOfferService.java

@@ -115,7 +115,7 @@ public class TestBPOfferService {
             0, HdfsConstants.LAYOUT_VERSION))
             0, HdfsConstants.LAYOUT_VERSION))
       .when(mock).versionRequest();
       .when(mock).versionRequest();
     
     
-    Mockito.doReturn(new DatanodeRegistration("fake-node", 100))
+    Mockito.doReturn(new DatanodeRegistration("1.2.3.4", 100))
       .when(mock).registerDatanode(Mockito.any(DatanodeRegistration.class));
       .when(mock).registerDatanode(Mockito.any(DatanodeRegistration.class));
     
     
     Mockito.doAnswer(new HeartbeatAnswer(nnIdx))
     Mockito.doAnswer(new HeartbeatAnswer(nnIdx))

+ 4 - 4
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockRecovery.java

@@ -47,6 +47,7 @@ import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
@@ -197,9 +198,9 @@ public class TestBlockRecovery {
         locs, RECOVERY_ID);
         locs, RECOVERY_ID);
     ArrayList<BlockRecord> syncList = new ArrayList<BlockRecord>(2);
     ArrayList<BlockRecord> syncList = new ArrayList<BlockRecord>(2);
     BlockRecord record1 = new BlockRecord(
     BlockRecord record1 = new BlockRecord(
-        new DatanodeID("xx", "yy", "zz", 1, 2, 3), dn1, replica1);
+        DFSTestUtil.getDatanodeInfo("1.2.3.4", "bogus", 1234), dn1, replica1);
     BlockRecord record2 = new BlockRecord(
     BlockRecord record2 = new BlockRecord(
-        new DatanodeID("aa", "bb", "cc", 1, 2, 3), dn2, replica2);
+        DFSTestUtil.getDatanodeInfo("1.2.3.4", "bogus", 1234), dn2, replica2);
     syncList.add(record1);
     syncList.add(record1);
     syncList.add(record2);
     syncList.add(record2);
     
     
@@ -401,8 +402,7 @@ public class TestBlockRecovery {
 
 
   private Collection<RecoveringBlock> initRecoveringBlocks() throws IOException {
   private Collection<RecoveringBlock> initRecoveringBlocks() throws IOException {
     Collection<RecoveringBlock> blocks = new ArrayList<RecoveringBlock>(1);
     Collection<RecoveringBlock> blocks = new ArrayList<RecoveringBlock>(1);
-    DatanodeInfo mockOtherDN = new DatanodeInfo(
-        new DatanodeID("127.0.0.1", "localhost", "storage-1234", 0, 0, 0));
+    DatanodeInfo mockOtherDN = DFSTestUtil.getLocalDatanodeInfo();
     DatanodeInfo[] locs = new DatanodeInfo[] {
     DatanodeInfo[] locs = new DatanodeInfo[] {
         new DatanodeInfo(dn.getDNRegistrationForBP(block.getBlockPoolId())),
         new DatanodeInfo(dn.getDNRegistrationForBP(block.getBlockPoolId())),
         mockOtherDN };
         mockOtherDN };

+ 0 - 6
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetTestUtil.java

@@ -36,12 +36,6 @@ public class FsDatasetTestUtil {
       ) throws IOException {
       ) throws IOException {
     return ((FsDatasetImpl)fsd).getBlockFile(bpid, b);
     return ((FsDatasetImpl)fsd).getBlockFile(bpid, b);
   }
   }
-  
-  public static File getMetaFile(FsDatasetSpi<?> fsd, String bpid, Block b)
-      throws IOException {
-    return FsDatasetUtil.getMetaFile(getBlockFile(fsd, bpid, b), b
-        .getGenerationStamp());
-  }
 
 
   public static boolean unlinkBlock(FsDatasetSpi<?> fsd,
   public static boolean unlinkBlock(FsDatasetSpi<?> fsd,
       ExtendedBlock block, int numLinks) throws IOException {
       ExtendedBlock block, int numLinks) throws IOException {

+ 1 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestInterDatanodeProtocol.java

@@ -356,8 +356,7 @@ public class TestInterDatanodeProtocol {
     server.start();
     server.start();
 
 
     final InetSocketAddress addr = NetUtils.getConnectAddress(server);
     final InetSocketAddress addr = NetUtils.getConnectAddress(server);
-    DatanodeID fakeDnId = new DatanodeID(
-        "localhost", "localhost", "fake-storage", addr.getPort(), 0, addr.getPort());
+    DatanodeID fakeDnId = DFSTestUtil.getLocalDatanodeID(addr.getPort());
     DatanodeInfo dInfo = new DatanodeInfo(fakeDnId);
     DatanodeInfo dInfo = new DatanodeInfo(fakeDnId);
     InterDatanodeProtocol proxy = null;
     InterDatanodeProtocol proxy = null;
 
 

+ 72 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java

@@ -29,6 +29,7 @@ import java.util.Arrays;
 import java.util.concurrent.Callable;
 import java.util.concurrent.Callable;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.Executors;
+import java.util.Random;
 import java.util.regex.Matcher;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 import java.util.regex.Pattern;
 
 
@@ -1155,4 +1156,75 @@ public class TestEditLog extends TestCase {
           "No non-corrupt logs for txid " + startGapTxId, ioe);
           "No non-corrupt logs for txid " + startGapTxId, ioe);
     }
     }
   }
   }
+
+  /**
+   * Test that we can read from a byte stream without crashing.
+   *
+   */
+  static void validateNoCrash(byte garbage[]) throws IOException {
+    final String TEST_LOG_NAME = "test_edit_log";
+
+    EditLogFileOutputStream elfos = null;
+    File file = null;
+    EditLogFileInputStream elfis = null;
+    try {
+      file = new File(TEST_LOG_NAME);
+      elfos = new EditLogFileOutputStream(file, 0);
+      elfos.create();
+      elfos.writeRaw(garbage, 0, garbage.length);
+      elfos.setReadyToFlush();
+      elfos.flushAndSync();
+      elfos.close();
+      elfos = null;
+      file = new File(TEST_LOG_NAME);
+      elfis = new EditLogFileInputStream(file);
+
+      // verify that we can read everything without killing the JVM or
+      // throwing an exception other than IOException
+      try {
+        while (true) {
+          FSEditLogOp op = elfis.readOp();
+          if (op == null)
+            break;
+        }
+      } catch (IOException e) {
+      } catch (Throwable t) {
+        StringWriter sw = new StringWriter();
+        t.printStackTrace(new PrintWriter(sw));
+        fail("caught non-IOException throwable with message " +
+            t.getMessage() + "\nstack trace\n" + sw.toString());
+      }
+    } finally {
+      if ((elfos != null) && (elfos.isOpen()))
+        elfos.close();
+      if (elfis != null)
+        elfis.close();
+    }
+  }
+
+  static byte[][] invalidSequenecs = null;
+
+  /**
+   * "Fuzz" test for the edit log.
+   *
+   * This tests that we can read random garbage from the edit log without
+   * crashing the JVM or throwing an unchecked exception.
+   */
+  @Test
+  public void testFuzzSequences() throws IOException {
+    final int MAX_GARBAGE_LENGTH = 512;
+    final int MAX_INVALID_SEQ = 5000;
+    // The seed to use for our random number generator.  When given the same
+    // seed, Java.util.Random will always produce the same sequence of values.
+    // This is important because it means that the test is deterministic and
+    // repeatable on any machine.
+    final int RANDOM_SEED = 123;
+
+    Random r = new Random(RANDOM_SEED);
+    for (int i = 0; i < MAX_INVALID_SEQ; i++) {
+      byte[] garbage = new byte[r.nextInt(MAX_GARBAGE_LENGTH)];
+      r.nextBytes(garbage);
+      validateNoCrash(garbage);
+    }
+  }
 }
 }

+ 36 - 22
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/net/TestNetworkTopology.java

@@ -18,52 +18,60 @@
 
 
 package org.apache.hadoop.net;
 package org.apache.hadoop.net;
 
 
-
 import java.util.HashMap;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.Map;
 
 
-import junit.framework.TestCase;
-
+import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 
 
-public class TestNetworkTopology extends TestCase {
+import org.junit.Test;
+import org.junit.Before;
+
+import static org.junit.Assert.*;
+
+public class TestNetworkTopology {
   private final static NetworkTopology cluster = new NetworkTopology();
   private final static NetworkTopology cluster = new NetworkTopology();
-  private final static DatanodeDescriptor dataNodes[] = new DatanodeDescriptor[] {
-    new DatanodeDescriptor(new DatanodeID("h1", 5020), "/d1/r1"),
-    new DatanodeDescriptor(new DatanodeID("h2", 5020), "/d1/r1"),
-    new DatanodeDescriptor(new DatanodeID("h3", 5020), "/d1/r2"),
-    new DatanodeDescriptor(new DatanodeID("h4", 5020), "/d1/r2"),
-    new DatanodeDescriptor(new DatanodeID("h5", 5020), "/d1/r2"),
-    new DatanodeDescriptor(new DatanodeID("h6", 5020), "/d2/r3"),
-    new DatanodeDescriptor(new DatanodeID("h7", 5020), "/d2/r3")
-  };
-  private final static DatanodeDescriptor NODE = 
-    new DatanodeDescriptor(new DatanodeID("h8", 5020), "/d2/r4");
+  private DatanodeDescriptor dataNodes[];
   
   
-  static {
-    for(int i=0; i<dataNodes.length; i++) {
+  @Before
+  public void setupDatanodes() {
+    dataNodes = new DatanodeDescriptor[] {
+        DFSTestUtil.getDatanodeDescriptor("1.1.1.1", "/d1/r1"),
+        DFSTestUtil.getDatanodeDescriptor("2.2.2.2", "/d1/r1"),
+        DFSTestUtil.getDatanodeDescriptor("3.3.3.3", "/d1/r2"),
+        DFSTestUtil.getDatanodeDescriptor("4.4.4.4", "/d1/r2"),
+        DFSTestUtil.getDatanodeDescriptor("5.5.5.5", "/d1/r2"),
+        DFSTestUtil.getDatanodeDescriptor("6.6.6.6", "/d2/r3"),
+        DFSTestUtil.getDatanodeDescriptor("7.7.7.7", "/d2/r3")
+    };
+    for (int i = 0; i < dataNodes.length; i++) {
       cluster.add(dataNodes[i]);
       cluster.add(dataNodes[i]);
     }
     }
   }
   }
   
   
+  @Test
   public void testContains() throws Exception {
   public void testContains() throws Exception {
-    for(int i=0; i<dataNodes.length; i++) {
+    DatanodeDescriptor nodeNotInMap = 
+      DFSTestUtil.getDatanodeDescriptor("8.8.8.8", "/d2/r4");
+    for (int i=0; i < dataNodes.length; i++) {
       assertTrue(cluster.contains(dataNodes[i]));
       assertTrue(cluster.contains(dataNodes[i]));
     }
     }
-    assertFalse(cluster.contains(NODE));
+    assertFalse(cluster.contains(nodeNotInMap));
   }
   }
   
   
+  @Test
   public void testNumOfChildren() throws Exception {
   public void testNumOfChildren() throws Exception {
     assertEquals(cluster.getNumOfLeaves(), dataNodes.length);
     assertEquals(cluster.getNumOfLeaves(), dataNodes.length);
   }
   }
 
 
+  @Test
   public void testCreateInvalidTopology() throws Exception {
   public void testCreateInvalidTopology() throws Exception {
     NetworkTopology invalCluster = new NetworkTopology();
     NetworkTopology invalCluster = new NetworkTopology();
     DatanodeDescriptor invalDataNodes[] = new DatanodeDescriptor[] {
     DatanodeDescriptor invalDataNodes[] = new DatanodeDescriptor[] {
-      new DatanodeDescriptor(new DatanodeID("h1", 5020), "/d1/r1"),
-      new DatanodeDescriptor(new DatanodeID("h2", 5020), "/d1/r1"),
-      new DatanodeDescriptor(new DatanodeID("h3", 5020), "/d1")
+        DFSTestUtil.getDatanodeDescriptor("1.1.1.1", "/d1/r1"),
+        DFSTestUtil.getDatanodeDescriptor("2.2.2.2", "/d1/r1"),
+        DFSTestUtil.getDatanodeDescriptor("3.3.3.3", "/d1")
     };
     };
     invalCluster.add(invalDataNodes[0]);
     invalCluster.add(invalDataNodes[0]);
     invalCluster.add(invalDataNodes[1]);
     invalCluster.add(invalDataNodes[1]);
@@ -77,6 +85,7 @@ public class TestNetworkTopology extends TestCase {
     }
     }
   }
   }
 
 
+  @Test
   public void testRacks() throws Exception {
   public void testRacks() throws Exception {
     assertEquals(cluster.getNumOfRacks(), 3);
     assertEquals(cluster.getNumOfRacks(), 3);
     assertTrue(cluster.isOnSameRack(dataNodes[0], dataNodes[1]));
     assertTrue(cluster.isOnSameRack(dataNodes[0], dataNodes[1]));
@@ -87,6 +96,7 @@ public class TestNetworkTopology extends TestCase {
     assertTrue(cluster.isOnSameRack(dataNodes[5], dataNodes[6]));
     assertTrue(cluster.isOnSameRack(dataNodes[5], dataNodes[6]));
   }
   }
   
   
+  @Test
   public void testGetDistance() throws Exception {
   public void testGetDistance() throws Exception {
     assertEquals(cluster.getDistance(dataNodes[0], dataNodes[0]), 0);
     assertEquals(cluster.getDistance(dataNodes[0], dataNodes[0]), 0);
     assertEquals(cluster.getDistance(dataNodes[0], dataNodes[1]), 2);
     assertEquals(cluster.getDistance(dataNodes[0], dataNodes[1]), 2);
@@ -94,6 +104,7 @@ public class TestNetworkTopology extends TestCase {
     assertEquals(cluster.getDistance(dataNodes[0], dataNodes[6]), 6);
     assertEquals(cluster.getDistance(dataNodes[0], dataNodes[6]), 6);
   }
   }
 
 
+  @Test
   public void testPseudoSortByDistance() throws Exception {
   public void testPseudoSortByDistance() throws Exception {
     DatanodeDescriptor[] testNodes = new DatanodeDescriptor[3];
     DatanodeDescriptor[] testNodes = new DatanodeDescriptor[3];
     
     
@@ -136,6 +147,7 @@ public class TestNetworkTopology extends TestCase {
     assertTrue(testNodes[2] == dataNodes[3]);
     assertTrue(testNodes[2] == dataNodes[3]);
   }
   }
   
   
+  @Test
   public void testRemove() throws Exception {
   public void testRemove() throws Exception {
     for(int i=0; i<dataNodes.length; i++) {
     for(int i=0; i<dataNodes.length; i++) {
       cluster.remove(dataNodes[i]);
       cluster.remove(dataNodes[i]);
@@ -173,6 +185,7 @@ public class TestNetworkTopology extends TestCase {
   /**
   /**
    * This test checks that chooseRandom works for an excluded node.
    * This test checks that chooseRandom works for an excluded node.
    */
    */
+  @Test
   public void testChooseRandomExcludedNode() {
   public void testChooseRandomExcludedNode() {
     String scope = "~" + NodeBase.getPath(dataNodes[0]);
     String scope = "~" + NodeBase.getPath(dataNodes[0]);
     Map<Node, Integer> frequency = pickNodesAtRandom(100, scope);
     Map<Node, Integer> frequency = pickNodesAtRandom(100, scope);
@@ -186,6 +199,7 @@ public class TestNetworkTopology extends TestCase {
   /**
   /**
    * This test checks that chooseRandom works for an excluded rack.
    * This test checks that chooseRandom works for an excluded rack.
    */
    */
+  @Test
   public void testChooseRandomExcludedRack() {
   public void testChooseRandomExcludedRack() {
     Map<Node, Integer> frequency = pickNodesAtRandom(100, "~" + "/d2");
     Map<Node, Integer> frequency = pickNodesAtRandom(100, "~" + "/d2");
     // all the nodes on the second rack should be zero
     // all the nodes on the second rack should be zero

+ 3 - 0
hadoop-mapreduce-project/CHANGES.txt

@@ -475,6 +475,9 @@ Release 0.23.3 - UNRELEASED
     MAPREDUCE-4215. RM app page shows 500 error on appid parse error 
     MAPREDUCE-4215. RM app page shows 500 error on appid parse error 
     (Jonathon Eagles via tgraves)
     (Jonathon Eagles via tgraves)
 
 
+    MAPREDUCE-4237. TestNodeStatusUpdater can fail if localhost has a domain
+    associated with it (bobby)
+
 Release 0.23.2 - UNRELEASED
 Release 0.23.2 - UNRELEASED
 
 
   INCOMPATIBLE CHANGES
   INCOMPATIBLE CHANGES

+ 7 - 2
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeStatusUpdater.java

@@ -21,6 +21,7 @@ package org.apache.hadoop.yarn.server.nodemanager;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.mock;
 
 
 import java.io.IOException;
 import java.io.IOException;
+import java.net.InetSocketAddress;
 import java.util.ArrayList;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashMap;
@@ -35,6 +36,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileContext;
 import org.apache.hadoop.fs.FileContext;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
 import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
+import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.yarn.YarnException;
 import org.apache.hadoop.yarn.YarnException;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
@@ -88,7 +90,7 @@ public class TestNodeStatusUpdater {
   int heartBeatID = 0;
   int heartBeatID = 0;
   volatile Throwable nmStartError = null;
   volatile Throwable nmStartError = null;
   private final List<NodeId> registeredNodes = new ArrayList<NodeId>();
   private final List<NodeId> registeredNodes = new ArrayList<NodeId>();
-  private final Configuration conf = new YarnConfiguration();
+  private final Configuration conf = createNMConfig();
   private NodeManager nm;
   private NodeManager nm;
   protected NodeManager rebootedNodeManager;
   protected NodeManager rebootedNodeManager;
 
 
@@ -117,7 +119,9 @@ public class TestNodeStatusUpdater {
       Resource resource = request.getResource();
       Resource resource = request.getResource();
       LOG.info("Registering " + nodeId.toString());
       LOG.info("Registering " + nodeId.toString());
       // NOTE: this really should be checking against the config value
       // NOTE: this really should be checking against the config value
-      Assert.assertEquals("localhost:12345", nodeId.toString());
+      InetSocketAddress expected = NetUtils.getConnectAddress(
+          conf.getSocketAddr(YarnConfiguration.NM_ADDRESS, null, -1));
+      Assert.assertEquals(NetUtils.getHostPortString(expected), nodeId.toString());
       Assert.assertEquals(5 * 1024, resource.getMemory());
       Assert.assertEquals(5 * 1024, resource.getMemory());
       registeredNodes.add(nodeId);
       registeredNodes.add(nodeId);
       RegistrationResponse regResponse = recordFactory
       RegistrationResponse regResponse = recordFactory
@@ -429,6 +433,7 @@ public class TestNodeStatusUpdater {
     while (nm.getServiceState() == STATE.INITED && waitCount++ != 20) {
     while (nm.getServiceState() == STATE.INITED && waitCount++ != 20) {
       LOG.info("Waiting for NM to start..");
       LOG.info("Waiting for NM to start..");
       if (nmStartError != null) {
       if (nmStartError != null) {
+        LOG.error("Error during startup. ", nmStartError);
         Assert.fail(nmStartError.getCause().getMessage());
         Assert.fail(nmStartError.getCause().getMessage());
       }
       }
       Thread.sleep(1000);
       Thread.sleep(1000);

+ 14 - 14
hadoop-mapreduce-project/src/contrib/raid/src/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyRaid.java

@@ -144,25 +144,25 @@ public class BlockPlacementPolicyRaid extends BlockPlacementPolicy {
 
 
   /** {@inheritDoc} */
   /** {@inheritDoc} */
   @Override
   @Override
-  public DatanodeDescriptor chooseReplicaToDelete(BlockCollection inode,
+  public DatanodeDescriptor chooseReplicaToDelete(BlockCollection bc,
       Block block, short replicationFactor,
       Block block, short replicationFactor,
       Collection<DatanodeDescriptor> first,
       Collection<DatanodeDescriptor> first,
       Collection<DatanodeDescriptor> second) {
       Collection<DatanodeDescriptor> second) {
 
 
     DatanodeDescriptor chosenNode = null;
     DatanodeDescriptor chosenNode = null;
     try {
     try {
-      String path = cachedFullPathNames.get(inode);
+      String path = cachedFullPathNames.get(bc);
       FileType type = getFileType(path);
       FileType type = getFileType(path);
       if (type == FileType.NOT_RAID) {
       if (type == FileType.NOT_RAID) {
         return defaultPolicy.chooseReplicaToDelete(
         return defaultPolicy.chooseReplicaToDelete(
-            inode, block, replicationFactor, first, second);
+            bc, block, replicationFactor, first, second);
       }
       }
       List<LocatedBlock> companionBlocks =
       List<LocatedBlock> companionBlocks =
           getCompanionBlocks(path, type, block);
           getCompanionBlocks(path, type, block);
       if (companionBlocks == null || companionBlocks.size() == 0) {
       if (companionBlocks == null || companionBlocks.size() == 0) {
         // Use the default method if it is not a valid raided or parity file
         // Use the default method if it is not a valid raided or parity file
         return defaultPolicy.chooseReplicaToDelete(
         return defaultPolicy.chooseReplicaToDelete(
-            inode, block, replicationFactor, first, second);
+            bc, block, replicationFactor, first, second);
       }
       }
       // Delete from the first collection first
       // Delete from the first collection first
       // This ensures the number of unique rack of this block is not reduced
       // This ensures the number of unique rack of this block is not reduced
@@ -174,12 +174,12 @@ public class BlockPlacementPolicyRaid extends BlockPlacementPolicy {
         return chosenNode;
         return chosenNode;
       }
       }
       return defaultPolicy.chooseReplicaToDelete(
       return defaultPolicy.chooseReplicaToDelete(
-          inode, block, replicationFactor, first, second);
+          bc, block, replicationFactor, first, second);
     } catch (Exception e) {
     } catch (Exception e) {
       LOG.debug("Error happend when choosing replica to delete" +
       LOG.debug("Error happend when choosing replica to delete" +
         StringUtils.stringifyException(e));
         StringUtils.stringifyException(e));
       return defaultPolicy.chooseReplicaToDelete(
       return defaultPolicy.chooseReplicaToDelete(
-          inode, block, replicationFactor, first, second);
+          bc, block, replicationFactor, first, second);
     }
     }
   }
   }
 
 
@@ -446,25 +446,25 @@ public class BlockPlacementPolicyRaid extends BlockPlacementPolicy {
       };
       };
 
 
     static private class INodeWithHashCode {
     static private class INodeWithHashCode {
-      BlockCollection inode;
-      INodeWithHashCode(BlockCollection inode) {
-        this.inode = inode;
+      BlockCollection bc;
+      INodeWithHashCode(BlockCollection bc) {
+        this.bc= bc;
       }
       }
       @Override
       @Override
       public boolean equals(Object obj) {
       public boolean equals(Object obj) {
-        return inode == obj;
+        return bc== obj;
       }
       }
       @Override
       @Override
       public int hashCode() {
       public int hashCode() {
-        return System.identityHashCode(inode);
+        return System.identityHashCode(bc);
       }
       }
       String getFullPathName() {
       String getFullPathName() {
-        return inode.getName();
+        return bc.getName();
       }
       }
     }
     }
 
 
-    public String get(BlockCollection inode) throws IOException {
-      return cacheInternal.get(new INodeWithHashCode(inode));
+    public String get(BlockCollection bc) throws IOException {
+      return cacheInternal.get(new INodeWithHashCode(bc));
     }
     }
   }
   }
 
 

+ 13 - 13
hadoop-mapreduce-project/src/contrib/raid/src/test/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockPlacementPolicyRaid.java

@@ -240,19 +240,19 @@ public class TestBlockPlacementPolicyRaid {
       // test full path cache
       // test full path cache
       CachedFullPathNames cachedFullPathNames =
       CachedFullPathNames cachedFullPathNames =
           new CachedFullPathNames(namesystem);
           new CachedFullPathNames(namesystem);
-      final BlockCollection[] inodes = NameNodeRaidTestUtil.getBlockCollections(
+      final BlockCollection[] bcs = NameNodeRaidTestUtil.getBlockCollections(
           namesystem, file1, file2);
           namesystem, file1, file2);
 
 
-      verifyCachedFullPathNameResult(cachedFullPathNames, inodes[0]);
-      verifyCachedFullPathNameResult(cachedFullPathNames, inodes[0]);
-      verifyCachedFullPathNameResult(cachedFullPathNames, inodes[1]);
-      verifyCachedFullPathNameResult(cachedFullPathNames, inodes[1]);
+      verifyCachedFullPathNameResult(cachedFullPathNames, bcs[0]);
+      verifyCachedFullPathNameResult(cachedFullPathNames, bcs[0]);
+      verifyCachedFullPathNameResult(cachedFullPathNames, bcs[1]);
+      verifyCachedFullPathNameResult(cachedFullPathNames, bcs[1]);
       try {
       try {
         Thread.sleep(1200L);
         Thread.sleep(1200L);
       } catch (InterruptedException e) {
       } catch (InterruptedException e) {
       }
       }
-      verifyCachedFullPathNameResult(cachedFullPathNames, inodes[1]);
-      verifyCachedFullPathNameResult(cachedFullPathNames, inodes[0]);
+      verifyCachedFullPathNameResult(cachedFullPathNames, bcs[1]);
+      verifyCachedFullPathNameResult(cachedFullPathNames, bcs[0]);
     } finally {
     } finally {
       if (cluster != null) {
       if (cluster != null) {
         cluster.shutdown();
         cluster.shutdown();
@@ -476,14 +476,14 @@ public class TestBlockPlacementPolicyRaid {
   }
   }
 
 
   private void verifyCachedFullPathNameResult(
   private void verifyCachedFullPathNameResult(
-      CachedFullPathNames cachedFullPathNames, BlockCollection inode)
+      CachedFullPathNames cachedFullPathNames, BlockCollection bc)
   throws IOException {
   throws IOException {
-    String res1 = inode.getName();
-    String res2 = cachedFullPathNames.get(inode);
+    String res1 = bc.getName();
+    String res2 = cachedFullPathNames.get(bc);
     LOG.info("Actual path name: " + res1);
     LOG.info("Actual path name: " + res1);
     LOG.info("Cached path name: " + res2);
     LOG.info("Cached path name: " + res2);
-    Assert.assertEquals(cachedFullPathNames.get(inode),
-                        inode.getName());
+    Assert.assertEquals(cachedFullPathNames.get(bc),
+                        bc.getName());
   }
   }
 
 
   private void verifyCachedBlocksResult(CachedLocatedBlocks cachedBlocks,
   private void verifyCachedBlocksResult(CachedLocatedBlocks cachedBlocks,
@@ -502,7 +502,7 @@ public class TestBlockPlacementPolicyRaid {
   private Collection<LocatedBlock> getCompanionBlocks(
   private Collection<LocatedBlock> getCompanionBlocks(
       FSNamesystem namesystem, BlockPlacementPolicyRaid policy,
       FSNamesystem namesystem, BlockPlacementPolicyRaid policy,
       ExtendedBlock block) throws IOException {
       ExtendedBlock block) throws IOException {
-    INodeFile inode = (INodeFile)blockManager.blocksMap.getINode(block
+    INodeFile inode = (INodeFile)blockManager.blocksMap.getBlockCollection(block
         .getLocalBlock());
         .getLocalBlock());
     FileType type = policy.getFileType(inode.getFullPathName());
     FileType type = policy.getFileType(inode.getFullPathName());
     return policy.getCompanionBlocks(inode.getFullPathName(), type,
     return policy.getCompanionBlocks(inode.getFullPathName(), type,