فهرست منبع

HADOOP-4687 Merge of 776174:784663 from trunk to the branch of:
src/hdfs
src/contrib/hdfsproxy
src/test/hdfs
src/test/hdfs-with-mr
src/webapps/datanode
src/webapps/hdfs
src/webapps/secondary


git-svn-id: https://svn.apache.org/repos/asf/hadoop/core/branches/HADOOP-4687/hdfs@785005 13f79535-47bb-0310-9956-ffa450edef68

Owen O'Malley 16 سال پیش
والد
کامیت
29aecea97d
61فایلهای تغییر یافته به همراه2646 افزوده شده و 1812 حذف شده
  1. BIN
      lib/hadoop-core-0.21.0-dev.jar
  2. BIN
      lib/hadoop-core-test-0.21.0-dev.jar
  3. 6 1
      src/contrib/hdfsproxy/bin/hdfsproxy
  4. 7 5
      src/contrib/hdfsproxy/build.xml
  5. 580 338
      src/java/org/apache/hadoop/hdfs/DFSClient.java
  6. 62 0
      src/java/org/apache/hadoop/hdfs/DeprecatedUTF8.java
  7. 10 5
      src/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
  8. 4 3
      src/java/org/apache/hadoop/hdfs/HftpFileSystem.java
  9. 7 5
      src/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
  10. 1 1
      src/java/org/apache/hadoop/hdfs/protocol/DatanodeID.java
  11. 16 9
      src/java/org/apache/hadoop/hdfs/protocol/DatanodeInfo.java
  12. 5 0
      src/java/org/apache/hadoop/hdfs/protocol/FSConstants.java
  13. 2 69
      src/java/org/apache/hadoop/hdfs/server/common/JspHelper.java
  14. 1 1
      src/java/org/apache/hadoop/hdfs/server/common/Storage.java
  15. 17 0
      src/java/org/apache/hadoop/hdfs/server/common/StorageInfo.java
  16. 4 8
      src/java/org/apache/hadoop/hdfs/server/datanode/DataBlockScanner.java
  17. 11 1
      src/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
  18. 11 2
      src/java/org/apache/hadoop/hdfs/server/datanode/DataStorage.java
  19. 11 3
      src/java/org/apache/hadoop/hdfs/server/datanode/DatanodeBlockInfo.java
  20. 580 0
      src/java/org/apache/hadoop/hdfs/server/datanode/DatanodeJspHelper.java
  21. 4 1
      src/java/org/apache/hadoop/hdfs/server/namenode/BackupNode.java
  22. 96 71
      src/java/org/apache/hadoop/hdfs/server/namenode/BlockManager.java
  23. 20 4
      src/java/org/apache/hadoop/hdfs/server/namenode/BlocksMap.java
  24. 20 7
      src/java/org/apache/hadoop/hdfs/server/namenode/DatanodeDescriptor.java
  25. 1 0
      src/java/org/apache/hadoop/hdfs/server/namenode/DfsServlet.java
  26. 4 5
      src/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
  27. 1 1
      src/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java
  28. 20 40
      src/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
  29. 1 0
      src/java/org/apache/hadoop/hdfs/server/namenode/FileDataServlet.java
  30. 2 1
      src/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectory.java
  31. 21 4
      src/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
  32. 452 0
      src/java/org/apache/hadoop/hdfs/server/namenode/NamenodeJspHelper.java
  33. 9 2
      src/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java
  34. 1 0
      src/java/org/apache/hadoop/hdfs/server/namenode/StreamFile.java
  35. 1 1
      src/java/org/apache/hadoop/hdfs/server/protocol/NamespaceInfo.java
  36. 89 2
      src/test/hdfs-with-mr/org/apache/hadoop/fs/TestCopyFiles.java
  37. 46 0
      src/test/hdfs-with-mr/org/apache/hadoop/fs/TestFileSystem.java
  38. 46 35
      src/test/hdfs-with-mr/org/apache/hadoop/hdfs/NNBench.java
  39. 36 21
      src/test/hdfs-with-mr/org/apache/hadoop/hdfs/NNBenchWithoutMR.java
  40. 71 0
      src/test/hdfs/org/apache/hadoop/cli/clitest_data/data1k
  41. 54 27
      src/test/hdfs/org/apache/hadoop/cli/testHDFSConf.xml
  42. 2 3
      src/test/hdfs/org/apache/hadoop/hdfs/TestDFSShellGenericOptions.java
  43. 7 10
      src/test/hdfs/org/apache/hadoop/hdfs/TestDFSStartupVersions.java
  44. 97 0
      src/test/hdfs/org/apache/hadoop/hdfs/TestFileCreation.java
  45. 0 5
      src/test/hdfs/org/apache/hadoop/hdfs/TestFileStatus.java
  46. 137 19
      src/test/hdfs/org/apache/hadoop/hdfs/TestHDFSServerPorts.java
  47. 0 1
      src/test/hdfs/org/apache/hadoop/hdfs/TestModTime.java
  48. 1 1
      src/test/hdfs/org/apache/hadoop/hdfs/TestSeekBug.java
  49. 1 1
      src/test/hdfs/org/apache/hadoop/hdfs/TestSmallBlock.java
  50. 8 3
      src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/NNThroughputBenchmark.java
  51. 9 0
      src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java
  52. 4 0
      src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestNameEditsConfigs.java
  53. 2 0
      src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestStartup.java
  54. 1 0
      src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestStorageRestore.java
  55. 7 367
      src/webapps/datanode/browseBlock.jsp
  56. 6 132
      src/webapps/datanode/browseDirectory.jsp
  57. 5 96
      src/webapps/datanode/tail.jsp
  58. 11 232
      src/webapps/hdfs/dfshealth.jsp
  59. 8 231
      src/webapps/hdfs/dfsnodelist.jsp
  60. 4 37
      src/webapps/hdfs/nn_browsedfscontent.jsp
  61. 6 1
      src/webapps/secondary/status.jsp

BIN
lib/hadoop-core-0.21.0-dev.jar


BIN
lib/hadoop-core-test-0.21.0-dev.jar


+ 6 - 1
src/contrib/hdfsproxy/bin/hdfsproxy

@@ -114,12 +114,17 @@ if [ -d "$HDFSPROXY_HOME/../../" ]; then
     CLASSPATH=${CLASSPATH}:$f;
   done
 fi
-
 if [ -d "$HDFSPROXY_HOME/../../lib" ]; then
 	for f in $HDFSPROXY_HOME/../../lib/*.jar; do
     CLASSPATH=${CLASSPATH}:$f;
   done
 fi
+if [ -d "$HDFSPROXY_HOME/../../lib/jsp-2.1" ]; then
+	for f in $HDFSPROXY_HOME/../../lib/jsp-2.1/*.jar; do
+    CLASSPATH=${CLASSPATH}:$f;
+  done
+fi
+
 
 # add user-specified CLASSPATH last
 if [ "$HDFSPROXY_CLASSPATH" != "" ]; then

+ 7 - 5
src/contrib/hdfsproxy/build.xml

@@ -151,7 +151,7 @@
         <include name="slf4j-api-${slf4j-api.version}.jar"/>
         <include name="slf4j-log4j12-${slf4j-log4j12.version}.jar"/>
         <include name="xmlenc-${xmlenc.version}.jar"/>
-        <include name="core-${core.version}.jar"/> 
+        <include name="core-${core.vesion}.jar"/> 
 	    </lib>
 	    <classes dir="${proxy.conf.dir}" excludes="**/*.example **/*.template **/*.sh hadoop-site.xml"/>
 	    <classes dir="${build.classes}"/>
@@ -355,10 +355,12 @@
         <include name="jetty-util-${jetty-util.version}.jar"/>
         <include name="jetty-${jetty.version}.jar"/>
         <include name="servlet-api-2.5-${servlet-api-2.5.version}.jar"/>
-        <include name="core-${core.version}.jar"/> 
-        <include name="jsp-${jsp.version}-${jetty.version}.jar"/> 
-        <include name="jsp-api-${jsp.version}-${jetty.version}.jar"/> 
-		       </fileset>
+        <include name="core-${core.vesion}.jar"/> 
+                       </fileset>
+		       <fileset dir="${hadoop.root}/lib/jsp-${jsp.version}">
+        <include name="jsp-${jsp.version}.jar"/> 
+        <include name="jsp-api-${jsp.version}.jar"/> 
+			</fileset>
 		</copy>
 
 		<copy todir="${build.dir}/${final.name}/lib" includeEmptyDirs="false">

تفاوت فایلی نمایش داده نمی شود زیرا این فایل بسیار بزرگ است
+ 580 - 338
src/java/org/apache/hadoop/hdfs/DFSClient.java


+ 62 - 0
src/java/org/apache/hadoop/hdfs/DeprecatedUTF8.java

@@ -0,0 +1,62 @@
+/**
+ * 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;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+/**
+ * A simple wrapper around {@link org.apache.hadoop.io.UTF8}.
+ * This class should be used only when it is absolutely necessary
+ * to use {@link org.apache.hadoop.io.UTF8}. The only difference is that 
+ * using this class does not require "@SuppressWarning" annotation to avoid 
+ * javac warning. Instead the deprecation is implied in the class name.
+ * 
+ * This should be treated as package private class to HDFS.
+ */
+@SuppressWarnings("deprecation")
+public class DeprecatedUTF8 extends org.apache.hadoop.io.UTF8 {
+  
+  public DeprecatedUTF8() {
+    super();
+  }
+
+  /** Construct from a given string. */
+  public DeprecatedUTF8(String string) {
+    super(string);
+  }
+
+  /** Construct from a given string. */
+  public DeprecatedUTF8(DeprecatedUTF8 utf8) {
+    super(utf8);
+  }
+  
+  /* The following two are the mostly commonly used methods.
+   * wrapping them so that editors do not complain about the deprecation.
+   */
+  
+  public static String readString(DataInput in) throws IOException {
+    return org.apache.hadoop.io.UTF8.readString(in);
+  }
+  
+  public static int writeString(DataOutput out, String s) throws IOException {
+    return org.apache.hadoop.io.UTF8.writeString(out, s);
+  }
+}

+ 10 - 5
src/java/org/apache/hadoop/hdfs/DistributedFileSystem.java

@@ -20,6 +20,7 @@ package org.apache.hadoop.hdfs;
 
 import java.io.*;
 import java.net.*;
+import java.util.EnumSet;
 
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.*;
@@ -32,6 +33,7 @@ import org.apache.hadoop.hdfs.protocol.FSConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.protocol.FSConstants.UpgradeAction;
 import org.apache.hadoop.hdfs.server.common.UpgradeStatusReport;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
+import org.apache.hadoop.hdfs.DFSClient.DFSDataInputStream;
 import org.apache.hadoop.hdfs.DFSClient.DFSOutputStream;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.util.Progressable;
@@ -79,7 +81,7 @@ public class DistributedFileSystem extends FileSystem {
 
     InetSocketAddress namenode = NameNode.getAddress(uri.getAuthority());
     this.dfs = new DFSClient(namenode, conf, statistics);
-    this.uri = URI.create("hdfs://" + uri.getAuthority());
+    this.uri = URI.create(FSConstants.HDFS_URI_SCHEME + "://" + uri.getAuthority());
     this.workingDir = getHomeDirectory();
   }
 
@@ -198,13 +200,12 @@ public class DistributedFileSystem extends FileSystem {
 
   @Override
   public FSDataOutputStream create(Path f, FsPermission permission,
-    boolean overwrite,
-    int bufferSize, short replication, long blockSize,
+    EnumSet<CreateFlag> flag, int bufferSize, short replication, long blockSize,
     Progressable progress) throws IOException {
 
     return new FSDataOutputStream
        (dfs.create(getPathName(f), permission,
-                   overwrite, replication, blockSize, progress, bufferSize),
+                   flag, replication, blockSize, progress, bufferSize),
         statistics);
   }
 
@@ -439,7 +440,11 @@ public class DistributedFileSystem extends FileSystem {
   public boolean reportChecksumFailure(Path f, 
     FSDataInputStream in, long inPos, 
     FSDataInputStream sums, long sumsPos) {
-
+    
+    if(!(in instanceof DFSDataInputStream && sums instanceof DFSDataInputStream))
+      throw new IllegalArgumentException("Input streams must be types " +
+                                         "of DFSDataInputStream");
+    
     LocatedBlock lblocks[] = new LocatedBlock[2];
 
     // Find block in data stream.

+ 4 - 3
src/java/org/apache/hadoop/hdfs/HftpFileSystem.java

@@ -29,11 +29,13 @@ import java.net.URL;
 import java.text.ParseException;
 import java.text.SimpleDateFormat;
 import java.util.ArrayList;
+import java.util.EnumSet;
 import java.util.Random;
 
 import javax.security.auth.login.LoginException;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FSInputStream;
@@ -298,9 +300,8 @@ public class HftpFileSystem extends FileSystem {
 
   @Override
   public FSDataOutputStream create(Path f, FsPermission permission,
-                                   boolean overwrite, int bufferSize,
-                                   short replication, long blockSize,
-                                   Progressable progress) throws IOException {
+      EnumSet<CreateFlag> flag, int bufferSize, short replication,
+      long blockSize, Progressable progress) throws IOException {
     throw new IOException("Not supported");
   }
 

+ 7 - 5
src/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java

@@ -22,9 +22,11 @@ import java.io.IOException;
 
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.protocol.FSConstants.UpgradeAction;
 import org.apache.hadoop.hdfs.server.common.UpgradeStatusReport;
+import org.apache.hadoop.io.EnumSetWritable;
 import org.apache.hadoop.ipc.VersionedProtocol;
 import org.apache.hadoop.security.AccessControlException;
 
@@ -41,9 +43,9 @@ public interface ClientProtocol extends VersionedProtocol {
    * Compared to the previous version the following changes have been introduced:
    * (Only the latest change is reflected.
    * The log of historical changes can be retrieved from the svn).
-   * 44: All LocatedBlock objects contain access tokens
+   * 45: add create flag for create command, see Hadoop-5438
    */
-  public static final long versionID = 44L;
+  public static final long versionID = 45L;
   
   ///////////////////////////////////////
   // File contents
@@ -89,8 +91,8 @@ public interface ClientProtocol extends VersionedProtocol {
    * @param src path of the file being created.
    * @param masked masked permission.
    * @param clientName name of the current client.
-   * @param overwrite indicates whether the file should be 
-   * overwritten if it already exists.
+   * @param flag indicates whether the file should be 
+   * overwritten if it already exists or create if it does not exist or append.
    * @param replication block replication factor.
    * @param blockSize maximum block size.
    * 
@@ -104,7 +106,7 @@ public interface ClientProtocol extends VersionedProtocol {
   public void create(String src, 
                      FsPermission masked,
                              String clientName, 
-                             boolean overwrite, 
+                             EnumSetWritable<CreateFlag> flag, 
                              short replication,
                              long blockSize
                              ) throws IOException;

+ 1 - 1
src/java/org/apache/hadoop/hdfs/protocol/DatanodeID.java

@@ -22,7 +22,7 @@ import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
 
-import org.apache.hadoop.io.DeprecatedUTF8;
+import org.apache.hadoop.hdfs.DeprecatedUTF8;
 import org.apache.hadoop.io.WritableComparable;
 
 /**

+ 16 - 9
src/java/org/apache/hadoop/hdfs/protocol/DatanodeInfo.java

@@ -246,24 +246,18 @@ public class DatanodeInfo extends DatanodeID implements Node {
    * Returns true if the node is in the process of being decommissioned
    */
   public boolean isDecommissionInProgress() {
-    if (adminState == AdminStates.DECOMMISSION_INPROGRESS) {
-      return true;
-    }
-    return false;
+    return adminState == AdminStates.DECOMMISSION_INPROGRESS;
   }
 
   /**
    * Returns true if the node has been decommissioned.
    */
   public boolean isDecommissioned() {
-    if (adminState == AdminStates.DECOMMISSIONED) {
-      return true;
-    }
-    return false;
+    return adminState == AdminStates.DECOMMISSIONED;
   }
 
   /**
-   * Sets the admin state to indicate that decommision is complete.
+   * Sets the admin state to indicate that decommission is complete.
    */
   public void setDecommissioned() {
     adminState = AdminStates.DECOMMISSIONED;
@@ -348,4 +342,17 @@ public class DatanodeInfo extends DatanodeID implements Node {
     this.hostName = Text.readString(in);
     setAdminState(WritableUtils.readEnum(in, AdminStates.class));
   }
+
+  @Override
+  public int hashCode() {
+    // Super implementation is sufficient
+    return super.hashCode();
+  }
+  
+  @Override
+  public boolean equals(Object obj) {
+    // Sufficient to use super equality as datanodes are uniquely identified
+    // by DatanodeID
+    return (this == obj) || super.equals(obj);
+  }
 }

+ 5 - 0
src/java/org/apache/hadoop/hdfs/protocol/FSConstants.java

@@ -76,6 +76,11 @@ public interface FSConstants {
     FORCE_PROCEED;
   }
 
+  /**
+   * URI Scheme for hdfs://namenode/ URIs.
+   */
+  public static final String HDFS_URI_SCHEME = "hdfs";
+
   // Version is reflected in the dfs image and edit log files.
   // Version is reflected in the data storage file.
   // Versions are negative.

+ 2 - 69
src/java/org/apache/hadoop/hdfs/server/namenode/JspHelper.java → src/java/org/apache/hadoop/hdfs/server/common/JspHelper.java

@@ -16,12 +16,11 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.hdfs.server.namenode;
+package org.apache.hadoop.hdfs.server.common;
 
 import java.io.IOException;
 import java.io.UnsupportedEncodingException;
 import java.net.InetSocketAddress;
-import java.net.MalformedURLException;
 import java.net.Socket;
 import java.net.URL;
 import java.net.URLEncoder;
@@ -39,13 +38,10 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.DFSClient;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
-import org.apache.hadoop.hdfs.protocol.FSConstants.UpgradeAction;
-import org.apache.hadoop.hdfs.server.common.HdfsConstants;
-import org.apache.hadoop.hdfs.server.common.UpgradeStatusReport;
+import org.apache.hadoop.hdfs.server.namenode.DatanodeDescriptor;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.AccessToken;
 import org.apache.hadoop.security.UnixUserGroupInformation;
-import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.VersionInfo;
 
 public class JspHelper {
@@ -178,59 +174,6 @@ public class JspHelper {
     out.print("</tbody></table>");
   }
 
-  public static String getSafeModeText(FSNamesystem fsn) {
-    if (!fsn.isInSafeMode())
-      return "";
-    return "Safe mode is ON. <em>" + fsn.getSafeModeTip() + "</em><br>";
-  }
-
-  public static String getWarningText(FSNamesystem fsn) {
-    // Ideally this should be displayed in RED
-    long missingBlocks = fsn.getMissingBlocksCount();
-    if (missingBlocks > 0) {
-      return "<br> WARNING :" + 
-             " There are about " + missingBlocks +
-             " missing blocks. Please check the log or run fsck. <br><br>";
-    }
-    return "";
-  }
-  
-  public static String getInodeLimitText(FSNamesystem fsn) {
-    long inodes = fsn.dir.totalInodes();
-    long blocks = fsn.getBlocksTotal();
-    long maxobjects = fsn.getMaxObjects();
-    long totalMemory = Runtime.getRuntime().totalMemory();   
-    long maxMemory = Runtime.getRuntime().maxMemory();   
-
-    long used = (totalMemory * 100)/maxMemory;
- 
-    String str = inodes + " files and directories, " +
-                 blocks + " blocks = " +
-                 (inodes + blocks) + " total";
-    if (maxobjects != 0) {
-      long pct = ((inodes + blocks) * 100)/maxobjects;
-      str += " / " + maxobjects + " (" + pct + "%)";
-    }
-    str += ".  Heap Size is " + StringUtils.byteDesc(totalMemory) + " / " + 
-           StringUtils.byteDesc(maxMemory) + 
-           " (" + used + "%) <br>";
-    return str;
-  }
-
-  public static String getUpgradeStatusText(FSNamesystem fsn) {
-    String statusText = "";
-    try {
-      UpgradeStatusReport status = 
-        fsn.distributedUpgradeProgress(UpgradeAction.GET_STATUS);
-      statusText = (status == null ? 
-          "There are no upgrades in progress." :
-            status.getStatusText(false));
-    } catch(IOException e) {
-      statusText = "Upgrade status unknown.";
-    }
-    return statusText;
-  }
-
   public static void sortNodeList(ArrayList<DatanodeDescriptor> nodes,
                            String field, String order) {
         
@@ -377,16 +320,6 @@ public class JspHelper {
     return n > 0? n: defaultChunkSizeToView;
   }
 
-  /** Return a table containing version information. */
-  public static String getVersionTable(FSNamesystem fsn) {
-    return "<div id='dfstable'><table>"       
-        + "\n  <tr><td id='col1'>Started:</td><td>" + fsn.getStartTime() + "</td></tr>\n"
-        + "\n  <tr><td id='col1'>Version:</td><td>" + VersionInfo.getVersion() + ", " + VersionInfo.getRevision()
-        + "\n  <tr><td id='col1'>Compiled:</td><td>" + VersionInfo.getDate() + " by " + VersionInfo.getUser() + " from " + VersionInfo.getBranch()
-        + "\n  <tr><td id='col1'>Upgrades:</td><td>" + getUpgradeStatusText(fsn)
-        + "\n</table></div>";
-  }
-
   /** Return a table containing version information. */
   public static String getVersionTable() {
     return "<div id='dfstable'><table>"       

+ 1 - 1
src/java/org/apache/hadoop/hdfs/server/common/Storage.java

@@ -767,7 +767,7 @@ public abstract class Storage extends StorageInfo {
   
     file.seek(0);
     file.writeInt(FSConstants.LAYOUT_VERSION);
-    org.apache.hadoop.io.DeprecatedUTF8.writeString(file, "");
+    org.apache.hadoop.hdfs.DeprecatedUTF8.writeString(file, "");
     file.writeBytes(messageForPreUpgradeVersion);
     file.getFD().sync();
   }

+ 17 - 0
src/java/org/apache/hadoop/hdfs/server/common/StorageInfo.java

@@ -1,3 +1,20 @@
+/**
+ * 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.common;
 
 import java.io.DataInput;

+ 4 - 8
src/java/org/apache/hadoop/hdfs/server/datanode/DataBlockScanner.java

@@ -346,7 +346,10 @@ class DataBlockScanner implements Runnable {
     info.lastLogTime = now;
     LogFileHandler log = verificationLog;
     if (log != null) {
-      log.appendLine(LogEntry.newEnry(block, now));
+      log.appendLine("date=\"" + dateFormat.format(new Date(now)) + "\"\t " +
+          "time=\"" + now + "\"\t " +
+          "genstamp=\"" + block.getGenerationStamp() + "\"\t " +
+          "id=\"" + block.getBlockId() +"\"");
     }
   }
   
@@ -381,13 +384,6 @@ class DataBlockScanner implements Runnable {
     private static Pattern entryPattern = 
       Pattern.compile("\\G\\s*([^=\\p{Space}]+)=\"(.*?)\"\\s*");
     
-    static String newEnry(Block block, long time) {
-      return "date=\"" + dateFormat.format(new Date(time)) + "\"\t " +
-             "time=\"" + time + "\"\t " +
-             "genstamp=\"" + block.getGenerationStamp() + "\"\t " +
-             "id=\"" + block.getBlockId() +"\"";
-    }
-    
     static LogEntry parseEntry(String line) {
       LogEntry entry = new LogEntry();
       

+ 11 - 1
src/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java

@@ -93,6 +93,7 @@ import org.apache.hadoop.security.authorize.PolicyProvider;
 import org.apache.hadoop.security.authorize.ServiceAuthorizationManager;
 import org.apache.hadoop.util.Daemon;
 import org.apache.hadoop.util.DiskChecker;
+import org.apache.hadoop.util.GenericOptionsParser;
 import org.apache.hadoop.util.ServicePlugin;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.StringUtils;
@@ -632,6 +633,7 @@ public class DataNode extends Configured
 
       // wait for all data receiver threads to exit
       if (this.threadGroup != null) {
+        int sleepMs = 2;
         while (true) {
           this.threadGroup.interrupt();
           LOG.info("Waiting for threadgroup to exit, active threads is " +
@@ -640,8 +642,12 @@ public class DataNode extends Configured
             break;
           }
           try {
-            Thread.sleep(1000);
+            Thread.sleep(sleepMs);
           } catch (InterruptedException e) {}
+          sleepMs = sleepMs * 3 / 2; // exponential backoff
+          if (sleepMs > 1000) {
+            sleepMs = 1000;
+          }
         }
       }
       // wait for dataXceiveServer to terminate
@@ -1304,6 +1310,10 @@ public class DataNode extends Configured
                                       Configuration conf) throws IOException {
     if (conf == null)
       conf = new Configuration();
+    // parse generic hadoop options
+    GenericOptionsParser hParser = new GenericOptionsParser(conf, args);
+    args = hParser.getRemainingArgs();
+    
     if (!parseArguments(args, conf)) {
       printUsage();
       return null;

+ 11 - 2
src/java/org/apache/hadoop/hdfs/server/datanode/DataStorage.java

@@ -363,8 +363,17 @@ public class DataStorage extends Storage {
   static void linkBlocks(File from, File to, int oldLV) throws IOException {
     if (!from.isDirectory()) {
       if (from.getName().startsWith(COPY_FILE_PREFIX)) {
-        IOUtils.copyBytes(new FileInputStream(from), 
-                          new FileOutputStream(to), 16*1024, true);
+        FileInputStream in = new FileInputStream(from);
+        try {
+          FileOutputStream out = new FileOutputStream(to);
+          try {
+            IOUtils.copyBytes(in, out, 16*1024);
+          } finally {
+            out.close();
+          }
+        } finally {
+          in.close();
+        }
       } else {
         
         //check if we are upgrading from pre-generation stamp version.

+ 11 - 3
src/java/org/apache/hadoop/hdfs/server/datanode/DatanodeBlockInfo.java

@@ -86,9 +86,17 @@ class DatanodeBlockInfo {
   private void detachFile(File file, Block b) throws IOException {
     File tmpFile = volume.createDetachFile(b, file.getName());
     try {
-      IOUtils.copyBytes(new FileInputStream(file),
-                        new FileOutputStream(tmpFile),
-                        16*1024, true);
+      FileInputStream in = new FileInputStream(file);
+      try {
+        FileOutputStream out = new FileOutputStream(tmpFile);
+        try {
+          IOUtils.copyBytes(in, out, 16*1024);
+        } finally {
+          out.close();
+        }
+      } finally {
+        in.close();
+      }
       if (file.length() != tmpFile.length()) {
         throw new IOException("Copy of file " + file + " size " + file.length()+
                               " into file " + tmpFile +

+ 580 - 0
src/java/org/apache/hadoop/hdfs/server/datanode/DatanodeJspHelper.java

@@ -0,0 +1,580 @@
+/**
+ * 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.datanode;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.net.URLEncoder;
+import java.util.Date;
+import java.util.List;
+
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+import javax.servlet.jsp.JspWriter;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FsShell;
+import org.apache.hadoop.hdfs.DFSClient;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.server.common.JspHelper;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.security.AccessToken;
+import org.apache.hadoop.security.AccessTokenHandler;
+import org.apache.hadoop.util.StringUtils;
+
+class DatanodeJspHelper {
+  private static final DataNode datanode = DataNode.getDataNode();
+
+  static void generateDirectoryStructure(JspWriter out, HttpServletRequest req,
+      HttpServletResponse resp) throws IOException {
+    final String dir = JspHelper.validatePath(req.getParameter("dir"));
+    if (dir == null) {
+      out.print("Invalid input");
+      return;
+    }
+
+    String namenodeInfoPortStr = req.getParameter("namenodeInfoPort");
+    int namenodeInfoPort = -1;
+    if (namenodeInfoPortStr != null)
+      namenodeInfoPort = Integer.parseInt(namenodeInfoPortStr);
+
+    final DFSClient dfs = new DFSClient(datanode.getNameNodeAddr(),
+        JspHelper.conf);
+    String target = dir;
+    final FileStatus targetStatus = dfs.getFileInfo(target);
+    if (targetStatus == null) { // not exists
+      out.print("<h3>File or directory : " + target + " does not exist</h3>");
+      JspHelper.printGotoForm(out, namenodeInfoPort, target);
+    } else {
+      if (!targetStatus.isDir()) { // a file
+        List<LocatedBlock> blocks = dfs.namenode.getBlockLocations(dir, 0, 1)
+            .getLocatedBlocks();
+
+        LocatedBlock firstBlock = null;
+        DatanodeInfo[] locations = null;
+        if (blocks.size() > 0) {
+          firstBlock = blocks.get(0);
+          locations = firstBlock.getLocations();
+        }
+        if (locations == null || locations.length == 0) {
+          out.print("Empty file");
+        } else {
+          DatanodeInfo chosenNode = JspHelper.bestNode(firstBlock);
+          String fqdn = InetAddress.getByName(chosenNode.getHost())
+              .getCanonicalHostName();
+          String datanodeAddr = chosenNode.getName();
+          int datanodePort = Integer.parseInt(datanodeAddr.substring(
+              datanodeAddr.indexOf(':') + 1, datanodeAddr.length()));
+          String redirectLocation = "http://" + fqdn + ":"
+              + chosenNode.getInfoPort() + "/browseBlock.jsp?blockId="
+              + firstBlock.getBlock().getBlockId() + "&blockSize="
+              + firstBlock.getBlock().getNumBytes() + "&genstamp="
+              + firstBlock.getBlock().getGenerationStamp() + "&filename="
+              + URLEncoder.encode(dir, "UTF-8") + "&datanodePort="
+              + datanodePort + "&namenodeInfoPort=" + namenodeInfoPort;
+          resp.sendRedirect(redirectLocation);
+        }
+        return;
+      }
+      // directory
+      FileStatus[] files = dfs.listPaths(target);
+      // generate a table and dump the info
+      String[] headings = { "Name", "Type", "Size", "Replication",
+          "Block Size", "Modification Time", "Permission", "Owner", "Group" };
+      out.print("<h3>Contents of directory ");
+      JspHelper.printPathWithLinks(dir, out, namenodeInfoPort);
+      out.print("</h3><hr>");
+      JspHelper.printGotoForm(out, namenodeInfoPort, dir);
+      out.print("<hr>");
+
+      File f = new File(dir);
+      String parent;
+      if ((parent = f.getParent()) != null)
+        out.print("<a href=\"" + req.getRequestURL() + "?dir=" + parent
+            + "&namenodeInfoPort=" + namenodeInfoPort
+            + "\">Go to parent directory</a><br>");
+
+      if (files == null || files.length == 0) {
+        out.print("Empty directory");
+      } else {
+        JspHelper.addTableHeader(out);
+        int row = 0;
+        JspHelper.addTableRow(out, headings, row++);
+        String cols[] = new String[headings.length];
+        for (int i = 0; i < files.length; i++) {
+          // Get the location of the first block of the file
+          if (files[i].getPath().toString().endsWith(".crc"))
+            continue;
+          if (!files[i].isDir()) {
+            cols[1] = "file";
+            cols[2] = StringUtils.byteDesc(files[i].getLen());
+            cols[3] = Short.toString(files[i].getReplication());
+            cols[4] = StringUtils.byteDesc(files[i].getBlockSize());
+          } else {
+            cols[1] = "dir";
+            cols[2] = "";
+            cols[3] = "";
+            cols[4] = "";
+          }
+          String datanodeUrl = req.getRequestURL() + "?dir="
+              + URLEncoder.encode(files[i].getPath().toString(), "UTF-8")
+              + "&namenodeInfoPort=" + namenodeInfoPort;
+          cols[0] = "<a href=\"" + datanodeUrl + "\">"
+              + files[i].getPath().getName() + "</a>";
+          cols[5] = FsShell.dateForm.format(new Date((files[i]
+              .getModificationTime())));
+          cols[6] = files[i].getPermission().toString();
+          cols[7] = files[i].getOwner();
+          cols[8] = files[i].getGroup();
+          JspHelper.addTableRow(out, cols, row++);
+        }
+        JspHelper.addTableFooter(out);
+      }
+    }
+    String namenodeHost = datanode.getNameNodeAddr().getHostName();
+    out.print("<br><a href=\"http://"
+        + InetAddress.getByName(namenodeHost).getCanonicalHostName() + ":"
+        + namenodeInfoPort + "/dfshealth.jsp\">Go back to DFS home</a>");
+    dfs.close();
+  }
+
+  static void generateFileDetails(JspWriter out, HttpServletRequest req)
+      throws IOException {
+
+    long startOffset = 0;
+    int datanodePort;
+
+    final Long blockId = JspHelper.validateLong(req.getParameter("blockId"));
+    if (blockId == null) {
+      out.print("Invalid input (blockId absent)");
+      return;
+    }
+
+    String datanodePortStr = req.getParameter("datanodePort");
+    if (datanodePortStr == null) {
+      out.print("Invalid input (datanodePort absent)");
+      return;
+    }
+    datanodePort = Integer.parseInt(datanodePortStr);
+
+    String namenodeInfoPortStr = req.getParameter("namenodeInfoPort");
+    int namenodeInfoPort = -1;
+    if (namenodeInfoPortStr != null)
+      namenodeInfoPort = Integer.parseInt(namenodeInfoPortStr);
+
+    final int chunkSizeToView = JspHelper.string2ChunkSizeToView(
+        req.getParameter("chunkSizeToView"));
+
+    String startOffsetStr = req.getParameter("startOffset");
+    if (startOffsetStr == null || Long.parseLong(startOffsetStr) < 0)
+      startOffset = 0;
+    else
+      startOffset = Long.parseLong(startOffsetStr);
+
+    final String filename=JspHelper.validatePath(req.getParameter("filename"));
+    if (filename == null) {
+      out.print("Invalid input");
+      return;
+    }
+
+    String blockSizeStr = req.getParameter("blockSize");
+    long blockSize = 0;
+    if (blockSizeStr == null || blockSizeStr.length() == 0) {
+      out.print("Invalid input");
+      return;
+    }
+    blockSize = Long.parseLong(blockSizeStr);
+
+    final DFSClient dfs = new DFSClient(datanode.getNameNodeAddr(),
+        JspHelper.conf);
+    List<LocatedBlock> blocks = dfs.namenode.getBlockLocations(filename, 0,
+        Long.MAX_VALUE).getLocatedBlocks();
+    // Add the various links for looking at the file contents
+    // URL for downloading the full file
+    String downloadUrl = "http://" + req.getServerName() + ":"
+        + req.getServerPort() + "/streamFile?" + "filename="
+        + URLEncoder.encode(filename, "UTF-8");
+    out.print("<a name=\"viewOptions\"></a>");
+    out.print("<a href=\"" + downloadUrl + "\">Download this file</a><br>");
+
+    DatanodeInfo chosenNode;
+    // URL for TAIL
+    LocatedBlock lastBlk = blocks.get(blocks.size() - 1);
+    try {
+      chosenNode = JspHelper.bestNode(lastBlk);
+    } catch (IOException e) {
+      out.print(e.toString());
+      dfs.close();
+      return;
+    }
+    String fqdn = InetAddress.getByName(chosenNode.getHost())
+        .getCanonicalHostName();
+    String tailUrl = "http://" + fqdn + ":" + chosenNode.getInfoPort()
+        + "/tail.jsp?filename=" + URLEncoder.encode(filename, "UTF-8")
+        + "&namenodeInfoPort=" + namenodeInfoPort
+        + "&chunkSizeToView=" + chunkSizeToView
+        + "&referrer=" + URLEncoder.encode(
+            req.getRequestURL() + "?" + req.getQueryString(), "UTF-8");
+    out.print("<a href=\"" + tailUrl + "\">Tail this file</a><br>");
+
+    out.print("<form action=\"/browseBlock.jsp\" method=GET>");
+    out.print("<b>Chunk size to view (in bytes, up to file's DFS block size): </b>");
+    out.print("<input type=\"hidden\" name=\"blockId\" value=\"" + blockId
+        + "\">");
+    out.print("<input type=\"hidden\" name=\"blockSize\" value=\"" + blockSize
+        + "\">");
+    out.print("<input type=\"hidden\" name=\"startOffset\" value=\""
+        + startOffset + "\">");
+    out.print("<input type=\"hidden\" name=\"filename\" value=\"" + filename
+        + "\">");
+    out.print("<input type=\"hidden\" name=\"datanodePort\" value=\""
+        + datanodePort + "\">");
+    out.print("<input type=\"hidden\" name=\"namenodeInfoPort\" value=\""
+        + namenodeInfoPort + "\">");
+    out.print("<input type=\"text\" name=\"chunkSizeToView\" value="
+        + chunkSizeToView + " size=10 maxlength=10>");
+    out.print("&nbsp;&nbsp;<input type=\"submit\" name=\"submit\" value=\"Refresh\">");
+    out.print("</form>");
+    out.print("<hr>");
+    out.print("<a name=\"blockDetails\"></a>");
+    out.print("<B>Total number of blocks: " + blocks.size() + "</B><br>");
+    // generate a table and dump the info
+    out.println("\n<table>");
+    for (LocatedBlock cur : blocks) {
+      out.print("<tr>");
+      final String blockidstring = Long.toString(cur.getBlock().getBlockId());
+      blockSize = cur.getBlock().getNumBytes();
+      out.print("<td>" + blockidstring + ":</td>");
+      DatanodeInfo[] locs = cur.getLocations();
+      for (int j = 0; j < locs.length; j++) {
+        String datanodeAddr = locs[j].getName();
+        datanodePort = Integer.parseInt(datanodeAddr.substring(datanodeAddr
+            .indexOf(':') + 1, datanodeAddr.length()));
+        fqdn = InetAddress.getByName(locs[j].getHost()).getCanonicalHostName();
+        String blockUrl = "http://" + fqdn + ":" + locs[j].getInfoPort()
+            + "/browseBlock.jsp?blockId=" + blockidstring
+            + "&blockSize=" + blockSize
+            + "&filename=" + URLEncoder.encode(filename, "UTF-8")
+            + "&datanodePort=" + datanodePort
+            + "&genstamp=" + cur.getBlock().getGenerationStamp()
+            + "&namenodeInfoPort=" + namenodeInfoPort
+            + "&chunkSizeToView=" + chunkSizeToView;
+        out.print("<td>&nbsp</td><td><a href=\"" + blockUrl + "\">"
+            + datanodeAddr + "</a></td>");
+      }
+      out.println("</tr>");
+    }
+    out.println("</table>");
+    out.print("<hr>");
+    String namenodeHost = datanode.getNameNodeAddr().getHostName();
+    out.print("<br><a href=\"http://"
+        + InetAddress.getByName(namenodeHost).getCanonicalHostName() + ":"
+        + namenodeInfoPort + "/dfshealth.jsp\">Go back to DFS home</a>");
+    dfs.close();
+  }
+
+  static void generateFileChunks(JspWriter out, HttpServletRequest req)
+      throws IOException {
+    long startOffset = 0;
+    int datanodePort = 0;
+
+    String namenodeInfoPortStr = req.getParameter("namenodeInfoPort");
+    int namenodeInfoPort = -1;
+    if (namenodeInfoPortStr != null)
+      namenodeInfoPort = Integer.parseInt(namenodeInfoPortStr);
+
+    final String filename = JspHelper
+        .validatePath(req.getParameter("filename"));
+    if (filename == null) {
+      out.print("Invalid input (filename absent)");
+      return;
+    }
+
+    final Long blockId = JspHelper.validateLong(req.getParameter("blockId"));
+    if (blockId == null) {
+      out.print("Invalid input (blockId absent)");
+      return;
+    }
+
+    final DFSClient dfs = new DFSClient(datanode.getNameNodeAddr(),
+        JspHelper.conf);
+
+    AccessToken accessToken = AccessToken.DUMMY_TOKEN;
+    if (JspHelper.conf.getBoolean(
+        AccessTokenHandler.STRING_ENABLE_ACCESS_TOKEN, false)) {
+      List<LocatedBlock> blks = dfs.namenode.getBlockLocations(filename, 0,
+          Long.MAX_VALUE).getLocatedBlocks();
+      if (blks == null || blks.size() == 0) {
+        out.print("Can't locate file blocks");
+        dfs.close();
+        return;
+      }
+      for (int i = 0; i < blks.size(); i++) {
+        if (blks.get(i).getBlock().getBlockId() == blockId) {
+          accessToken = blks.get(i).getAccessToken();
+          break;
+        }
+      }
+    }
+
+    final Long genStamp = JspHelper.validateLong(req.getParameter("genstamp"));
+    if (genStamp == null) {
+      out.print("Invalid input (genstamp absent)");
+      return;
+    }
+
+    String blockSizeStr;
+    long blockSize = 0;
+    blockSizeStr = req.getParameter("blockSize");
+    if (blockSizeStr == null) {
+      out.print("Invalid input (blockSize absent)");
+      return;
+    }
+    blockSize = Long.parseLong(blockSizeStr);
+
+    final int chunkSizeToView = JspHelper.string2ChunkSizeToView(req
+        .getParameter("chunkSizeToView"));
+
+    String startOffsetStr = req.getParameter("startOffset");
+    if (startOffsetStr == null || Long.parseLong(startOffsetStr) < 0)
+      startOffset = 0;
+    else
+      startOffset = Long.parseLong(startOffsetStr);
+
+    String datanodePortStr = req.getParameter("datanodePort");
+    if (datanodePortStr == null) {
+      out.print("Invalid input (datanodePort absent)");
+      return;
+    }
+    datanodePort = Integer.parseInt(datanodePortStr);
+    out.print("<h3>File: ");
+    JspHelper.printPathWithLinks(filename, out, namenodeInfoPort);
+    out.print("</h3><hr>");
+    String parent = new File(filename).getParent();
+    JspHelper.printGotoForm(out, namenodeInfoPort, parent);
+    out.print("<hr>");
+    out.print("<a href=\"http://"
+        + req.getServerName() + ":" + req.getServerPort()
+        + "/browseDirectory.jsp?dir=" + URLEncoder.encode(parent, "UTF-8")
+        + "&namenodeInfoPort=" + namenodeInfoPort
+        + "\"><i>Go back to dir listing</i></a><br>");
+    out.print("<a href=\"#viewOptions\">Advanced view/download options</a><br>");
+    out.print("<hr>");
+
+    // Determine the prev & next blocks
+    long nextStartOffset = 0;
+    long nextBlockSize = 0;
+    String nextBlockIdStr = null;
+    String nextGenStamp = null;
+    String nextHost = req.getServerName();
+    int nextPort = req.getServerPort();
+    int nextDatanodePort = datanodePort;
+    // determine data for the next link
+    if (startOffset + chunkSizeToView >= blockSize) {
+      // we have to go to the next block from this point onwards
+      List<LocatedBlock> blocks = dfs.namenode.getBlockLocations(filename, 0,
+          Long.MAX_VALUE).getLocatedBlocks();
+      for (int i = 0; i < blocks.size(); i++) {
+        if (blocks.get(i).getBlock().getBlockId() == blockId) {
+          if (i != blocks.size() - 1) {
+            LocatedBlock nextBlock = blocks.get(i + 1);
+            nextBlockIdStr = Long.toString(nextBlock.getBlock().getBlockId());
+            nextGenStamp = Long.toString(nextBlock.getBlock()
+                .getGenerationStamp());
+            nextStartOffset = 0;
+            nextBlockSize = nextBlock.getBlock().getNumBytes();
+            DatanodeInfo d = JspHelper.bestNode(nextBlock);
+            String datanodeAddr = d.getName();
+            nextDatanodePort = Integer.parseInt(datanodeAddr.substring(
+                datanodeAddr.indexOf(':') + 1, datanodeAddr.length()));
+            nextHost = InetAddress.getByName(d.getHost())
+                .getCanonicalHostName();
+            nextPort = d.getInfoPort();
+          }
+        }
+      }
+    } else {
+      // we are in the same block
+      nextBlockIdStr = blockId.toString();
+      nextStartOffset = startOffset + chunkSizeToView;
+      nextBlockSize = blockSize;
+      nextGenStamp = genStamp.toString();
+    }
+    String nextUrl = null;
+    if (nextBlockIdStr != null) {
+      nextUrl = "http://" + nextHost + ":" + nextPort
+          + "/browseBlock.jsp?blockId=" + nextBlockIdStr
+          + "&blockSize=" + nextBlockSize
+          + "&startOffset=" + nextStartOffset
+          + "&genstamp=" + nextGenStamp
+          + "&filename=" + URLEncoder.encode(filename, "UTF-8")
+          + "&chunkSizeToView=" + chunkSizeToView
+          + "&datanodePort=" + nextDatanodePort
+          + "&namenodeInfoPort=" + namenodeInfoPort;
+      out.print("<a href=\"" + nextUrl + "\">View Next chunk</a>&nbsp;&nbsp;");
+    }
+    // determine data for the prev link
+    String prevBlockIdStr = null;
+    String prevGenStamp = null;
+    long prevStartOffset = 0;
+    long prevBlockSize = 0;
+    String prevHost = req.getServerName();
+    int prevPort = req.getServerPort();
+    int prevDatanodePort = datanodePort;
+    if (startOffset == 0) {
+      List<LocatedBlock> blocks = dfs.namenode.getBlockLocations(filename, 0,
+          Long.MAX_VALUE).getLocatedBlocks();
+      for (int i = 0; i < blocks.size(); i++) {
+        if (blocks.get(i).getBlock().getBlockId() == blockId) {
+          if (i != 0) {
+            LocatedBlock prevBlock = blocks.get(i - 1);
+            prevBlockIdStr = Long.toString(prevBlock.getBlock().getBlockId());
+            prevGenStamp = Long.toString(prevBlock.getBlock()
+                .getGenerationStamp());
+            prevStartOffset = prevBlock.getBlock().getNumBytes()
+                - chunkSizeToView;
+            if (prevStartOffset < 0)
+              prevStartOffset = 0;
+            prevBlockSize = prevBlock.getBlock().getNumBytes();
+            DatanodeInfo d = JspHelper.bestNode(prevBlock);
+            String datanodeAddr = d.getName();
+            prevDatanodePort = Integer.parseInt(datanodeAddr.substring(
+                datanodeAddr.indexOf(':') + 1, datanodeAddr.length()));
+            prevHost = InetAddress.getByName(d.getHost())
+                .getCanonicalHostName();
+            prevPort = d.getInfoPort();
+          }
+        }
+      }
+    } else {
+      // we are in the same block
+      prevBlockIdStr = blockId.toString();
+      prevStartOffset = startOffset - chunkSizeToView;
+      if (prevStartOffset < 0)
+        prevStartOffset = 0;
+      prevBlockSize = blockSize;
+      prevGenStamp = genStamp.toString();
+    }
+
+    String prevUrl = null;
+    if (prevBlockIdStr != null) {
+      prevUrl = "http://" + prevHost + ":" + prevPort
+          + "/browseBlock.jsp?blockId=" + prevBlockIdStr
+          + "&blockSize=" + prevBlockSize
+          + "&startOffset=" + prevStartOffset
+          + "&filename=" + URLEncoder.encode(filename, "UTF-8")
+          + "&chunkSizeToView=" + chunkSizeToView
+          + "&genstamp=" + prevGenStamp
+          + "&datanodePort=" + prevDatanodePort
+          + "&namenodeInfoPort=" + namenodeInfoPort;
+      out.print("<a href=\"" + prevUrl + "\">View Prev chunk</a>&nbsp;&nbsp;");
+    }
+    out.print("<hr>");
+    out.print("<textarea cols=\"100\" rows=\"25\" wrap=\"virtual\" style=\"width:100%\" READONLY>");
+    try {
+      JspHelper.streamBlockInAscii(new InetSocketAddress(req.getServerName(),
+          datanodePort), blockId, accessToken, genStamp, blockSize,
+          startOffset, chunkSizeToView, out);
+    } catch (Exception e) {
+      out.print(e);
+    }
+    out.print("</textarea>");
+    dfs.close();
+  }
+
+  static void generateFileChunksForTail(JspWriter out, HttpServletRequest req)
+      throws IOException {
+    final String referrer = JspHelper.validateURL(req.getParameter("referrer"));
+    boolean noLink = false;
+    if (referrer == null) {
+      noLink = true;
+    }
+
+    final String filename = JspHelper
+        .validatePath(req.getParameter("filename"));
+    if (filename == null) {
+      out.print("Invalid input (file name absent)");
+      return;
+    }
+
+    String namenodeInfoPortStr = req.getParameter("namenodeInfoPort");
+    int namenodeInfoPort = -1;
+    if (namenodeInfoPortStr != null)
+      namenodeInfoPort = Integer.parseInt(namenodeInfoPortStr);
+
+    final int chunkSizeToView = JspHelper.string2ChunkSizeToView(req
+        .getParameter("chunkSizeToView"));
+
+    if (!noLink) {
+      out.print("<h3>Tail of File: ");
+      JspHelper.printPathWithLinks(filename, out, namenodeInfoPort);
+      out.print("</h3><hr>");
+      out.print("<a href=\"" + referrer + "\">Go Back to File View</a><hr>");
+    } else {
+      out.print("<h3>" + filename + "</h3>");
+    }
+    out.print("<b>Chunk size to view (in bytes, up to file's DFS block size): </b>");
+    out.print("<input type=\"text\" name=\"chunkSizeToView\" value="
+        + chunkSizeToView + " size=10 maxlength=10>");
+    out.print("&nbsp;&nbsp;<input type=\"submit\" name=\"submit\" value=\"Refresh\"><hr>");
+    out.print("<input type=\"hidden\" name=\"filename\" value=\"" + filename
+        + "\">");
+    out.print("<input type=\"hidden\" name=\"namenodeInfoPort\" value=\""
+        + namenodeInfoPort + "\">");
+    if (!noLink)
+      out.print("<input type=\"hidden\" name=\"referrer\" value=\"" + referrer
+          + "\">");
+
+    // fetch the block from the datanode that has the last block for this file
+    final DFSClient dfs = new DFSClient(datanode.getNameNodeAddr(),
+        JspHelper.conf);
+    List<LocatedBlock> blocks = dfs.namenode.getBlockLocations(filename, 0,
+        Long.MAX_VALUE).getLocatedBlocks();
+    if (blocks == null || blocks.size() == 0) {
+      out.print("No datanodes contain blocks of file " + filename);
+      dfs.close();
+      return;
+    }
+    LocatedBlock lastBlk = blocks.get(blocks.size() - 1);
+    long blockSize = lastBlk.getBlock().getNumBytes();
+    long blockId = lastBlk.getBlock().getBlockId();
+    AccessToken accessToken = lastBlk.getAccessToken();
+    long genStamp = lastBlk.getBlock().getGenerationStamp();
+    DatanodeInfo chosenNode;
+    try {
+      chosenNode = JspHelper.bestNode(lastBlk);
+    } catch (IOException e) {
+      out.print(e.toString());
+      dfs.close();
+      return;
+    }
+    InetSocketAddress addr = NetUtils.createSocketAddr(chosenNode.getName());
+    // view the last chunkSizeToView bytes while Tailing
+    final long startOffset = blockSize >= chunkSizeToView ? blockSize
+        - chunkSizeToView : 0;
+
+    out.print("<textarea cols=\"100\" rows=\"25\" wrap=\"virtual\" style=\"width:100%\" READONLY>");
+    JspHelper.streamBlockInAscii(addr, blockId, accessToken, genStamp,
+        blockSize, startOffset, chunkSizeToView, out);
+    out.print("</textarea>");
+    dfs.close();
+  }
+}

+ 4 - 1
src/java/org/apache/hadoop/hdfs/server/namenode/BackupNode.java

@@ -51,7 +51,7 @@ import org.apache.hadoop.util.Daemon;
  * namespace image to local disk(s).</li>
  * </ol>
  */
-class BackupNode extends NameNode {
+public class BackupNode extends NameNode {
   private static final String BN_ADDRESS_NAME_KEY = "dfs.backup.address";
   private static final String BN_ADDRESS_DEFAULT = "localhost:50100";
   private static final String BN_HTTP_ADDRESS_NAME_KEY = "dfs.backup.http.address";
@@ -90,6 +90,9 @@ class BackupNode extends NameNode {
 
   @Override // NameNode
   protected InetSocketAddress getHttpServerAddress(Configuration conf) {
+    // It is necessary to resolve the hostname at this point in order
+    // to ensure that the server address that is sent to the namenode
+    // is correct.
     assert rpcAddress != null : "rpcAddress should be calculated first";
     String addr = conf.get(BN_HTTP_ADDRESS_NAME_KEY, BN_HTTP_ADDRESS_DEFAULT);
     int port = NetUtils.createSocketAddr(addr).getPort();

+ 96 - 71
src/java/org/apache/hadoop/hdfs/server/namenode/BlockManager.java

@@ -1,3 +1,20 @@
+/**
+ * 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.namenode;
 
 import java.io.IOException;
@@ -102,9 +119,9 @@ public class BlockManager {
 
   void setConfigurationParameters(Configuration conf) throws IOException {
     this.replicator = new ReplicationTargetChooser(
-                          conf.getBoolean("dfs.replication.considerLoad", true),
-                          namesystem,
-                          namesystem.clusterMap);
+                         conf.getBoolean("dfs.replication.considerLoad", true),
+                         namesystem,
+                         namesystem.clusterMap);
 
     this.defaultReplication = conf.getInt("dfs.replication", 3);
     this.maxReplication = conf.getInt("dfs.replication.max", 512);
@@ -144,8 +161,8 @@ public class BlockManager {
     // Dump contents of neededReplication
     //
     synchronized (neededReplications) {
-      out.println("Metasave: Blocks waiting for replication: "
-          + neededReplications.size());
+      out.println("Metasave: Blocks waiting for replication: " + 
+                  neededReplications.size());
       for (Block block : neededReplications) {
         List<DatanodeDescriptor> containingNodes =
                                           new ArrayList<DatanodeDescriptor>();
@@ -201,8 +218,7 @@ public class BlockManager {
   }
 
   /**
-   * Get all valid locations of the block & add the block to results
-   * return the length of the added block; 0 if the block is not added
+   * Get all valid locations of the block
    */
   ArrayList<String> addBlock(Block block) {
     ArrayList<String> machineSet =
@@ -251,16 +267,16 @@ public class BlockManager {
             + " but corrupt replicas map has " + numCorruptReplicas);
       }
       boolean blockCorrupt = (numCorruptNodes == numNodes);
-      int numMachineSet = blockCorrupt ? numNodes
-          : (numNodes - numCorruptNodes);
+      int numMachineSet = blockCorrupt ? numNodes :
+                          (numNodes - numCorruptNodes);
       DatanodeDescriptor[] machineSet = new DatanodeDescriptor[numMachineSet];
       if (numMachineSet > 0) {
         numNodes = 0;
-        for (Iterator<DatanodeDescriptor> it = blocksMap
-            .nodeIterator(blocks[curBlk]); it.hasNext();) {
+        for (Iterator<DatanodeDescriptor> it = 
+             blocksMap.nodeIterator(blocks[curBlk]); it.hasNext();) {
           DatanodeDescriptor dn = it.next();
-          boolean replicaCorrupt = corruptReplicas.isReplicaCorrupt(
-              blocks[curBlk], dn);
+          boolean replicaCorrupt = 
+            corruptReplicas.isReplicaCorrupt(blocks[curBlk], dn);
           if (blockCorrupt || (!blockCorrupt && !replicaCorrupt))
             machineSet[numNodes++] = dn;
         }
@@ -274,8 +290,9 @@ public class BlockManager {
       results.add(b);
       curPos += blocks[curBlk].getNumBytes();
       curBlk++;
-    } while (curPos < endOff && curBlk < blocks.length
-        && results.size() < nrBlocksToReturn);
+    } while (curPos < endOff 
+          && curBlk < blocks.length
+          && results.size() < nrBlocksToReturn);
     return results;
   }
 
@@ -291,9 +308,11 @@ public class BlockManager {
       //common case. avoid building 'text'
       return;
     }
-
-    String text = "file " + src + ((clientName != null) ? " on client "
-      + clientName : "") + ".\n" + "Requested replication " + replication;
+    
+    String text = "file " + src 
+      + ((clientName != null) ? " on client " + clientName : "")
+      + ".\n"
+      + "Requested replication " + replication;
 
     if (replication > maxReplication)
       throw new IOException(text + " exceeds maximum " + maxReplication);
@@ -362,9 +381,9 @@ public class BlockManager {
   void markBlockAsCorrupt(Block blk, DatanodeInfo dn) throws IOException {
     DatanodeDescriptor node = namesystem.getDatanode(dn);
     if (node == null) {
-      throw new IOException("Cannot mark block" + blk.getBlockName()
-          + " as corrupt because datanode " + dn.getName()
-          + " does not exist. ");
+      throw new IOException("Cannot mark block" + blk.getBlockName() +
+                            " as corrupt because datanode " + dn.getName() +
+                            " does not exist. ");
     }
 
     final BlockInfo storedBlockInfo = blocksMap.getStoredBlock(blk);
@@ -373,18 +392,20 @@ public class BlockManager {
       // ignore the request for now. This could happen when BlockScanner
       // thread of Datanode reports bad block before Block reports are sent
       // by the Datanode on startup
-      NameNode.stateChangeLog.info("BLOCK NameSystem.markBlockAsCorrupt: "
-          + "block " + blk + " could not be marked "
-          + "as corrupt as it does not exists in " + "blocksMap");
+      NameNode.stateChangeLog.info("BLOCK NameSystem.markBlockAsCorrupt: " +
+                                   "block " + blk + " could not be marked " +
+                                   "as corrupt as it does not exists in " +
+                                   "blocksMap");
     } else {
       INodeFile inode = storedBlockInfo.getINode();
       if (inode == null) {
-        NameNode.stateChangeLog.info("BLOCK NameSystem.markBlockAsCorrupt: "
-            + "block " + blk + " could not be marked "
-            + "as corrupt as it does not belong to " + "any file");
+        NameNode.stateChangeLog.info("BLOCK NameSystem.markBlockAsCorrupt: " +
+                                     "block " + blk + " could not be marked " +
+                                     "as corrupt as it does not belong to " +
+                                     "any file");
         addToInvalidates(storedBlockInfo, node);
         return;
-      }
+      } 
       // Add this replica to corruptReplicas Map
       corruptReplicas.addToCorruptReplicasMap(storedBlockInfo, node);
       if (countNodes(storedBlockInfo).liveReplicas() > inode.getReplication()) {
@@ -402,12 +423,13 @@ public class BlockManager {
    */
   private void invalidateBlock(Block blk, DatanodeInfo dn)
       throws IOException {
-    NameNode.stateChangeLog.info("DIR* NameSystem.invalidateBlock: " + blk
-        + " on " + dn.getName());
+    NameNode.stateChangeLog.info("DIR* NameSystem.invalidateBlock: "
+                                 + blk + " on " + dn.getName());
     DatanodeDescriptor node = namesystem.getDatanode(dn);
     if (node == null) {
-      throw new IOException("Cannot invalidate block " + blk
-          + " because datanode " + dn.getName() + " does not exist.");
+      throw new IOException("Cannot invalidate block " + blk +
+                            " because datanode " + dn.getName() +
+                            " does not exist.");
     }
 
     // Check how many copies we have of the block. If we have at least one
@@ -417,7 +439,8 @@ public class BlockManager {
       addToInvalidates(blk, dn);
       removeStoredBlock(blk, node);
       NameNode.stateChangeLog.debug("BLOCK* NameSystem.invalidateBlocks: "
-          + blk + " on " + dn.getName() + " listed for deletion.");
+                                   + blk + " on "
+                                   + dn.getName() + " listed for deletion.");
     } else {
       NameNode.stateChangeLog.info("BLOCK* NameSystem.invalidateBlocks: "
           + blk + " on " + dn.getName()
@@ -799,9 +822,8 @@ public class BlockManager {
   }
 
   /**
-   * Modify (block-->datanode) map. Remove block from set of needed replications
-   * if this takes care of the problem.
-   *
+   * Modify (block-->datanode) map. Remove block from set of
+   * needed replications if this takes care of the problem.
    * @return the block that is stored in blockMap.
    */
   private Block addStoredBlock(Block block, DatanodeDescriptor node,
@@ -810,9 +832,10 @@ public class BlockManager {
     if (storedBlock == null || storedBlock.getINode() == null) {
       // If this block does not belong to anyfile, then we are done.
       NameNode.stateChangeLog.info("BLOCK* NameSystem.addStoredBlock: "
-          + "addStoredBlock request received for " + block + " on "
-          + node.getName() + " size " + block.getNumBytes()
-          + " But it does not belong to any file.");
+                                   + "addStoredBlock request received for "
+                                   + block + " on " + node.getName()
+                                   + " size " + block.getNumBytes()
+                                   + " But it does not belong to any file.");
       // we could add this block to invalidate set of this datanode.
       // it will happen in next block report otherwise.
       return block;
@@ -829,9 +852,10 @@ public class BlockManager {
         if (cursize == 0) {
           storedBlock.setNumBytes(block.getNumBytes());
         } else if (cursize != block.getNumBytes()) {
-          FSNamesystem.LOG.warn("Inconsistent size for block " + block
-              + " reported from " + node.getName() + " current size is "
-              + cursize + " reported size is " + block.getNumBytes());
+          FSNamesystem.LOG.warn("Inconsistent size for block " + block +
+                   " reported from " + node.getName() +
+                   " current size is " + cursize +
+                   " reported size is " + block.getNumBytes());
           try {
             if (cursize > block.getNumBytes()) {
               // new replica is smaller in size than existing block.
@@ -847,7 +871,7 @@ public class BlockManager {
               int count = 0;
               DatanodeDescriptor nodes[] = new DatanodeDescriptor[numNodes];
               Iterator<DatanodeDescriptor> it = blocksMap.nodeIterator(block);
-              for (; it != null && it.hasNext();) {
+              for (; it != null && it.hasNext(); ) {
                 DatanodeDescriptor dd = it.next();
                 if (!dd.equals(node)) {
                   nodes[count++] = dd;
@@ -878,11 +902,11 @@ public class BlockManager {
 
         // Updated space consumed if required.
         INodeFile file = (storedBlock != null) ? storedBlock.getINode() : null;
-        long diff = (file == null) ? 0
-            : (file.getPreferredBlockSize() - storedBlock.getNumBytes());
-
-        if (diff > 0 && file.isUnderConstruction()
-            && cursize < storedBlock.getNumBytes()) {
+        long diff = (file == null) ? 0 :
+                    (file.getPreferredBlockSize() - storedBlock.getNumBytes());
+        
+        if (diff > 0 && file.isUnderConstruction() &&
+            cursize < storedBlock.getNumBytes()) {
           try {
             String path = /* For finding parents */
             namesystem.leaseManager.findPath((INodeFileUnderConstruction) file);
@@ -923,7 +947,7 @@ public class BlockManager {
     NumberReplicas num = countNodes(storedBlock);
     int numLiveReplicas = num.liveReplicas();
     int numCurrentReplica = numLiveReplicas
-        + pendingReplications.getNumReplicas(block);
+      + pendingReplications.getNumReplicas(block);
 
     // check whether safe replication is reached for the block
     namesystem.incrementSafeBlockCount(numCurrentReplica);
@@ -958,9 +982,9 @@ public class BlockManager {
     int corruptReplicasCount = corruptReplicas.numCorruptReplicas(block);
     int numCorruptNodes = num.corruptReplicas();
     if (numCorruptNodes != corruptReplicasCount) {
-      FSNamesystem.LOG.warn("Inconsistent number of corrupt replicas for "
-          + block + "blockMap has " + numCorruptNodes
-          + " but corrupt replicas map has " + corruptReplicasCount);
+      FSNamesystem.LOG.warn("Inconsistent number of corrupt replicas for " +
+          block + "blockMap has " + numCorruptNodes + 
+          " but corrupt replicas map has " + corruptReplicasCount);
     }
     if ((corruptReplicasCount > 0) && (numLiveReplicas >= fileReplication))
       invalidateCorruptReplicas(block);
@@ -970,28 +994,29 @@ public class BlockManager {
   /**
    * Invalidate corrupt replicas.
    * <p>
-   * This will remove the replicas from the block's location list, add them to
-   * {@link #recentInvalidateSets} so that they could be further deleted from
-   * the respective data-nodes, and remove the block from corruptReplicasMap.
+   * This will remove the replicas from the block's location list,
+   * add them to {@link #recentInvalidateSets} so that they could be further
+   * deleted from the respective data-nodes,
+   * and remove the block from corruptReplicasMap.
    * <p>
-   * This method should be called when the block has sufficient number of live
-   * replicas.
+   * This method should be called when the block has sufficient
+   * number of live replicas.
    *
-   * @param blk
-   *          Block whose corrupt replicas need to be invalidated
+   * @param blk Block whose corrupt replicas need to be invalidated
    */
   private void invalidateCorruptReplicas(Block blk) {
     Collection<DatanodeDescriptor> nodes = corruptReplicas.getNodes(blk);
     boolean gotException = false;
     if (nodes == null)
       return;
-    for (Iterator<DatanodeDescriptor> it = nodes.iterator(); it.hasNext();) {
+    for (Iterator<DatanodeDescriptor> it = nodes.iterator(); it.hasNext(); ) {
       DatanodeDescriptor node = it.next();
       try {
         invalidateBlock(blk, node);
       } catch (IOException e) {
-        NameNode.stateChangeLog.info("NameNode.invalidateCorruptReplicas "
-            + "error in deleting bad block " + blk + " on " + node + e);
+        NameNode.stateChangeLog.info("NameNode.invalidateCorruptReplicas " +
+                                      "error in deleting bad block " + blk +
+                                      " on " + node + e);
         gotException = true;
       }
     }
@@ -1040,9 +1065,9 @@ public class BlockManager {
   }
 
   /**
-   * Find how many of the containing nodes are "extra", if any. If there are any
-   * extras, call chooseExcessReplicates() to mark them in the
-   * excessReplicateMap.
+   * Find how many of the containing nodes are "extra", if any.
+   * If there are any extras, call chooseExcessReplicates() to
+   * mark them in the excessReplicateMap.
    */
   void processOverReplicatedBlock(Block block, short replication,
       DatanodeDescriptor addedNode, DatanodeDescriptor delNodeHint) {
@@ -1052,8 +1077,8 @@ public class BlockManager {
     Collection<DatanodeDescriptor> nonExcess = new ArrayList<DatanodeDescriptor>();
     Collection<DatanodeDescriptor> corruptNodes = corruptReplicas
         .getNodes(block);
-    for (Iterator<DatanodeDescriptor> it = blocksMap.nodeIterator(block); it
-        .hasNext();) {
+    for (Iterator<DatanodeDescriptor> it = blocksMap.nodeIterator(block);
+         it.hasNext();) {
       DatanodeDescriptor cur = it.next();
       Collection<Block> excessBlocks = excessReplicateMap.get(cur
           .getStorageID());
@@ -1066,8 +1091,8 @@ public class BlockManager {
         }
       }
     }
-    namesystem.chooseExcessReplicates(nonExcess, block, replication, addedNode,
-        delNodeHint);
+    namesystem.chooseExcessReplicates(nonExcess, block, replication, 
+        addedNode, delNodeHint);
   }
 
   void addToExcessReplicate(DatanodeInfo dn, Block block) {
@@ -1171,8 +1196,8 @@ public class BlockManager {
       } else if (node.isDecommissionInProgress() || node.isDecommissioned()) {
         count++;
       } else {
-        Collection<Block> blocksExcess = excessReplicateMap.get(node
-            .getStorageID());
+        Collection<Block> blocksExcess =
+          excessReplicateMap.get(node.getStorageID());
         if (blocksExcess != null && blocksExcess.contains(b)) {
           excess++;
         } else {

+ 20 - 4
src/java/org/apache/hadoop/hdfs/server/namenode/BlocksMap.java

@@ -17,7 +17,11 @@
  */
 package org.apache.hadoop.hdfs.server.namenode;
 
-import java.util.*;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
 
 import org.apache.hadoop.hdfs.protocol.Block;
 
@@ -32,11 +36,11 @@ class BlocksMap {
    * Internal class for block metadata.
    */
   static class BlockInfo extends Block {
-    private INodeFile          inode;
+    private INodeFile inode;
 
     /**
      * This array contains triplets of references.
-     * For each i-th data-node the block belongs to
+     * For each i-th datanode the block belongs to
      * triplets[3*i] is the reference to the DatanodeDescriptor
      * and triplets[3*i+1] and triplets[3*i+2] are references 
      * to the previous and the next blocks, respectively, in the 
@@ -217,7 +221,7 @@ class BlocksMap {
      * If this block is the head of the list then return the next block as 
      * the new head.
      * @return the new head of the list or null if the list becomes
-     * empy after deletion.
+     * empty after deletion.
      */
     BlockInfo listRemove(BlockInfo head, DatanodeDescriptor dn) {
       if(head == null)
@@ -266,6 +270,18 @@ class BlocksMap {
       }
       return true;
     }
+
+    @Override
+    public int hashCode() {
+      // Super implementation is sufficient
+      return super.hashCode();
+    }
+    
+    @Override
+    public boolean equals(Object obj) {
+      // Sufficient to rely on super's implementation
+      return (this == obj) || super.equals(obj);
+    }
   }
 
   private static class NodeIterator implements Iterator<DatanodeDescriptor> {

+ 20 - 7
src/java/org/apache/hadoop/hdfs/server/namenode/DatanodeDescriptor.java

@@ -29,7 +29,7 @@ import org.apache.hadoop.hdfs.server.namenode.BlocksMap.BlockInfo;
 import org.apache.hadoop.hdfs.server.protocol.BlockCommand;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
 import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.DeprecatedUTF8;
+import org.apache.hadoop.hdfs.DeprecatedUTF8;
 import org.apache.hadoop.io.WritableUtils;
 
 /**************************************************
@@ -94,8 +94,8 @@ public class DatanodeDescriptor extends DatanodeInfo {
   /** A set of blocks to be invalidated by this datanode */
   private Set<Block> invalidateBlocks = new TreeSet<Block>();
 
-  /* Variables for maintaning number of blocks scheduled to be written to
-   * this datanode. This count is approximate and might be slightly higger
+  /* Variables for maintaining number of blocks scheduled to be written to
+   * this datanode. This count is approximate and might be slightly bigger
    * in case of errors (e.g. datanode does not report if an error occurs 
    * while writing the block).
    */
@@ -159,7 +159,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
    * @param networkLocation location of the data node in network
    * @param capacity capacity of the data node, including space used by non-dfs
    * @param dfsUsed the used space by dfs datanode
-   * @param remaining remaing capacity of the data node
+   * @param remaining remaining capacity of the data node
    * @param xceiverCount # of data transfers at the data node
    */
   public DatanodeDescriptor(DatanodeID nodeID,
@@ -174,7 +174,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
   }
 
   /**
-   * Add data-node to the block.
+   * Add datanode to the block.
    * Add block to the head of the list of blocks belonging to the data-node.
    */
   boolean addBlock(BlockInfo b) {
@@ -187,7 +187,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
   
   /**
    * Remove block from the list of blocks belonging to the data-node.
-   * Remove data-node from the block.
+   * Remove datanode from the block.
    */
   boolean removeBlock(BlockInfo b) {
     blockList = b.listRemove(blockList, this);
@@ -228,7 +228,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
   }
 
   /**
-   * Iterates over the list of blocks belonging to the data-node.
+   * Iterates over the list of blocks belonging to the datanode.
    */
   static private class BlockIterator implements Iterator<Block> {
     private BlockInfo current;
@@ -463,4 +463,17 @@ public class DatanodeDescriptor extends DatanodeInfo {
       lastBlocksScheduledRollTime = now;
     }
   }
+  
+  @Override
+  public int hashCode() {
+    // Super implementation is sufficient
+    return super.hashCode();
+  }
+  
+  @Override
+  public boolean equals(Object obj) {
+    // Sufficient to use super equality as datanodes are uniquely identified
+    // by DatanodeID
+    return (this == obj) || super.equals(obj);
+  }
 }

+ 1 - 0
src/java/org/apache/hadoop/hdfs/server/namenode/DfsServlet.java

@@ -34,6 +34,7 @@ import org.apache.hadoop.hdfs.DFSClient;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.server.common.JspHelper;
 import org.apache.hadoop.security.UnixUserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation;
 

+ 4 - 5
src/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java

@@ -43,7 +43,7 @@ import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration;
 import org.apache.hadoop.io.ArrayWritable;
 import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.io.DeprecatedUTF8;
+import org.apache.hadoop.hdfs.DeprecatedUTF8;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableFactories;
 import org.apache.hadoop.io.WritableFactory;
@@ -262,13 +262,12 @@ public class FSEditLog {
       ArrayList<EditLogOutputStream> errorStreams,
       boolean propagate) {
     
-    String lsd = fsimage.listStorageDirectories();
-    FSNamesystem.LOG.info("current list of storage dirs:" + lsd);
-    
     if (errorStreams == null || errorStreams.size() == 0) {
       return;                       // nothing to do
     }
 
+    String lsd = fsimage.listStorageDirectories();
+    FSNamesystem.LOG.info("current list of storage dirs:" + lsd);
     //EditLogOutputStream
     if (editStreams == null || editStreams.size() <= 1) {
       FSNamesystem.LOG.fatal(
@@ -853,7 +852,7 @@ public class FSEditLog {
     if (lastPrintTime + 60000 > now && !force) {
       return;
     }
-    if (editStreams == null) {
+    if (editStreams == null || editStreams.size()==0) {
       return;
     }
     lastPrintTime = now;

+ 1 - 1
src/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java

@@ -61,7 +61,7 @@ import org.apache.hadoop.hdfs.server.protocol.CheckpointCommand;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeCommand;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration;
-import org.apache.hadoop.io.DeprecatedUTF8;
+import org.apache.hadoop.hdfs.DeprecatedUTF8;
 import org.apache.hadoop.io.Writable;
 
 /**

+ 20 - 40
src/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

@@ -54,6 +54,7 @@ import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
 import org.apache.hadoop.hdfs.server.protocol.UpgradeCommand;
 import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations.BlockWithLocations;
 import org.apache.hadoop.fs.ContentSummary;
+import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.*;
@@ -815,10 +816,10 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean {
    */
   void startFile(String src, PermissionStatus permissions,
                  String holder, String clientMachine,
-                 boolean overwrite, short replication, long blockSize
+                 EnumSet<CreateFlag> flag, short replication, long blockSize
                 ) throws IOException {
-    startFileInternal(src, permissions, holder, clientMachine, overwrite, false,
-                      replication, blockSize);
+    startFileInternal(src, permissions, holder, clientMachine, flag,
+        replication, blockSize);
     getEditLog().logSync();
     if (auditLog.isInfoEnabled()) {
       final FileStatus stat = dir.getFileInfo(src);
@@ -832,11 +833,14 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean {
                                               PermissionStatus permissions,
                                               String holder, 
                                               String clientMachine, 
-                                              boolean overwrite,
-                                              boolean append,
+                                              EnumSet<CreateFlag> flag,
                                               short replication,
                                               long blockSize
                                               ) throws IOException {
+    boolean overwrite = flag.contains(CreateFlag.OVERWRITE);
+    boolean append = flag.contains(CreateFlag.APPEND);
+    boolean create = flag.contains(CreateFlag.CREATE);
+
     if (NameNode.stateChangeLog.isDebugEnabled()) {
       NameNode.stateChangeLog.debug("DIR* NameSystem.startFile: src=" + src
           + ", holder=" + holder
@@ -918,8 +922,15 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean {
       }
       if (append) {
         if (myFile == null) {
-          throw new FileNotFoundException("failed to append to non-existent file "
+          if(!create)
+            throw new FileNotFoundException("failed to append to non-existent file "
               + src + " on client " + clientMachine);
+          else {
+            //append & create a nonexist file equals to overwrite
+            this.startFileInternal(src, permissions, holder, clientMachine,
+                EnumSet.of(CreateFlag.OVERWRITE), replication, blockSize);
+            return;
+          }
         } else if (myFile.isDirectory()) {
           throw new IOException("failed to append to directory " + src 
                                 +" on client " + clientMachine);
@@ -992,7 +1003,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean {
       throw new IOException("Append to hdfs not supported." +
                             " Please refer to dfs.support.append configuration parameter.");
     }
-    startFileInternal(src, null, holder, clientMachine, false, true, 
+    startFileInternal(src, null, holder, clientMachine, EnumSet.of(CreateFlag.APPEND), 
                       (short)blockManager.maxReplication, (long)0);
     getEditLog().logSync();
 
@@ -2842,43 +2853,12 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean {
     }
     return node;
   }
-    
-  /** Stop at and return the datanode at index (used for content browsing)*/
-  @Deprecated
-  private DatanodeDescriptor getDatanodeByIndex(int index) {
-    int i = 0;
-    for (DatanodeDescriptor node : datanodeMap.values()) {
-      if (i == index) {
-        return node;
-      }
-      i++;
-    }
-    return null;
-  }
-    
-  @Deprecated
-  public String randomDataNode() {
-    int size = datanodeMap.size();
-    int index = 0;
-    if (size != 0) {
-      index = r.nextInt(size);
-      for(int i=0; i<size; i++) {
-        DatanodeDescriptor d = getDatanodeByIndex(index);
-        if (d != null && !d.isDecommissioned() && !isDatanodeDead(d) &&
-            !d.isDecommissionInProgress()) {
-          return d.getHost() + ":" + d.getInfoPort();
-        }
-        index = (index + 1) % size;
-      }
-    }
-    return null;
-  }
 
   /** Choose a random datanode
    * 
    * @return a randomly chosen datanode
    */
-  public DatanodeDescriptor getRandomDatanode() {
+  DatanodeDescriptor getRandomDatanode() {
     return (DatanodeDescriptor)clusterMap.chooseRandom(NodeBase.ROOT);
   }
 
@@ -3131,7 +3111,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean {
           return leaveMsg + " upon completion of " + 
             "the distributed upgrade: upgrade progress = " + 
             getDistributedUpgradeStatus() + "%";
-        leaveMsg = "Use \"hadoop dfs -safemode leave\" to turn safe mode off";
+        leaveMsg = "Use \"hdfs dfsadmin -safemode leave\" to turn safe mode off";
       }
       if(blockTotal < 0)
         return leaveMsg + ".";

+ 1 - 0
src/java/org/apache/hadoop/hdfs/server/namenode/FileDataServlet.java

@@ -28,6 +28,7 @@ import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
+import org.apache.hadoop.hdfs.server.common.JspHelper;
 import org.apache.hadoop.security.UnixUserGroupInformation;
 
 /** Redirect queries about the hosted filesystem to an appropriate datanode.

+ 2 - 1
src/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectory.java

@@ -160,7 +160,8 @@ class INodeDirectory extends INode {
    */
   int getExistingPathINodes(byte[][] components, INode[] existing) {
     assert compareBytes(this.name, components[0]) == 0 :
-      "Incorrect name " + getLocalName() + " expected " + components[0];
+      "Incorrect name " + getLocalName() + " expected " + 
+      bytes2String(components[0]);
 
     INode curNode = this;
     int count = 0;

+ 21 - 4
src/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java

@@ -29,6 +29,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.ContentSummary;
+import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -62,6 +63,7 @@ import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
 import org.apache.hadoop.hdfs.server.protocol.NodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.UpgradeCommand;
 import org.apache.hadoop.http.HttpServer;
+import org.apache.hadoop.io.EnumSetWritable;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ipc.Server;
 import org.apache.hadoop.net.NetUtils;
@@ -189,13 +191,28 @@ public class NameNode implements ClientProtocol, DatanodeProtocol,
   }
 
   public static InetSocketAddress getAddress(Configuration conf) {
-    return getAddress(FileSystem.getDefaultUri(conf).getAuthority());
+    URI filesystemURI = FileSystem.getDefaultUri(conf);
+    String authority = filesystemURI.getAuthority();
+    if (authority == null) {
+      throw new IllegalArgumentException(String.format(
+          "Invalid URI for NameNode address (check %s): %s has no authority.",
+          FileSystem.FS_DEFAULT_NAME_KEY, filesystemURI.toString()));
+    }
+    if (!FSConstants.HDFS_URI_SCHEME.equalsIgnoreCase(
+        filesystemURI.getScheme())) {
+      throw new IllegalArgumentException(String.format(
+          "Invalid URI for NameNode address (check %s): %s is not of scheme '%s'.",
+          FileSystem.FS_DEFAULT_NAME_KEY, filesystemURI.toString(),
+          FSConstants.HDFS_URI_SCHEME));
+    }
+    return getAddress(authority);
   }
 
   public static URI getUri(InetSocketAddress namenode) {
     int port = namenode.getPort();
     String portString = port == DEFAULT_PORT ? "" : (":"+port);
-    return URI.create("hdfs://"+ namenode.getHostName()+portString);
+    return URI.create(FSConstants.HDFS_URI_SCHEME + "://" 
+        + namenode.getHostName()+portString);
   }
 
   /**
@@ -532,7 +549,7 @@ public class NameNode implements ClientProtocol, DatanodeProtocol,
   public void create(String src, 
                      FsPermission masked,
                              String clientName, 
-                             boolean overwrite,
+                             EnumSetWritable<CreateFlag> flag,
                              short replication,
                              long blockSize
                              ) throws IOException {
@@ -548,7 +565,7 @@ public class NameNode implements ClientProtocol, DatanodeProtocol,
     namesystem.startFile(src,
         new PermissionStatus(UserGroupInformation.getCurrentUGI().getUserName(),
             null, masked),
-        clientName, clientMachine, overwrite, replication, blockSize);
+        clientName, clientMachine, flag.get(), replication, blockSize);
     myMetrics.numFilesCreated.inc();
     myMetrics.numCreateFileOps.inc();
   }

+ 452 - 0
src/java/org/apache/hadoop/hdfs/server/namenode/NamenodeJspHelper.java

@@ -0,0 +1,452 @@
+/**
+ * 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.namenode;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.URLEncoder;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+import javax.servlet.jsp.JspWriter;
+
+import org.apache.hadoop.hdfs.protocol.DatanodeID;
+import org.apache.hadoop.hdfs.protocol.FSConstants.UpgradeAction;
+import org.apache.hadoop.hdfs.server.common.JspHelper;
+import org.apache.hadoop.hdfs.server.common.Storage;
+import org.apache.hadoop.hdfs.server.common.UpgradeStatusReport;
+import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
+import org.apache.hadoop.util.ServletUtil;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.util.VersionInfo;
+
+class NamenodeJspHelper {
+  static String getSafeModeText(FSNamesystem fsn) {
+    if (!fsn.isInSafeMode())
+      return "";
+    return "Safe mode is ON. <em>" + fsn.getSafeModeTip() + "</em><br>";
+  }
+
+  static String getInodeLimitText(FSNamesystem fsn) {
+    long inodes = fsn.dir.totalInodes();
+    long blocks = fsn.getBlocksTotal();
+    long maxobjects = fsn.getMaxObjects();
+    long totalMemory = Runtime.getRuntime().totalMemory();
+    long maxMemory = Runtime.getRuntime().maxMemory();
+
+    long used = (totalMemory * 100) / maxMemory;
+
+    String str = inodes + " files and directories, " + blocks + " blocks = "
+        + (inodes + blocks) + " total";
+    if (maxobjects != 0) {
+      long pct = ((inodes + blocks) * 100) / maxobjects;
+      str += " / " + maxobjects + " (" + pct + "%)";
+    }
+    str += ".  Heap Size is " + StringUtils.byteDesc(totalMemory) + " / "
+        + StringUtils.byteDesc(maxMemory) + " (" + used + "%) <br>";
+    return str;
+  }
+
+  static String getUpgradeStatusText(FSNamesystem fsn) {
+    String statusText = "";
+    try {
+      UpgradeStatusReport status = fsn
+          .distributedUpgradeProgress(UpgradeAction.GET_STATUS);
+      statusText = (status == null ? "There are no upgrades in progress."
+          : status.getStatusText(false));
+    } catch (IOException e) {
+      statusText = "Upgrade status unknown.";
+    }
+    return statusText;
+  }
+
+  /** Return a table containing version information. */
+  static String getVersionTable(FSNamesystem fsn) {
+    return "<div id='dfstable'><table>"
+        + "\n  <tr><td id='col1'>Started:</td><td>" + fsn.getStartTime()
+        + "</td></tr>\n" + "\n  <tr><td id='col1'>Version:</td><td>"
+        + VersionInfo.getVersion() + ", " + VersionInfo.getRevision()
+        + "\n  <tr><td id='col1'>Compiled:</td><td>" + VersionInfo.getDate()
+        + " by " + VersionInfo.getUser() + " from " + VersionInfo.getBranch()
+        + "\n  <tr><td id='col1'>Upgrades:</td><td>"
+        + getUpgradeStatusText(fsn) + "\n</table></div>";
+  }
+
+  static String getWarningText(FSNamesystem fsn) {
+    // Ideally this should be displayed in RED
+    long missingBlocks = fsn.getMissingBlocksCount();
+    if (missingBlocks > 0) {
+      return "<br> WARNING :" + " There are about " + missingBlocks
+          + " missing blocks. Please check the log or run fsck. <br><br>";
+    }
+    return "";
+  }
+
+  static class HealthJsp {
+    private int rowNum = 0;
+    private int colNum = 0;
+    private String sorterField = null;
+    private String sorterOrder = null;
+
+    private String rowTxt() {
+      colNum = 0;
+      return "<tr class=\"" + (((rowNum++) % 2 == 0) ? "rowNormal" : "rowAlt")
+          + "\"> ";
+    }
+
+    private String colTxt() {
+      return "<td id=\"col" + ++colNum + "\"> ";
+    }
+
+    private void counterReset() {
+      colNum = 0;
+      rowNum = 0;
+    }
+
+    void generateConfReport(JspWriter out, NameNode nn,
+        HttpServletRequest request) throws IOException {
+      FSNamesystem fsn = nn.getNamesystem();
+      FSImage fsImage = fsn.getFSImage();
+      List<Storage.StorageDirectory> removedStorageDirs = fsImage
+          .getRemovedStorageDirs();
+
+      // FS Image storage configuration
+      out.print("<h3> " + nn.getRole() + " Storage: </h3>");
+      out.print("<div id=\"dfstable\"> <table border=1 cellpadding=10 cellspacing=0 title=\"NameNode Storage\">\n"
+              + "<thead><tr><td><b>Storage Directory</b></td><td><b>Type</b></td><td><b>State</b></td></tr></thead>");
+
+      StorageDirectory st = null;
+      for (Iterator<StorageDirectory> it = fsImage.dirIterator(); it.hasNext();) {
+        st = it.next();
+        String dir = "" + st.getRoot();
+        String type = "" + st.getStorageDirType();
+        out.print("<tr><td>" + dir + "</td><td>" + type
+            + "</td><td>Active</td></tr>");
+      }
+
+      long storageDirsSize = removedStorageDirs.size();
+      for (int i = 0; i < storageDirsSize; i++) {
+        st = removedStorageDirs.get(i);
+        String dir = "" + st.getRoot();
+        String type = "" + st.getStorageDirType();
+        out.print("<tr><td>" + dir + "</td><td>" + type
+            + "</td><td><font color=red>Failed</font></td></tr>");
+      }
+
+      out.print("</table></div><br>\n");
+    }
+
+    void generateHealthReport(JspWriter out, NameNode nn,
+        HttpServletRequest request) throws IOException {
+      FSNamesystem fsn = nn.getNamesystem();
+      ArrayList<DatanodeDescriptor> live = new ArrayList<DatanodeDescriptor>();
+      ArrayList<DatanodeDescriptor> dead = new ArrayList<DatanodeDescriptor>();
+      fsn.DFSNodesStatus(live, dead);
+
+      sorterField = request.getParameter("sorter/field");
+      sorterOrder = request.getParameter("sorter/order");
+      if (sorterField == null)
+        sorterField = "name";
+      if (sorterOrder == null)
+        sorterOrder = "ASC";
+
+      // Find out common suffix. Should this be before or after the sort?
+      String port_suffix = null;
+      if (live.size() > 0) {
+        String name = live.get(0).getName();
+        int idx = name.indexOf(':');
+        if (idx > 0) {
+          port_suffix = name.substring(idx);
+        }
+
+        for (int i = 1; port_suffix != null && i < live.size(); i++) {
+          if (live.get(i).getName().endsWith(port_suffix) == false) {
+            port_suffix = null;
+            break;
+          }
+        }
+      }
+
+      counterReset();
+      long[] fsnStats = fsn.getStats();
+      long total = fsnStats[0];
+      long remaining = fsnStats[2];
+      long used = fsnStats[1];
+      long nonDFS = total - remaining - used;
+      nonDFS = nonDFS < 0 ? 0 : nonDFS;
+      float percentUsed = total <= 0 ? 0f : ((float) used * 100.0f)
+          / (float) total;
+      float percentRemaining = total <= 0 ? 100f : ((float) remaining * 100.0f)
+          / (float) total;
+
+      out.print("<div id=\"dfstable\"> <table>\n" + rowTxt() + colTxt()
+          + "Configured Capacity" + colTxt() + ":" + colTxt()
+          + StringUtils.byteDesc(total) + rowTxt() + colTxt() + "DFS Used"
+          + colTxt() + ":" + colTxt() + StringUtils.byteDesc(used) + rowTxt()
+          + colTxt() + "Non DFS Used" + colTxt() + ":" + colTxt()
+          + StringUtils.byteDesc(nonDFS) + rowTxt() + colTxt()
+          + "DFS Remaining" + colTxt() + ":" + colTxt()
+          + StringUtils.byteDesc(remaining) + rowTxt() + colTxt() + "DFS Used%"
+          + colTxt() + ":" + colTxt()
+          + StringUtils.limitDecimalTo2(percentUsed) + " %" + rowTxt()
+          + colTxt() + "DFS Remaining%" + colTxt() + ":" + colTxt()
+          + StringUtils.limitDecimalTo2(percentRemaining) + " %" + rowTxt()
+          + colTxt()
+          + "<a href=\"dfsnodelist.jsp?whatNodes=LIVE\">Live Nodes</a> "
+          + colTxt() + ":" + colTxt() + live.size() + rowTxt() + colTxt()
+          + "<a href=\"dfsnodelist.jsp?whatNodes=DEAD\">Dead Nodes</a> "
+          + colTxt() + ":" + colTxt() + dead.size() + "</table></div><br>\n");
+
+      if (live.isEmpty() && dead.isEmpty()) {
+        out.print("There are no datanodes in the cluster");
+      }
+    }
+  }
+
+  static void redirectToRandomDataNode(NameNode nn, HttpServletResponse resp)
+      throws IOException {
+    final DatanodeID datanode = nn.getNamesystem().getRandomDatanode();
+    final String redirectLocation;
+    final String nodeToRedirect;
+    int redirectPort;
+    if (datanode != null) {
+      nodeToRedirect = datanode.getHost();
+      redirectPort = datanode.getInfoPort();
+    } else {
+      nodeToRedirect = nn.getHttpAddress().getHostName();
+      redirectPort = nn.getHttpAddress().getPort();
+    }
+    String fqdn = InetAddress.getByName(nodeToRedirect).getCanonicalHostName();
+    redirectLocation = "http://" + fqdn + ":" + redirectPort
+        + "/browseDirectory.jsp?namenodeInfoPort="
+        + nn.getHttpAddress().getPort() + "&dir="
+        + URLEncoder.encode("/", "UTF-8");
+    resp.sendRedirect(redirectLocation);
+  }
+
+  static class NodeListJsp {
+    private int rowNum = 0;
+
+    private long diskBytes = 1024 * 1024 * 1024;
+    private String diskByteStr = "GB";
+
+    private String sorterField = null;
+    private String sorterOrder = null;
+
+    private String whatNodes = "LIVE";
+
+    private String rowTxt() {
+      return "<tr class=\"" + (((rowNum++) % 2 == 0) ? "rowNormal" : "rowAlt")
+          + "\"> ";
+    }
+
+    private void counterReset() {
+      rowNum = 0;
+    }
+
+    private String nodeHeaderStr(String name) {
+      String ret = "class=header";
+      String order = "ASC";
+      if (name.equals(sorterField)) {
+        ret += sorterOrder;
+        if (sorterOrder.equals("ASC"))
+          order = "DSC";
+      }
+      ret += " onClick=\"window.document.location="
+          + "'/dfsnodelist.jsp?whatNodes=" + whatNodes + "&sorter/field="
+          + name + "&sorter/order=" + order
+          + "'\" title=\"sort on this column\"";
+
+      return ret;
+    }
+
+    void generateNodeData(JspWriter out, DatanodeDescriptor d,
+        String suffix, boolean alive, int nnHttpPort) throws IOException {
+      /*
+       * Say the datanode is dn1.hadoop.apache.org with ip 192.168.0.5 we use:
+       * 1) d.getHostName():d.getPort() to display. Domain and port are stripped
+       *    if they are common across the nodes. i.e. "dn1"
+       * 2) d.getHost():d.Port() for "title". i.e. "192.168.0.5:50010"
+       * 3) d.getHostName():d.getInfoPort() for url.
+       *    i.e. "http://dn1.hadoop.apache.org:50075/..."
+       * Note that "d.getHost():d.getPort()" is what DFS clients use to
+       * interact with datanodes.
+       */
+
+      // from nn_browsedfscontent.jsp:
+      String url = "http://" + d.getHostName() + ":" + d.getInfoPort()
+          + "/browseDirectory.jsp?namenodeInfoPort=" + nnHttpPort + "&dir="
+          + URLEncoder.encode("/", "UTF-8");
+
+      String name = d.getHostName() + ":" + d.getPort();
+      if (!name.matches("\\d+\\.\\d+.\\d+\\.\\d+.*"))
+        name = name.replaceAll("\\.[^.:]*", "");
+      int idx = (suffix != null && name.endsWith(suffix)) ? name
+          .indexOf(suffix) : -1;
+
+      out.print(rowTxt() + "<td class=\"name\"><a title=\"" + d.getHost() + ":"
+          + d.getPort() + "\" href=\"" + url + "\">"
+          + ((idx > 0) ? name.substring(0, idx) : name) + "</a>"
+          + ((alive) ? "" : "\n"));
+      if (!alive)
+        return;
+
+      long c = d.getCapacity();
+      long u = d.getDfsUsed();
+      long nu = d.getNonDfsUsed();
+      long r = d.getRemaining();
+      String percentUsed = StringUtils.limitDecimalTo2(d.getDfsUsedPercent());
+      String percentRemaining = StringUtils.limitDecimalTo2(d
+          .getRemainingPercent());
+
+      String adminState = (d.isDecommissioned() ? "Decommissioned" : (d
+          .isDecommissionInProgress() ? "Decommission In Progress"
+          : "In Service"));
+
+      long timestamp = d.getLastUpdate();
+      long currentTime = System.currentTimeMillis();
+      out.print("<td class=\"lastcontact\"> "
+          + ((currentTime - timestamp) / 1000)
+          + "<td class=\"adminstate\">"
+          + adminState
+          + "<td align=\"right\" class=\"capacity\">"
+          + StringUtils.limitDecimalTo2(c * 1.0 / diskBytes)
+          + "<td align=\"right\" class=\"used\">"
+          + StringUtils.limitDecimalTo2(u * 1.0 / diskBytes)
+          + "<td align=\"right\" class=\"nondfsused\">"
+          + StringUtils.limitDecimalTo2(nu * 1.0 / diskBytes)
+          + "<td align=\"right\" class=\"remaining\">"
+          + StringUtils.limitDecimalTo2(r * 1.0 / diskBytes)
+          + "<td align=\"right\" class=\"pcused\">"
+          + percentUsed
+          + "<td class=\"pcused\">"
+          + ServletUtil.percentageGraph((int) Double.parseDouble(percentUsed),
+              100) + "<td align=\"right\" class=\"pcremaining`\">"
+          + percentRemaining + "<td title=" + "\"blocks scheduled : "
+          + d.getBlocksScheduled() + "\" class=\"blocks\">" + d.numBlocks()
+          + "\n");
+    }
+
+    void generateNodesList(JspWriter out, NameNode nn,
+        HttpServletRequest request) throws IOException {
+      ArrayList<DatanodeDescriptor> live = new ArrayList<DatanodeDescriptor>();
+      ArrayList<DatanodeDescriptor> dead = new ArrayList<DatanodeDescriptor>();
+      nn.getNamesystem().DFSNodesStatus(live, dead);
+
+      whatNodes = request.getParameter("whatNodes"); // show only live or only
+                                                     // dead nodes
+      sorterField = request.getParameter("sorter/field");
+      sorterOrder = request.getParameter("sorter/order");
+      if (sorterField == null)
+        sorterField = "name";
+      if (sorterOrder == null)
+        sorterOrder = "ASC";
+
+      JspHelper.sortNodeList(live, sorterField, sorterOrder);
+      JspHelper.sortNodeList(dead, "name", "ASC");
+
+      // Find out common suffix. Should this be before or after the sort?
+      String port_suffix = null;
+      if (live.size() > 0) {
+        String name = live.get(0).getName();
+        int idx = name.indexOf(':');
+        if (idx > 0) {
+          port_suffix = name.substring(idx);
+        }
+
+        for (int i = 1; port_suffix != null && i < live.size(); i++) {
+          if (live.get(i).getName().endsWith(port_suffix) == false) {
+            port_suffix = null;
+            break;
+          }
+        }
+      }
+
+      counterReset();
+
+      try {
+        Thread.sleep(1000);
+      } catch (InterruptedException e) {
+      }
+
+      if (live.isEmpty() && dead.isEmpty()) {
+        out.print("There are no datanodes in the cluster");
+      } else {
+
+        int nnHttpPort = nn.getHttpAddress().getPort();
+        out.print("<div id=\"dfsnodetable\"> ");
+        if (whatNodes.equals("LIVE")) {
+
+          out.print("<a name=\"LiveNodes\" id=\"title\">" + "Live Datanodes : "
+              + live.size() + "</a>"
+              + "<br><br>\n<table border=1 cellspacing=0>\n");
+
+          counterReset();
+
+          if (live.size() > 0) {
+            if (live.get(0).getCapacity() > 1024 * diskBytes) {
+              diskBytes *= 1024;
+              diskByteStr = "TB";
+            }
+
+            out.print("<tr class=\"headerRow\"> <th " + nodeHeaderStr("name")
+                + "> Node <th " + nodeHeaderStr("lastcontact")
+                + "> Last <br>Contact <th " + nodeHeaderStr("adminstate")
+                + "> Admin State <th " + nodeHeaderStr("capacity")
+                + "> Configured <br>Capacity (" + diskByteStr + ") <th "
+                + nodeHeaderStr("used") + "> Used <br>(" + diskByteStr
+                + ") <th " + nodeHeaderStr("nondfsused")
+                + "> Non DFS <br>Used (" + diskByteStr + ") <th "
+                + nodeHeaderStr("remaining") + "> Remaining <br>("
+                + diskByteStr + ") <th " + nodeHeaderStr("pcused")
+                + "> Used <br>(%) <th " + nodeHeaderStr("pcused")
+                + "> Used <br>(%) <th " + nodeHeaderStr("pcremaining")
+                + "> Remaining <br>(%) <th " + nodeHeaderStr("blocks")
+                + "> Blocks\n");
+
+            JspHelper.sortNodeList(live, sorterField, sorterOrder);
+            for (int i = 0; i < live.size(); i++) {
+              generateNodeData(out, live.get(i), port_suffix, true, nnHttpPort);
+            }
+          }
+          out.print("</table>\n");
+        } else {
+
+          out.print("<br> <a name=\"DeadNodes\" id=\"title\"> "
+              + " Dead Datanodes : " + dead.size() + "</a><br><br>\n");
+
+          if (dead.size() > 0) {
+            out.print("<table border=1 cellspacing=0> <tr id=\"row1\"> "
+                + "<td> Node \n");
+
+            JspHelper.sortNodeList(dead, "name", "ASC");
+            for (int i = 0; i < dead.size(); i++) {
+              generateNodeData(out, dead.get(i), port_suffix, false, nnHttpPort);
+            }
+
+            out.print("</table>\n");
+          }
+        }
+        out.print("</div>");
+      }
+    }
+  }
+}

+ 9 - 2
src/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java

@@ -32,6 +32,7 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.hdfs.DFSUtil.ErrorSimulator;
+import org.apache.hadoop.hdfs.protocol.FSConstants;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants;
 import org.apache.hadoop.hdfs.server.common.InconsistentFSStateException;
 import org.apache.hadoop.hdfs.server.namenode.FSImage.NameNodeDirType;
@@ -276,10 +277,16 @@ public class SecondaryNameNode implements Runnable {
    */
   private String getInfoServer() throws IOException {
     URI fsName = FileSystem.getDefaultUri(conf);
-    if (!"hdfs".equals(fsName.getScheme())) {
+    if (!FSConstants.HDFS_URI_SCHEME.equalsIgnoreCase(fsName.getScheme())) {
       throw new IOException("This is not a DFS");
     }
-    return conf.get("dfs.http.address", "0.0.0.0:50070");
+    String configuredAddress = conf.get("dfs.http.address", "0.0.0.0:50070");
+    InetSocketAddress sockAddr = NetUtils.createSocketAddr(configuredAddress);
+    if (sockAddr.getAddress().isAnyLocalAddress()) {
+      return fsName.getHost() + ":" + sockAddr.getPort();
+    } else {
+      return configuredAddress;
+    }
   }
 
   /**

+ 1 - 0
src/java/org/apache/hadoop/hdfs/server/namenode/StreamFile.java

@@ -29,6 +29,7 @@ import javax.servlet.http.HttpServletResponse;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSInputStream;
 import org.apache.hadoop.hdfs.DFSClient;
+import org.apache.hadoop.hdfs.server.common.JspHelper;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.security.UnixUserGroupInformation;
 

+ 1 - 1
src/java/org/apache/hadoop/hdfs/server/protocol/NamespaceInfo.java

@@ -25,7 +25,7 @@ import java.io.IOException;
 import org.apache.hadoop.hdfs.protocol.FSConstants;
 import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.common.StorageInfo;
-import org.apache.hadoop.io.DeprecatedUTF8;
+import org.apache.hadoop.hdfs.DeprecatedUTF8;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableFactories;
 import org.apache.hadoop.io.WritableFactory;

+ 89 - 2
src/test/hdfs-with-mr/org/apache/hadoop/fs/TestCopyFiles.java

@@ -295,6 +295,37 @@ public class TestCopyFiles extends TestCase {
       if (cluster != null) { cluster.shutdown(); }
     }
   }
+
+  /** copy empty directory on dfs file system */
+  public void testEmptyDir() throws Exception {
+    String namenode = null;
+    MiniDFSCluster cluster = null;
+    try {
+      Configuration conf = new Configuration();
+      cluster = new MiniDFSCluster(conf, 2, true, null);
+      final FileSystem hdfs = cluster.getFileSystem();
+      namenode = FileSystem.getDefaultUri(conf).toString();
+      if (namenode.startsWith("hdfs://")) {
+        
+        FileSystem fs = FileSystem.get(URI.create(namenode), new Configuration());
+        fs.mkdirs(new Path("/empty"));
+
+        ToolRunner.run(new DistCp(conf), new String[] {
+                                         "-log",
+                                         namenode+"/logs",
+                                         namenode+"/empty",
+                                         namenode+"/dest"});
+        fs = FileSystem.get(URI.create(namenode+"/destdat"), conf);
+        assertTrue("Destination directory does not exist.",
+                   fs.exists(new Path(namenode+"/dest")));
+        deldir(hdfs, "/dest");
+        deldir(hdfs, "/empty");
+        deldir(hdfs, "/logs");
+      }
+    } finally {
+      if (cluster != null) { cluster.shutdown(); }
+    }
+  }
   
   /** copy files from local file system to dfs file system */
   public void testCopyFromLocalToDfs() throws Exception {
@@ -380,7 +411,7 @@ public class TestCopyFiles extends TestCase {
         deldir(hdfs, "/logs");
 
         ToolRunner.run(new DistCp(conf), new String[] {
-                                         "-p",
+                                         "-prbugp", // no t to avoid preserving mod. times
                                          "-update",
                                          "-log",
                                          namenode+"/logs",
@@ -393,7 +424,7 @@ public class TestCopyFiles extends TestCase {
 
         deldir(hdfs, "/logs");
         ToolRunner.run(new DistCp(conf), new String[] {
-                                         "-p",
+                                         "-prbugp", // no t to avoid preserving mod. times
                                          "-overwrite",
                                          "-log",
                                          namenode+"/logs",
@@ -551,6 +582,32 @@ public class TestCopyFiles extends TestCase {
         deldir(fs, "/destdat");
         deldir(fs, "/srcdat");
       }
+
+      {//test preserving times
+        MyFile[] files = createFiles(URI.create(nnUri), "/srcdat");
+        fs.mkdirs(new Path("/srcdat/tmpf1"));
+        fs.mkdirs(new Path("/srcdat/tmpf2"));
+        FileStatus[] srcstat = getFileStatus(fs, "/srcdat", files);
+        FsPermission[] permissions = new FsPermission[srcstat.length];
+        for(int i = 0; i < srcstat.length; i++) {
+          fs.setTimes(srcstat[i].getPath(), 40, 50);
+        }
+
+        ToolRunner.run(new DistCp(conf),
+            new String[]{"-pt", nnUri+"/srcdat", nnUri+"/destdat"});
+
+        FileStatus[] dststat = getFileStatus(fs, "/destdat", files);
+        for(int i = 0; i < dststat.length; i++) {
+          assertEquals("Modif. Time i=" + i, 40, dststat[i].getModificationTime());
+          assertEquals("Access Time i=" + i+ srcstat[i].getPath() + "-" + dststat[i].getPath(), 50, dststat[i].getAccessTime());
+        }
+        
+        assertTrue("Source and destination directories do not match.",
+                   checkFiles(fs, "/destdat", files));
+  
+        deldir(fs, "/destdat");
+        deldir(fs, "/srcdat");
+      }
     } finally {
       if (cluster != null) { cluster.shutdown(); }
     }
@@ -818,6 +875,36 @@ public class TestCopyFiles extends TestCase {
     }
   }
 
+  /** test globbing  */
+  public void testGlobbing() throws Exception {
+    String namenode = null;
+    MiniDFSCluster cluster = null;
+    try {
+      Configuration conf = new Configuration();
+      cluster = new MiniDFSCluster(conf, 2, true, null);
+      final FileSystem hdfs = cluster.getFileSystem();
+      namenode = FileSystem.getDefaultUri(conf).toString();
+      if (namenode.startsWith("hdfs://")) {
+        MyFile[] files = createFiles(URI.create(namenode), "/srcdat");
+        ToolRunner.run(new DistCp(conf), new String[] {
+                                         "-log",
+                                         namenode+"/logs",
+                                         namenode+"/srcdat/*",
+                                         namenode+"/destdat"});
+        assertTrue("Source and destination directories do not match.",
+                   checkFiles(hdfs, "/destdat", files));
+        FileSystem fs = FileSystem.get(URI.create(namenode+"/logs"), conf);
+        assertTrue("Log directory does not exist.",
+                   fs.exists(new Path(namenode+"/logs")));
+        deldir(hdfs, "/destdat");
+        deldir(hdfs, "/srcdat");
+        deldir(hdfs, "/logs");
+      }
+    } finally {
+      if (cluster != null) { cluster.shutdown(); }
+    }
+  }
+  
   static void create(FileSystem fs, Path f) throws IOException {
     FSDataOutputStream out = fs.create(f);
     try {

+ 46 - 0
src/test/hdfs-with-mr/org/apache/hadoop/fs/TestFileSystem.java

@@ -22,6 +22,7 @@ import java.io.DataInputStream;
 import java.io.IOException;
 import java.io.OutputStream;
 import java.util.Arrays;
+import java.util.Collections;
 import java.util.Random;
 import java.util.List;
 import java.util.ArrayList;
@@ -578,6 +579,37 @@ public class TestFileSystem extends TestCase {
     }
   }
 
+  public void testFsShutdownHook() throws Exception {
+    final Set<FileSystem> closed = Collections.synchronizedSet(new HashSet<FileSystem>());
+    Configuration conf = new Configuration();
+    Configuration confNoAuto = new Configuration();
+
+    conf.setClass("fs.test.impl", TestShutdownFileSystem.class, FileSystem.class);
+    confNoAuto.setClass("fs.test.impl", TestShutdownFileSystem.class, FileSystem.class);
+    confNoAuto.setBoolean("fs.automatic.close", false);
+
+    TestShutdownFileSystem fsWithAuto =
+      (TestShutdownFileSystem)(new Path("test://a/").getFileSystem(conf));
+    TestShutdownFileSystem fsWithoutAuto =
+      (TestShutdownFileSystem)(new Path("test://b/").getFileSystem(confNoAuto));
+
+    fsWithAuto.setClosedSet(closed);
+    fsWithoutAuto.setClosedSet(closed);
+
+    // Different URIs should result in different FS instances
+    assertNotSame(fsWithAuto, fsWithoutAuto);
+
+    FileSystem.CACHE.closeAll(true);
+    assertEquals(1, closed.size());
+    assertTrue(closed.contains(fsWithAuto));
+
+    closed.clear();
+
+    FileSystem.closeAll();
+    assertEquals(1, closed.size());
+    assertTrue(closed.contains(fsWithoutAuto));
+  }
+
 
   public void testCacheKeysAreCaseInsensitive()
     throws Exception
@@ -626,4 +658,18 @@ public class TestFileSystem extends TestCase {
     fs1.close();
     fs2.close();
   }
+
+  public static class TestShutdownFileSystem extends RawLocalFileSystem {
+    private Set<FileSystem> closedSet;
+
+    public void setClosedSet(Set<FileSystem> closedSet) {
+      this.closedSet = closedSet;
+    }
+    public void close() throws IOException {
+      if (closedSet != null) {
+        closedSet.add(this);
+      }
+      super.close();
+    }
+  }
 }

+ 46 - 35
src/test/hdfs-with-mr/org/apache/hadoop/hdfs/NNBench.java

@@ -150,13 +150,10 @@ public class NNBench {
         writer = SequenceFile.createWriter(tempFS, config, filePath, Text.class, 
                 LongWritable.class, CompressionType.NONE);
         writer.append(new Text(strFileName), new LongWritable(0l));
-      } catch(Exception e) {
-        throw new IOException(e.getLocalizedMessage());
       } finally {
         if (writer != null) {
           writer.close();
         }
-        writer = null;
       }
     }
   }
@@ -210,6 +207,9 @@ public class NNBench {
 
   /**
    * check for arguments and fail if the values are not specified
+   * @param index  positional number of an argument in the list of command
+   *   line's arguments
+   * @param length total number of arguments
    */
   public static void checkArgs(final int index, final int length) {
     if (index == length) {
@@ -220,8 +220,8 @@ public class NNBench {
   
   /**
    * Parse input arguments
-   * 
-   * @params args Command line inputs
+   *
+   * @param args array of command line's parameters to be parsed
    */
   public static void parseInputs(final String[] args) {
     // If there are no command line arguments, exit
@@ -358,8 +358,8 @@ public class NNBench {
     
     // Average latency is the average time to perform 'n' number of
     // operations, n being the number of files
-    double avgLatency1 = (double) totalTimeAL1 / (double) successfulFileOps;
-    double avgLatency2 = (double) totalTimeAL2 / (double) successfulFileOps;
+    double avgLatency1 = (double) totalTimeAL1 / successfulFileOps;
+    double avgLatency2 = (double) totalTimeAL2 / successfulFileOps;
     
     // The time it takes for the longest running map is measured. Using that,
     // cluster transactions per second is calculated. It includes time to 
@@ -367,7 +367,7 @@ public class NNBench {
     double longestMapTimeTPmS = (double) (mapEndTimeTPmS - mapStartTimeTPmS);
     double totalTimeTPS = (longestMapTimeTPmS == 0) ?
             (1000 * successfulFileOps) :
-            (double) (1000 * successfulFileOps) / (double) longestMapTimeTPmS;
+            (double) (1000 * successfulFileOps) / longestMapTimeTPmS;
             
     // The time it takes to perform 'n' operations is calculated (in ms),
     // n being the number of files. Using that time, the average execution 
@@ -375,22 +375,22 @@ public class NNBench {
     // failed operations
     double AverageExecutionTime = (totalTimeTPmS == 0) ?
         (double) successfulFileOps : 
-        (double) (totalTimeTPmS / successfulFileOps);
+        (double) totalTimeTPmS / successfulFileOps;
             
     if (operation.equals(OP_CREATE_WRITE)) {
       // For create/write/close, it is treated as two transactions,
       // since a file create from a client perspective involves create and close
       resultTPSLine1 = "               TPS: Create/Write/Close: " + 
         (int) (totalTimeTPS * 2);
-      resultTPSLine2 = "Avg exec time (ms): Create/Write/Close: " + 
-        (double) AverageExecutionTime;
+      resultTPSLine2 = "Avg exec time (ms): Create/Write/Close: " +
+        AverageExecutionTime;
       resultALLine1 = "            Avg Lat (ms): Create/Write: " + avgLatency1;
       resultALLine2 = "                   Avg Lat (ms): Close: " + avgLatency2;
     } else if (operation.equals(OP_OPEN_READ)) {
       resultTPSLine1 = "                        TPS: Open/Read: " + 
         (int) totalTimeTPS;
       resultTPSLine2 = "         Avg Exec time (ms): Open/Read: " + 
-        (double) AverageExecutionTime;
+        AverageExecutionTime;
       resultALLine1 = "                    Avg Lat (ms): Open: " + avgLatency1;
       if (readFileAfterOpen) {
         resultALLine2 = "                  Avg Lat (ms): Read: " + avgLatency2;
@@ -399,13 +399,13 @@ public class NNBench {
       resultTPSLine1 = "                           TPS: Rename: " + 
         (int) totalTimeTPS;
       resultTPSLine2 = "            Avg Exec time (ms): Rename: " + 
-        (double) AverageExecutionTime;
+        AverageExecutionTime;
       resultALLine1 = "                  Avg Lat (ms): Rename: " + avgLatency1;
     } else if (operation.equals(OP_DELETE)) {
       resultTPSLine1 = "                           TPS: Delete: " + 
         (int) totalTimeTPS;
       resultTPSLine2 = "            Avg Exec time (ms): Delete: " + 
-        (double) AverageExecutionTime;
+        AverageExecutionTime;
       resultALLine1 = "                  Avg Lat (ms): Delete: " + avgLatency1;
     }
     
@@ -558,6 +558,7 @@ public class NNBench {
   /**
   * Main method for running the NNBench benchmarks
   *
+  * @param args array of command line arguments
   * @throws IOException indicates a problem with test startup
   */
   public static void main(String[] args) throws IOException {
@@ -587,7 +588,7 @@ public class NNBench {
   /**
    * Mapper class
    */
-  static class NNBenchMapper extends Configured 
+  static class NNBenchMapper extends Configured
           implements Mapper<Text, LongWritable, Text, Text> {
     FileSystem filesystem = null;
     private String hostName = null;
@@ -639,13 +640,15 @@ public class NNBench {
      */
     public void close() throws IOException {
     }
-    
+
     /**
-    * Returns when the current number of seconds from the epoch equals
-    * the command line argument given by <code>-startTime</code>.
-    * This allows multiple instances of this program, running on clock
-    * synchronized nodes, to start at roughly the same time.
-    */
+     * Returns when the current number of seconds from the epoch equals
+     * the command line argument given by <code>-startTime</code>.
+     * This allows multiple instances of this program, running on clock
+     * synchronized nodes, to start at roughly the same time.
+     * @return true if the method was able to sleep for <code>-startTime</code>
+     * without interruption; false otherwise
+     */
     private boolean barrier() {
       long startTime = getConf().getLong("test.nnbench.starttime", 0l);
       long currentTime = System.currentTimeMillis();
@@ -698,16 +701,16 @@ public class NNBench {
       if (barrier()) {
         if (op.equals(OP_CREATE_WRITE)) {
           startTimeTPmS = System.currentTimeMillis();
-          doCreateWriteOp("file_" + hostName + "_", output, reporter);
+          doCreateWriteOp("file_" + hostName + "_", reporter);
         } else if (op.equals(OP_OPEN_READ)) {
           startTimeTPmS = System.currentTimeMillis();
-          doOpenReadOp("file_" + hostName + "_", output, reporter);
+          doOpenReadOp("file_" + hostName + "_", reporter);
         } else if (op.equals(OP_RENAME)) {
           startTimeTPmS = System.currentTimeMillis();
-          doRenameOp("file_" + hostName + "_", output, reporter);
+          doRenameOp("file_" + hostName + "_", reporter);
         } else if (op.equals(OP_DELETE)) {
           startTimeTPmS = System.currentTimeMillis();
-          doDeleteOp("file_" + hostName + "_", output, reporter);
+          doDeleteOp("file_" + hostName + "_", reporter);
         }
         
         endTimeTPms = System.currentTimeMillis();
@@ -735,11 +738,13 @@ public class NNBench {
     
     /**
      * Create and Write operation.
+     * @param name of the prefix of the putput file to be created
+     * @param reporter an instanse of (@link Reporter) to be used for
+     *   status' updates
      */
     private void doCreateWriteOp(String name,
-            OutputCollector<Text, Text> output,
-            Reporter reporter) {
-      FSDataOutputStream out = null;
+                                 Reporter reporter) {
+      FSDataOutputStream out;
       byte[] buffer = new byte[bytesToWrite];
       
       for (long l = 0l; l < numberOfFiles; l++) {
@@ -783,11 +788,13 @@ public class NNBench {
     
     /**
      * Open operation
+     * @param name of the prefix of the putput file to be read
+     * @param reporter an instanse of (@link Reporter) to be used for
+     *   status' updates
      */
     private void doOpenReadOp(String name,
-            OutputCollector<Text, Text> output,
-            Reporter reporter) {
-      FSDataInputStream input = null;
+                              Reporter reporter) {
+      FSDataInputStream input;
       byte[] buffer = new byte[bytesToWrite];
       
       for (long l = 0l; l < numberOfFiles; l++) {
@@ -824,10 +831,12 @@ public class NNBench {
     
     /**
      * Rename operation
+     * @param name of prefix of the file to be renamed
+     * @param reporter an instanse of (@link Reporter) to be used for
+     *   status' updates
      */
     private void doRenameOp(String name,
-            OutputCollector<Text, Text> output,
-            Reporter reporter) {
+                            Reporter reporter) {
       for (long l = 0l; l < numberOfFiles; l++) {
         Path filePath = new Path(new Path(baseDir, dataDirName), 
                 name + "_" + l);
@@ -857,10 +866,12 @@ public class NNBench {
     
     /**
      * Delete operation
+     * @param name of prefix of the file to be deleted
+     * @param reporter an instanse of (@link Reporter) to be used for
+     *   status' updates
      */
     private void doDeleteOp(String name,
-            OutputCollector<Text, Text> output,
-            Reporter reporter) {
+                            Reporter reporter) {
       for (long l = 0l; l < numberOfFiles; l++) {
         Path filePath = new Path(new Path(baseDir, dataDirName), 
                 name + "_" + l);

+ 36 - 21
src/test/hdfs-with-mr/org/apache/hadoop/hdfs/NNBenchWithoutMR.java

@@ -28,8 +28,8 @@ import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.mapred.JobConf;
 
 /**
  * This program executes a specified operation that applies load to 
@@ -59,7 +59,6 @@ public class NNBenchWithoutMR {
   // variables initialized in main()
   private static FileSystem fileSys = null;
   private static Path taskDir = null;
-  private static String uniqueId = null;
   private static byte[] buffer;
   private static long maxExceptionsPerFile = 200;
     
@@ -69,12 +68,14 @@ public class NNBenchWithoutMR {
    * This allows multiple instances of this program, running on clock
    * synchronized nodes, to start at roughly the same time.
    */
+
   static void barrier() {
     long sleepTime;
     while ((sleepTime = startTime - System.currentTimeMillis()) > 0) {
       try {
         Thread.sleep(sleepTime);
       } catch (InterruptedException ex) {
+        //This left empty on purpose
       }
     }
   }
@@ -98,18 +99,20 @@ public class NNBenchWithoutMR {
   static int createWrite() {
     int totalExceptions = 0;
     FSDataOutputStream out = null;
-    boolean success = false;
+    boolean success;
     for (int index = 0; index < numFiles; index++) {
       int singleFileExceptions = 0;
       do { // create file until is succeeds or max exceptions reached
         try {
           out = fileSys.create(
-                               new Path(taskDir, "" + index), false, 512, (short)1, bytesPerBlock);
+                  new Path(taskDir, "" + index), false, 512,
+                  (short)1, bytesPerBlock);
           success = true;
         } catch (IOException ioe) { 
           success=false; 
           totalExceptions++;
-          handleException("creating file #" + index, ioe, ++singleFileExceptions);
+          handleException("creating file #" + index, ioe,
+                  ++singleFileExceptions);
         }
       } while (!success);
       long toBeWritten = bytesPerFile;
@@ -120,7 +123,8 @@ public class NNBenchWithoutMR {
           out.write(buffer, 0, nbytes);
         } catch (IOException ioe) {
           totalExceptions++;
-          handleException("writing to file #" + index, ioe, ++singleFileExceptions);
+          handleException("writing to file #" + index, ioe,
+                  ++singleFileExceptions);
         }
       }
       do { // close file until is succeeds
@@ -130,7 +134,8 @@ public class NNBenchWithoutMR {
         } catch (IOException ioe) {
           success=false; 
           totalExceptions++;
-          handleException("closing file #" + index, ioe, ++singleFileExceptions);
+          handleException("closing file #" + index, ioe,
+                  ++singleFileExceptions);
         }
       } while (!success);
     }
@@ -144,7 +149,7 @@ public class NNBenchWithoutMR {
    */
   static int openRead() {
     int totalExceptions = 0;
-    FSDataInputStream in = null;
+    FSDataInputStream in;
     for (int index = 0; index < numFiles; index++) {
       int singleFileExceptions = 0;
       try {
@@ -153,11 +158,12 @@ public class NNBenchWithoutMR {
         while (toBeRead > 0) {
           int nbytes = (int) Math.min(buffer.length, toBeRead);
           toBeRead -= nbytes;
-          try { // only try once
+          try { // only try once && we don't care about a number of bytes read
             in.read(buffer, 0, nbytes);
           } catch (IOException ioe) {
             totalExceptions++;
-            handleException("reading from file #" + index, ioe, ++singleFileExceptions);
+            handleException("reading from file #" + index, ioe,
+                    ++singleFileExceptions);
           }
         }
         in.close();
@@ -177,19 +183,23 @@ public class NNBenchWithoutMR {
    */
   static int rename() {
     int totalExceptions = 0;
-    boolean success = false;
+    boolean success;
     for (int index = 0; index < numFiles; index++) {
       int singleFileExceptions = 0;
       do { // rename file until is succeeds
         try {
-          boolean result = fileSys.rename(
-                                          new Path(taskDir, "" + index), new Path(taskDir, "A" + index));
+          // Possible result of this operation is at no interest to us for it
+          // can return false only if the namesystem
+          // could rename the path from the name
+          // space (e.g. no Exception has been thrown)
+          fileSys.rename(new Path(taskDir, "" + index),
+              new Path(taskDir, "A" + index));
           success = true;
-        } catch (IOException ioe) { 
-          success=false; 
+        } catch (IOException ioe) {
+          success = false;
           totalExceptions++;
           handleException("creating file #" + index, ioe, ++singleFileExceptions);
-       }
+        }
       } while (!success);
     }
     return totalExceptions;
@@ -203,14 +213,18 @@ public class NNBenchWithoutMR {
    */
   static int delete() {
     int totalExceptions = 0;
-    boolean success = false;
+    boolean success;
     for (int index = 0; index < numFiles; index++) {
       int singleFileExceptions = 0;
       do { // delete file until is succeeds
         try {
-          boolean result = fileSys.delete(new Path(taskDir, "A" + index), true);
+          // Possible result of this operation is at no interest to us for it
+          // can return false only if namesystem
+          // delete could remove the path from the name
+          // space (e.g. no Exception has been thrown)
+          fileSys.delete(new Path(taskDir, "A" + index), true);
           success = true;
-        } catch (IOException ioe) { 
+        } catch (IOException ioe) {
           success=false; 
           totalExceptions++;
           handleException("creating file #" + index, ioe, ++singleFileExceptions);
@@ -239,6 +253,7 @@ public class NNBenchWithoutMR {
    *         [-bytesPerChecksum <value for io.bytes.per.checksum>]
    * </pre>
    *
+   * @param args is an array of the program command line arguments
    * @throws IOException indicates a problem with test startup
    */
   public static void main(String[] args) throws IOException {
@@ -281,7 +296,7 @@ public class NNBenchWithoutMR {
     bytesPerFile = bytesPerBlock * blocksPerFile;
     
     JobConf jobConf = new JobConf(new Configuration(), NNBench.class);
-    
+
     if ( bytesPerChecksum < 0 ) { // if it is not set in cmdline
       bytesPerChecksum = jobConf.getInt("io.bytes.per.checksum", 512);
     }
@@ -308,7 +323,7 @@ public class NNBenchWithoutMR {
       }
     
     fileSys = FileSystem.get(jobConf);
-    uniqueId = java.net.InetAddress.getLocalHost().getHostName();
+    String uniqueId = java.net.InetAddress.getLocalHost().getHostName();
     taskDir = new Path(baseDir, uniqueId);
     // initialize buffer used for writing/reading file
     buffer = new byte[(int) Math.min(bytesPerFile, 32768L)];

+ 71 - 0
src/test/hdfs/org/apache/hadoop/cli/clitest_data/data1k

@@ -0,0 +1,71 @@
+12345678901234
+12345678901234
+12345678901234
+12345678901234
+12345678901234
+12345678901234
+12345678901234
+12345678901234
+12345678901234
+12345678901234
+12345678901234
+12345678901234
+12345678901234
+12345678901234
+12345678901234
+12345678901234
+12345678901234
+12345678901234
+12345678901234
+12345678901234
+12345678901234
+12345678901234
+12345678901234
+12345678901234
+12345678901234
+12345678901234
+12345678901234
+12345678901234
+12345678901234
+12345678901234
+12345678901234
+12345678901234
+12345678901234
+12345678901234
+12345678901234
+12345678901234
+12345678901234
+12345678901234
+12345678901234
+12345678901234
+12345678901234
+12345678901234
+12345678901234
+12345678901234
+12345678901234
+12345678901234
+12345678901234
+12345678901234
+12345678901234
+12345678901234
+12345678901234
+12345678901234
+12345678901234
+12345678901234
+12345678901234
+12345678901234
+12345678901234
+12345678901234
+12345678901234
+12345678901234
+12345678901234
+12345678901234
+12345678901234
+12345678901234
+12345678901234
+12345678901234
+12345678901234
+12345678901234
+12345678901234
+12345678901234
+12345678901234

+ 54 - 27
src/test/hdfs/org/apache/hadoop/cli/testHDFSConf.xml

@@ -1132,6 +1132,33 @@
         </comparator>
       </comparators>
     </test>
+
+    <test> <!-- TESTED -->
+      <description>duh: Test for hdfs:// path - directory</description>
+      <test-commands>
+        <command>-fs NAMENODE -mkdir hdfs:///dir0</command>
+        <command>-fs NAMENODE -put CLITEST_DATA/data15bytes hdfs:///dir0/data15bytes</command>
+        <command>-fs NAMENODE -put CLITEST_DATA/data1k hdfs:///dir0/data1k</command>
+        <command>-fs NAMENODE -du -h hdfs:///dir0</command>
+      </test-commands>
+      <cleanup-commands>
+        <command>-fs NAMENODE -rmr hdfs:///dir0</command>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>TokenComparator</type>
+          <expected-output>Found 2 items</expected-output>
+        </comparator>
+        <comparator>
+          <type>RegexpComparator</type>
+          <expected-output>^15( |\t)*hdfs://localhost[.a-z]*:[0-9]*/dir0/data15bytes</expected-output>
+        </comparator>
+        <comparator>
+          <type>RegexpComparator</type>
+          <expected-output>^1.0k( |\t)*hdfs://localhost[.a-z]*:[0-9]*/dir0/data1k</expected-output>
+        </comparator>
+      </comparators>
+    </test>
     
     <test> <!-- TESTED -->
       <description>du: Test for hdfs:// path - directory using globbing</description>
@@ -1314,7 +1341,7 @@
       <comparators>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^hdfs://localhost[.a-z]*:[0-9]*/dir0( |\t)*450</expected-output>
+          <expected-output>^450\s+hdfs://localhost[.a-z]*:[0-9]*/dir0</expected-output>
         </comparator>
       </comparators>
     </test>
@@ -1346,7 +1373,7 @@
       <comparators>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^hdfs://localhost[.a-z]*:[0-9]*/user/[a-z]*/dir0( |\t)*450</expected-output>
+          <expected-output>^450\s+hdfs://localhost[.a-z]*:[0-9]*/user/[a-z]*/dir0</expected-output>
         </comparator>
       </comparators>
     </test>
@@ -1384,7 +1411,7 @@
       <comparators>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^hdfs://localhost[.a-z]*:[0-9]*/dir0( |\t)*450</expected-output>
+          <expected-output>^450\s+hdfs://localhost[.a-z]*:[0-9]*/dir0</expected-output>
         </comparator>
       </comparators>
     </test>
@@ -1417,7 +1444,7 @@
       <comparators>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^hdfs://localhost[.a-z]*:[0-9]*/dir0( |\t)*450</expected-output>
+          <expected-output>^450\s+hdfs://localhost[.a-z]*:[0-9]*/dir0</expected-output>
         </comparator>
       </comparators>
     </test>
@@ -1455,7 +1482,7 @@
       <comparators>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^hdfs://localhost[.a-z]*:[0-9]*/dir0( |\t)*450</expected-output>
+          <expected-output>^450\s+hdfs://localhost[.a-z]*:[0-9]*/dir0</expected-output>
         </comparator>
       </comparators>
     </test>
@@ -1487,7 +1514,7 @@
       <comparators>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^hdfs://localhost[.a-z]*:[0-9]*/dir0( |\t)*450</expected-output>
+          <expected-output>^450\s+hdfs://localhost[.a-z]*:[0-9]*/dir0</expected-output>
         </comparator>
       </comparators>
     </test>
@@ -1525,7 +1552,7 @@
       <comparators>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^hdfs://localhost[.a-z]*:[0-9]*/dir0( |\t)*450</expected-output>
+          <expected-output>^450\s+hdfs://localhost[.a-z]*:[0-9]*/dir0</expected-output>
         </comparator>
       </comparators>
     </test>
@@ -5485,7 +5512,7 @@
       <comparators>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^hdfs://localhost[.a-z]*:[0-9]*/dir0(|\t)*0</expected-output>
+          <expected-output>^0\s+hdfs://localhost[.a-z]*:[0-9]*/dir0</expected-output>
         </comparator>
       </comparators>
     </test>
@@ -5502,7 +5529,7 @@
       <comparators>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^hdfs://localhost[.a-z]*:[0-9]*/user/[a-z]*/dir0(|\t)*0</expected-output>
+          <expected-output>^0\s+hdfs://localhost[.a-z]*:[0-9]*/user/[a-z]*/dir0</expected-output>
         </comparator>
       </comparators>
     </test>
@@ -5522,19 +5549,19 @@
       <comparators>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^hdfs://localhost[.a-z]*:[0-9]*/dir0(|\t)*0</expected-output>
+          <expected-output>^0\s+hdfs://localhost[.a-z]*:[0-9]*/dir0</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^hdfs://localhost[.a-z]*:[0-9]*/dir1(|\t)*0</expected-output>
+          <expected-output>^0\s+hdfs://localhost[.a-z]*:[0-9]*/dir1</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^hdfs://localhost[.a-z]*:[0-9]*/dir2(|\t)*0</expected-output>
+          <expected-output>^0\s+hdfs://localhost[.a-z]*:[0-9]*/dir2</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^hdfs://localhost[.a-z]*:[0-9]*/dir3(|\t)*0</expected-output>
+          <expected-output>^0\s+hdfs://localhost[.a-z]*:[0-9]*/dir3</expected-output>
         </comparator>
       </comparators>
     </test>
@@ -5554,19 +5581,19 @@
       <comparators>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^hdfs://localhost[.a-z]*:[0-9]*/user/[a-z]*/dir0(|\t)*0</expected-output>
+          <expected-output>^0\s+hdfs://localhost[.a-z]*:[0-9]*/user/[a-z]*/dir0</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^hdfs://localhost[.a-z]*:[0-9]*/user/[a-z]*/dir1(|\t)*0</expected-output>
+          <expected-output>^0\s+hdfs://localhost[.a-z]*:[0-9]*/user/[a-z]*/dir1</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^hdfs://localhost[.a-z]*:[0-9]*/user/[a-z]*/dir2(|\t)*0</expected-output>
+          <expected-output>^0\s+hdfs://localhost[.a-z]*:[0-9]*/user/[a-z]*/dir2</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^hdfs://localhost[.a-z]*:[0-9]*/user/[a-z]*/dir3(|\t)*0</expected-output>
+          <expected-output>^0\s+hdfs://localhost[.a-z]*:[0-9]*/user/[a-z]*/dir3</expected-output>
         </comparator>
       </comparators>
     </test>
@@ -5617,7 +5644,7 @@
       <comparators>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^hdfs://localhost[.a-z]*:[0-9]*/dir0(|\t)*0</expected-output>
+          <expected-output>^0\s+hdfs://localhost[.a-z]*:[0-9]*/dir0(|\t)*</expected-output>
         </comparator>
       </comparators>
     </test>
@@ -5634,19 +5661,19 @@
       <comparators>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^hdfs://localhost[.a-z]*:[0-9]*/dir0(|\t)*0</expected-output>
+          <expected-output>^0\s+hdfs://localhost[.a-z]*:[0-9]*/dir0</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^hdfs://localhost[.a-z]*:[0-9]*/dir1(|\t)*0</expected-output>
+          <expected-output>^0\s+hdfs://localhost[.a-z]*:[0-9]*/dir1</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^hdfs://localhost[.a-z]*:[0-9]*/dir2(|\t)*0</expected-output>
+          <expected-output>^0\s+hdfs://localhost[.a-z]*:[0-9]*/dir2</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^hdfs://localhost[.a-z]*:[0-9]*/dir3(|\t)*0</expected-output>
+          <expected-output>^0\s+hdfs://localhost[.a-z]*:[0-9]*/dir3</expected-output>
         </comparator>
       </comparators>
     </test>
@@ -5697,7 +5724,7 @@
       <comparators>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^hdfs://localhost[.a-z]*:[0-9]*/dir0(|\t)*0</expected-output>
+          <expected-output>^0\s+hdfs://localhost[.a-z]*:[0-9]*/dir0</expected-output>
         </comparator>
       </comparators>
     </test>
@@ -5714,19 +5741,19 @@
       <comparators>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^hdfs://localhost[.a-z]*:[0-9]*/dir0(|\t)*0</expected-output>
+          <expected-output>^0\s+hdfs://localhost[.a-z]*:[0-9]*/dir0</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^hdfs://localhost[.a-z]*:[0-9]*/dir1(|\t)*0</expected-output>
+          <expected-output>^0\s+hdfs://localhost[.a-z]*:[0-9]*/dir1</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^hdfs://localhost[.a-z]*:[0-9]*/dir2(|\t)*0</expected-output>
+          <expected-output>^0\s+hdfs://localhost[.a-z]*:[0-9]*/dir2</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^hdfs://localhost[.a-z]*:[0-9]*/dir3(|\t)*0</expected-output>
+          <expected-output>^0\s+hdfs://localhost[.a-z]*:[0-9]*/dir3</expected-output>
         </comparator>
       </comparators>
     </test>

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

@@ -29,7 +29,6 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FsShell;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
-import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.util.ToolRunner;
 
 public class TestDFSShellGenericOptions extends TestCase {
@@ -101,8 +100,8 @@ public class TestDFSShellGenericOptions extends TestCase {
     FileSystem fs=null;
     try {
       ToolRunner.run(shell, args);
-      fs = new DistributedFileSystem(NameNode.getAddress(namenode), 
-                                     shell.getConf());
+      fs = FileSystem.get(NameNode.getUri(NameNode.getAddress(namenode)),
+          shell.getConf());
       assertTrue("Directory does not get created", 
                  fs.isDirectory(new Path("/data")));
       fs.delete(new Path("/data"), true);

+ 7 - 10
src/test/hdfs/org/apache/hadoop/hdfs/TestDFSStartupVersions.java

@@ -17,22 +17,21 @@
  */
 package org.apache.hadoop.hdfs;
 
+import static org.apache.hadoop.hdfs.server.common.HdfsConstants.NodeType.DATA_NODE;
+import static org.apache.hadoop.hdfs.server.common.HdfsConstants.NodeType.NAME_NODE;
+
 import java.io.File;
+
 import junit.framework.TestCase;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.protocol.FSConstants;
-import org.apache.hadoop.hdfs.server.common.HdfsConstants.NodeType;
-import org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption;
-
-import static org.apache.hadoop.hdfs.server.common.HdfsConstants.NodeType.NAME_NODE;
-import static org.apache.hadoop.hdfs.server.common.HdfsConstants.NodeType.DATA_NODE;
-
-import org.apache.hadoop.hdfs.server.common.HdfsConstants;
 import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.common.StorageInfo;
-import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.server.common.HdfsConstants.NodeType;
+import org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption;
 
 /**
  * This test ensures the appropriate response (successful or failure) from 
@@ -42,8 +41,6 @@ public class TestDFSStartupVersions extends TestCase {
   
   private static final Log LOG = LogFactory.getLog(
                                                    "org.apache.hadoop.hdfs.TestDFSStartupVersions");
-  private static Path TEST_ROOT_DIR = new Path(
-                                               System.getProperty("test.build.data","/tmp").toString().replace(' ', '+'));
   private MiniDFSCluster cluster = null;
   
   /**

+ 97 - 0
src/test/hdfs/org/apache/hadoop/hdfs/TestFileCreation.java

@@ -22,15 +22,18 @@ import java.io.File;
 import java.io.FileReader;
 import java.io.IOException;
 import java.net.InetSocketAddress;
+import java.util.EnumSet;
 
 import org.apache.commons.logging.impl.Log4JLogger;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.BlockLocation;
+import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.FSConstants;
@@ -617,6 +620,100 @@ public class TestFileCreation extends junit.framework.TestCase {
       cluster.shutdown();
     }
   }
+  
+  /**
+   * Test file creation with all supported flags.
+   */
+  public void testFileCreationWithFlags() throws IOException {
+    Configuration conf = new Configuration();
+    if (simulatedStorage) {
+      conf.setBoolean(SimulatedFSDataset.CONFIG_PROPERTY_SIMULATED, true);
+    }
+    MiniDFSCluster cluster = new MiniDFSCluster(conf, 1, true, null);
+    FileSystem fs = cluster.getFileSystem();
+    Path path = new Path("/" + System.currentTimeMillis()
+        + "-testFileCreationWithFlags");
+    FSDataOutputStream out = null;
+
+    // append to a non-exist file, it should throw an IOException
+    try {
+      IOException expectedException = null;
+      EnumSet<CreateFlag> appendNoFile = EnumSet.of(CreateFlag.APPEND);
+      // this should throw a IOException, because the file does not exist
+      try {
+        out = createFileWithFlag(fs, path, 1, appendNoFile);
+      } catch (IOException e) {
+        expectedException = e;
+      } finally {
+        if (out != null)
+          out.close();
+      }
+      assertTrue(
+          "Append a non-exists file with no create flag should throw an IOException ",
+          expectedException != null);
+
+      // the file already exists, and recreate it with CreateFlag.APPEND,
+      // CreateFlag.CREATE. It will not throw any exception.
+      EnumSet<CreateFlag> appendAndCreate = EnumSet.of(CreateFlag.APPEND,
+          CreateFlag.CREATE);
+      out = createFileWithFlag(fs, path, 1, appendAndCreate);
+      out.close();
+
+      // the file already exists, and recreate it only with CreateFlag.CREATE
+      // flag. it should throw an IOException
+      expectedException = null;
+      EnumSet<CreateFlag> createExistsFile = EnumSet.of(CreateFlag.CREATE);
+      // this should throw a IOException, because the file already exists
+      try {
+        createFileWithFlag(fs, path, 1, createExistsFile);
+      } catch (IOException e) {
+        expectedException = e;
+      }
+      assertTrue(
+          "create a file which already exists should throw an IOException ",
+          expectedException != null);
+
+      // the file exists, recreate it with the flag of CreateFlag.OVERWRITE.
+      EnumSet<CreateFlag> overwriteFile = EnumSet.of(CreateFlag.OVERWRITE);
+      out = createFileWithFlag(fs, path, 1, overwriteFile);
+      out.close();
+
+      // the file exists, recreate it with the flag of CreateFlag.OVERWRITE
+      // together with CreateFlag.CREATE. It has the same effect as only specify
+      // CreateFlag.OVERWRITE.
+      EnumSet<CreateFlag> overwriteWithCreateFile = EnumSet.of(
+          CreateFlag.OVERWRITE, CreateFlag.CREATE);
+      out = createFileWithFlag(fs, path, 1, overwriteWithCreateFile);
+      out.close();
+
+      // the file exists, recreate it with the flag of CreateFlag.OVERWRITE
+      // together with CreateFlag.APPEND. It has the same effect as only specify
+      // CreateFlag.OVERWRITE.
+      EnumSet<CreateFlag> overwriteWithAppendFile = EnumSet.of(
+          CreateFlag.OVERWRITE, CreateFlag.APPEND);
+      out = createFileWithFlag(fs, path, 1, overwriteWithAppendFile);
+      out.close();
+
+      fs.delete(path, true);
+
+      EnumSet<CreateFlag> createNonExistsFile = EnumSet.of(CreateFlag.CREATE,
+          CreateFlag.OVERWRITE);
+      out = createFileWithFlag(fs, path, 1, createNonExistsFile);
+      out.close();
+      fs.delete(path, true);
+    } finally {
+      cluster.shutdown();
+    }
+  }
+  
+  // creates a file with the flag api
+  static FSDataOutputStream createFileWithFlag(FileSystem fileSys, Path name, int repl, EnumSet<CreateFlag> flag)
+    throws IOException {
+    System.out.println("createFile: Created " + name + " with " + repl + " replica.");
+    FSDataOutputStream stm = fileSys.create(name, FsPermission.getDefault(), flag, 
+                                            fileSys.getConf().getInt("io.file.buffer.size", 4096),(short)repl, (long)blockSize, null);
+    return stm;
+  }
 
 /**
  * Test that file data becomes available before file is closed.

+ 0 - 5
src/test/hdfs/org/apache/hadoop/hdfs/TestFileStatus.java

@@ -35,10 +35,6 @@ public class TestFileStatus extends TestCase {
   static final int blockSize = 8192;
   static final int fileSize = 16384;
 
-  private static String TEST_ROOT_DIR =
-    new Path(System.getProperty("test.build.data","/tmp"))
-    .toString().replace(' ', '+');
-  
   private void writeFile(FileSystem fileSys, Path name, int repl,
                          int fileSize, int blockSize)
     throws IOException {
@@ -99,7 +95,6 @@ public class TestFileStatus extends TestCase {
 
       // create an empty directory
       //
-      Path parentDir = new Path("/test");
       Path dir = new Path("/test/mkdirs");
       assertTrue(fs.mkdirs(dir));
       assertTrue(fs.exists(dir));

+ 137 - 19
src/test/hdfs/org/apache/hadoop/hdfs/TestHDFSServerPorts.java

@@ -19,19 +19,24 @@ package org.apache.hadoop.hdfs;
 
 import java.io.File;
 import java.io.IOException;
+import java.net.UnknownHostException;
 
 import junit.framework.TestCase;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
+import org.apache.hadoop.hdfs.server.namenode.BackupNode;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
-import org.apache.hadoop.hdfs.server.namenode.SecondaryNameNode;
+import org.apache.hadoop.net.DNS;
 
 /**
  * This test checks correctness of port usage by hdfs components:
- * NameNode, DataNode, and SecondaryNamenode.
+ * NameNode, DataNode, SecondaryNamenode and BackupNode.
  * 
  * The correct behavior is:<br> 
  * - when a specific port is provided the server must either start on that port 
@@ -40,17 +45,47 @@ import org.apache.hadoop.hdfs.server.namenode.SecondaryNameNode;
  * a free port and start on it.
  */
 public class TestHDFSServerPorts extends TestCase {
+  public static final Log LOG = LogFactory.getLog(TestHDFSServerPorts.class);
+  
   public static final String NAME_NODE_HOST = "localhost:";
-  public static final String NAME_NODE_HTTP_HOST = "0.0.0.0:";
+  public static final String NAME_NODE_HTTP_HOST = getFullHostName() + ":";
 
   Configuration config;
   File hdfsDir;
 
   /**
-   * Start the name-node.
+   * Attempt to determine the fully qualified domain name for this host 
+   * to compare during testing.
+   * 
+   * This is necessary because in order for the BackupNode test to correctly 
+   * work, the namenode must have its http server started with the fully 
+   * qualified address, as this is the one the backupnode will attempt to start
+   * on as well.
+   * 
+   * @return Fully qualified hostname, or 127.0.0.1 if can't determine
+   */
+  private static String getFullHostName() {
+    try {
+      return DNS.getDefaultHost("default");
+    } catch (UnknownHostException e) {
+      LOG.warn("Unable to determine hostname.  May interfere with obtaining " +
+          "valid test results.");
+      return "127.0.0.1";
+    }
+  }
+  
+  /**
+   * Get base directory these tests should run in.
+   */
+  private String getTestingDir() {
+    return System.getProperty("test.build.data", "build/test/data");
+  }
+  
+  /**
+   * Start the namenode.
    */
   public NameNode startNameNode() throws IOException {
-    String dataDir = System.getProperty("test.build.data");
+    String dataDir = getTestingDir();
     hdfsDir = new File(dataDir, "dfs");
     if ( hdfsDir.exists() && !FileUtil.fullyDelete(hdfsDir) ) {
       throw new IOException("Could not delete hdfs directory '" + hdfsDir + "'");
@@ -67,11 +102,39 @@ public class TestHDFSServerPorts extends TestCase {
   }
 
   /**
-   * Start the data-node.
+   * Start the BackupNode
+   */
+  public BackupNode startBackupNode(Configuration conf) throws IOException {
+    String dataDir = getTestingDir();
+    // Set up testing environment directories
+    hdfsDir = new File(dataDir, "backupNode");
+    if ( hdfsDir.exists() && !FileUtil.fullyDelete(hdfsDir) ) {
+      throw new IOException("Could not delete hdfs directory '" + hdfsDir + "'");
+    }
+    File currDir = new File(hdfsDir, "name2");
+    File currDir2 = new File(currDir, "current");
+    File currDir3 = new File(currDir, "image");
+    
+    assertTrue(currDir.mkdirs());
+    assertTrue(currDir2.mkdirs());
+    assertTrue(currDir3.mkdirs());
+    
+    conf.set("dfs.name.dir", new File(hdfsDir, "name2").getPath());
+    conf.set("dfs.name.edits.dir", "${dfs.name.dir}");
+    
+    // Start BackupNode
+    String[] args = new String [] { StartupOption.BACKUP.getName() };
+    BackupNode bu = (BackupNode)NameNode.createNameNode(args, conf);
+
+    return bu;
+  }
+  
+  /**
+   * Start the datanode.
    */
   public DataNode startDataNode(int index, Configuration config) 
   throws IOException {
-    String dataDir = System.getProperty("test.build.data");
+    String dataDir = getTestingDir();
     File dataNodeDir = new File(dataDir, "data-" + index);
     config.set("dfs.data.dir", dataNodeDir.getPath());
 
@@ -100,7 +163,7 @@ public class TestHDFSServerPorts extends TestCase {
   }
 
   /**
-   * Check whether the name-node can be started.
+   * Check whether the namenode can be started.
    */
   private boolean canStartNameNode(Configuration conf) throws IOException {
     NameNode nn2 = null;
@@ -110,13 +173,14 @@ public class TestHDFSServerPorts extends TestCase {
       if (e instanceof java.net.BindException)
         return false;
       throw e;
+    } finally {
+      stopNameNode(nn2);
     }
-    stopNameNode(nn2);
     return true;
   }
 
   /**
-   * Check whether the data-node can be started.
+   * Check whether the datanode can be started.
    */
   private boolean canStartDataNode(Configuration conf) throws IOException {
     DataNode dn = null;
@@ -126,29 +190,53 @@ public class TestHDFSServerPorts extends TestCase {
       if (e instanceof java.net.BindException)
         return false;
       throw e;
+    } finally {
+      if(dn != null) dn.shutdown();
     }
-    dn.shutdown();
     return true;
   }
 
   /**
    * Check whether the secondary name-node can be started.
    */
+  @SuppressWarnings("deprecation")
   private boolean canStartSecondaryNode(Configuration conf) throws IOException {
-    SecondaryNameNode sn = null;
+    // Using full name allows us not to have to add deprecation tag to
+    // entire source file.
+    org.apache.hadoop.hdfs.server.namenode.SecondaryNameNode sn = null;
     try {
-      sn = new SecondaryNameNode(conf);
+      sn = new org.apache.hadoop.hdfs.server.namenode.SecondaryNameNode(conf);
     } catch(IOException e) {
       if (e instanceof java.net.BindException)
         return false;
       throw e;
+    } finally {
+      if(sn != null) sn.shutdown();
     }
-    sn.shutdown();
+    return true;
+  }
+  
+  /**
+   * Check whether the BackupNode can be started.
+   */
+  private boolean canStartBackupNode(Configuration conf) throws IOException {
+    BackupNode bn = null;
+
+    try {
+      bn = startBackupNode(conf);
+    } catch(IOException e) {
+      if (e instanceof java.net.BindException)
+        return false;
+      throw e;
+    } finally {
+      if(bn != null) bn.stop();
+    }
+
     return true;
   }
 
   /**
-   * Verify name-node port usage.
+   * Verify namenode port usage.
    */
   public void testNameNodePorts() throws Exception {
     NameNode nn = null;
@@ -179,7 +267,7 @@ public class TestHDFSServerPorts extends TestCase {
   }
 
   /**
-   * Verify data-node port usage.
+   * Verify datanode port usage.
    */
   public void testDataNodePorts() throws Exception {
     NameNode nn = null;
@@ -214,7 +302,7 @@ public class TestHDFSServerPorts extends TestCase {
   }
 
   /**
-   * Verify secondary name-node port usage.
+   * Verify secondary namenode port usage.
    */
   public void testSecondaryNodePorts() throws Exception {
     NameNode nn = null;
@@ -225,14 +313,14 @@ public class TestHDFSServerPorts extends TestCase {
       Configuration conf2 = new Configuration(config);
       conf2.set("dfs.secondary.http.address", 
                 config.get("dfs.http.address"));
-      SecondaryNameNode.LOG.info("= Starting 1 on: " + 
+      LOG.info("= Starting 1 on: " + 
                                  conf2.get("dfs.secondary.http.address"));
       boolean started = canStartSecondaryNode(conf2);
       assertFalse(started); // should fail
 
       // bind http server to a different port
       conf2.set("dfs.secondary.http.address", NAME_NODE_HTTP_HOST + "0");
-      SecondaryNameNode.LOG.info("= Starting 2 on: " + 
+      LOG.info("= Starting 2 on: " + 
                                  conf2.get("dfs.secondary.http.address"));
       started = canStartSecondaryNode(conf2);
       assertTrue(started); // should start now
@@ -240,4 +328,34 @@ public class TestHDFSServerPorts extends TestCase {
       stopNameNode(nn);
     }
   }
+    
+    /**
+     * Verify BackupNode port usage.
+     */
+    public void testBackupNodePorts() throws Exception {
+      NameNode nn = null;
+      try {
+        nn = startNameNode();
+
+        // bind http server to the same port as name-node
+        Configuration backup_config = new Configuration(config);
+        backup_config.set("dfs.backup.http.address", 
+                                        backup_config.get("dfs.http.address"));
+
+        LOG.info("= Starting 1 on: " + 
+                                  backup_config.get("dfs.backup.http.address"));
+
+        assertFalse("Backup started on same port as Namenode", 
+                           canStartBackupNode(backup_config)); // should fail
+
+        // bind http server to a different port
+        backup_config.set("dfs.backup.http.address", NAME_NODE_HTTP_HOST + "0");
+        LOG.info("= Starting 2 on: " + 
+                                  backup_config.get("dfs.backup.http.address"));
+
+        assertTrue(canStartBackupNode(backup_config)); // should start now
+      } finally {
+        stopNameNode(nn);
+      }
+  }
 }

+ 0 - 1
src/test/hdfs/org/apache/hadoop/hdfs/TestModTime.java

@@ -31,7 +31,6 @@ import org.apache.hadoop.fs.FileStatus;
 
 /**
  * This class tests the decommissioning of nodes.
- * @author Dhruba Borthakur
  */
 public class TestModTime extends TestCase {
   static final long seed = 0xDEADBEEFL;

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

@@ -49,7 +49,7 @@ public class TestSeekBug extends TestCase {
   
   private void checkAndEraseData(byte[] actual, int from, byte[] expected, String message) {
     for (int idx = 0; idx < actual.length; idx++) {
-      this.assertEquals(message+" byte "+(from+idx)+" differs. expected "+
+      assertEquals(message+" byte "+(from+idx)+" differs. expected "+
                         expected[from+idx]+" actual "+actual[idx],
                         actual[idx], expected[from+idx]);
       actual[idx] = 0;

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

@@ -52,7 +52,7 @@ public class TestSmallBlock extends TestCase {
   
   private void checkAndEraseData(byte[] actual, int from, byte[] expected, String message) {
     for (int idx = 0; idx < actual.length; idx++) {
-      this.assertEquals(message+" byte "+(from+idx)+" differs. expected "+
+      assertEquals(message+" byte "+(from+idx)+" differs. expected "+
                         expected[from+idx]+" actual "+actual[idx],
                         actual[idx], expected[from+idx]);
       actual[idx] = 0;

+ 8 - 3
src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/NNThroughputBenchmark.java

@@ -22,6 +22,7 @@ import java.io.FileOutputStream;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.EnumSet;
 import java.util.List;
 
 import javax.security.auth.login.LoginException;
@@ -30,6 +31,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.impl.Log4JLogger;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
@@ -43,6 +45,7 @@ import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
+import org.apache.hadoop.io.EnumSetWritable;
 import org.apache.hadoop.net.DNS;
 import org.apache.hadoop.net.NetworkTopology;
 import org.apache.hadoop.security.UnixUserGroupInformation;
@@ -512,7 +515,8 @@ public class NNThroughputBenchmark {
       long start = System.currentTimeMillis();
       // dummyActionNoSynch(fileIdx);
       nameNode.create(fileNames[daemonId][inputIdx], FsPermission.getDefault(),
-                      clientName, true, replication, BLOCK_SIZE);
+                      clientName, new EnumSetWritable<CreateFlag>(EnumSet
+              .of(CreateFlag.OVERWRITE)), replication, BLOCK_SIZE);
       long end = System.currentTimeMillis();
       for(boolean written = !closeUponCreate; !written; 
         written = nameNode.complete(fileNames[daemonId][inputIdx], clientName));
@@ -882,8 +886,9 @@ public class NNThroughputBenchmark {
       nameNode.setSafeMode(FSConstants.SafeModeAction.SAFEMODE_LEAVE);
       for(int idx=0; idx < nrFiles; idx++) {
         String fileName = nameGenerator.getNextFileName("ThroughputBench");
-        nameNode.create(fileName, FsPermission.getDefault(),
-                        clientName, true, replication, BLOCK_SIZE);
+        nameNode.create(fileName, FsPermission.getDefault(), clientName,
+            new EnumSetWritable<CreateFlag>(EnumSet.of(CreateFlag.OVERWRITE)), replication,
+            BLOCK_SIZE);
         addBlocks(fileName, clientName);
         nameNode.complete(fileName, clientName);
       }

+ 9 - 0
src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java

@@ -134,6 +134,7 @@ public class TestCheckpoint extends TestCase {
   /*
    * Simulate namenode crashing after rolling edit log.
    */
+  @SuppressWarnings("deprecation")
   private void testSecondaryNamenodeError1(Configuration conf)
     throws IOException {
     System.out.println("Starting testSecondaryNamenodeError 1");
@@ -210,6 +211,7 @@ public class TestCheckpoint extends TestCase {
   /*
    * Simulate a namenode crash after uploading new image
    */
+  @SuppressWarnings("deprecation")
   private void testSecondaryNamenodeError2(Configuration conf)
     throws IOException {
     System.out.println("Starting testSecondaryNamenodeError 21");
@@ -268,6 +270,7 @@ public class TestCheckpoint extends TestCase {
   /*
    * Simulate a secondary namenode crash after rolling the edit log.
    */
+  @SuppressWarnings("deprecation")
   private void testSecondaryNamenodeError3(Configuration conf)
     throws IOException {
     System.out.println("Starting testSecondaryNamenodeError 31");
@@ -336,6 +339,7 @@ public class TestCheckpoint extends TestCase {
    * back to the name-node.
    * Used to truncate primary fsimage file.
    */
+  @SuppressWarnings("deprecation")
   void testSecondaryFailsToReturnImage(Configuration conf)
     throws IOException {
     System.out.println("Starting testSecondaryFailsToReturnImage");
@@ -399,6 +403,7 @@ public class TestCheckpoint extends TestCase {
    * <li> Complete failed checkpoint for secondary node.
    * </ol>
    */
+  @SuppressWarnings("deprecation")
   void testStartup(Configuration conf) throws IOException {
     System.out.println("Startup of the name-node in the checkpoint directory.");
     String primaryDirs = conf.get("dfs.name.dir");
@@ -553,6 +558,9 @@ public class TestCheckpoint extends TestCase {
     return nn;
   }
 
+  // This deprecation suppress warning does not work due to known Java bug:
+  // http://bugs.sun.com/view_bug.do?bug_id=6460147
+  @SuppressWarnings("deprecation")
   SecondaryNameNode startSecondaryNameNode(Configuration conf
                                           ) throws IOException {
     conf.set("dfs.secondary.http.address", "0.0.0.0:0");
@@ -562,6 +570,7 @@ public class TestCheckpoint extends TestCase {
   /**
    * Tests checkpoint in HDFS.
    */
+  @SuppressWarnings("deprecation")
   public void testCheckpoint() throws IOException {
     Path file1 = new Path("checkpoint.dat");
     Path file2 = new Path("checkpoint2.dat");

+ 4 - 0
src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestNameEditsConfigs.java

@@ -88,6 +88,9 @@ public class TestNameEditsConfigs extends TestCase {
     assertTrue(!fileSys.exists(name));
   }
 
+  // This deprecation suppress warning does not work due to known Java bug:
+  // http://bugs.sun.com/view_bug.do?bug_id=6460147
+  @SuppressWarnings("deprecation")
   SecondaryNameNode startSecondaryNameNode(Configuration conf
                                           ) throws IOException {
     conf.set("dfs.secondary.http.address", "0.0.0.0:0");
@@ -106,6 +109,7 @@ public class TestNameEditsConfigs extends TestCase {
    * All along the test, we create and delete files at reach restart to make
    * sure we are reading proper edits and image.
    */
+  @SuppressWarnings("deprecation")
   public void testNameEditsConfigs() throws IOException {
     Path file1 = new Path("TestNameEditsConfigs1");
     Path file2 = new Path("TestNameEditsConfigs2");

+ 2 - 0
src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestStartup.java

@@ -83,6 +83,7 @@ public class TestStartup extends TestCase {
    * start MiniDFScluster, create a file (to create edits) and do a checkpoint  
    * @throws IOException
    */
+  @SuppressWarnings("deprecation")
   public void createCheckPoint() throws IOException {
     LOG.info("--starting mini cluster");
     // manage dirs parameter set to false 
@@ -248,6 +249,7 @@ public class TestStartup extends TestCase {
    * secondary node copies fsimage and edits into correct separate directories.
    * @throws IOException
    */
+  @SuppressWarnings("deprecation")
   public void testSNNStartup() throws IOException{
     //setUpConfig();
     LOG.info("--starting SecondNN startup test");

+ 1 - 0
src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestStorageRestore.java

@@ -282,6 +282,7 @@ public class TestStorageRestore extends TestCase {
    * 7. run doCheckpoint
    * 8. verify that all the image and edits files are the same.
    */
+  @SuppressWarnings("deprecation")
   public void testStorageRestore() throws Exception {
     int numDatanodes = 2;
     cluster = new MiniDFSCluster(0, config, numDatanodes, true, false, true,  null, null, null, null);

+ 7 - 367
src/webapps/datanode/browseBlock.jsp

@@ -19,382 +19,22 @@
 %>
 <%@ page
   contentType="text/html; charset=UTF-8"
-  import="javax.servlet.*"
-  import="javax.servlet.http.*"
-  import="java.io.*"
-  import="java.util.*"
-  import="java.net.*"
-
-  import="org.apache.hadoop.hdfs.*"
-  import="org.apache.hadoop.hdfs.server.namenode.*"
-  import="org.apache.hadoop.hdfs.protocol.*"
-  import="org.apache.hadoop.security.AccessToken"
-  import="org.apache.hadoop.security.AccessTokenHandler"
-  import="org.apache.hadoop.util.*"
+  import="org.apache.hadoop.hdfs.server.common.JspHelper"
+  import="org.apache.hadoop.util.ServletUtil"
 %>
-
 <%!
-  static final DataNode datanode = DataNode.getDataNode();
-
-  public void generateFileDetails(JspWriter out, HttpServletRequest req) 
-    throws IOException {
-
-    long startOffset = 0;
-    int datanodePort;
-
-    final Long blockId = JspHelper.validateLong(req.getParameter("blockId"));
-    if (blockId == null) {
-      out.print("Invalid input (blockId absent)");
-      return;
-    }
-
-    String datanodePortStr = req.getParameter("datanodePort");
-    if (datanodePortStr == null) {
-      out.print("Invalid input (datanodePort absent)");
-      return;
-    }
-    datanodePort = Integer.parseInt(datanodePortStr);
-
-    String namenodeInfoPortStr = req.getParameter("namenodeInfoPort");
-    int namenodeInfoPort = -1;
-    if (namenodeInfoPortStr != null)
-      namenodeInfoPort = Integer.parseInt(namenodeInfoPortStr);
-
-    final int chunkSizeToView = JspHelper.string2ChunkSizeToView(req.getParameter("chunkSizeToView"));
-
-    String startOffsetStr = req.getParameter("startOffset");
-    if (startOffsetStr == null || Long.parseLong(startOffsetStr) < 0)
-      startOffset = 0;
-    else startOffset = Long.parseLong(startOffsetStr);
-    
-    final String filename = JspHelper.validatePath(
-        req.getParameter("filename"));
-    if (filename == null) {
-      out.print("Invalid input");
-      return;
-    }
-
-    String blockSizeStr = req.getParameter("blockSize"); 
-    long blockSize = 0;
-    if (blockSizeStr == null || blockSizeStr.length() == 0) {
-      out.print("Invalid input");
-      return;
-    } 
-    blockSize = Long.parseLong(blockSizeStr);
-
-    final DFSClient dfs = new DFSClient(datanode.getNameNodeAddr(), JspHelper.conf);
-    List<LocatedBlock> blocks = 
-      dfs.namenode.getBlockLocations(filename, 0, Long.MAX_VALUE).getLocatedBlocks();
-    //Add the various links for looking at the file contents
-    //URL for downloading the full file
-    String downloadUrl = "http://" + req.getServerName() + ":" +
-                         + req.getServerPort() + "/streamFile?" + "filename=" +
-                         URLEncoder.encode(filename, "UTF-8");
-    out.print("<a name=\"viewOptions\"></a>");
-    out.print("<a href=\"" + downloadUrl + "\">Download this file</a><br>");
-    
-    DatanodeInfo chosenNode;
-    //URL for TAIL 
-    LocatedBlock lastBlk = blocks.get(blocks.size() - 1);
-    try {
-      chosenNode = JspHelper.bestNode(lastBlk);
-    } catch (IOException e) {
-      out.print(e.toString());
-      dfs.close();
-      return;
-    }
-    String fqdn = 
-           InetAddress.getByName(chosenNode.getHost()).getCanonicalHostName();
-    String tailUrl = "http://" + fqdn + ":" +
-                     chosenNode.getInfoPort() + 
-                 "/tail.jsp?filename=" + URLEncoder.encode(filename, "UTF-8") +
-                 "&namenodeInfoPort=" + namenodeInfoPort +
-                 "&chunkSizeToView=" + chunkSizeToView +
-                 "&referrer=" + 
-          URLEncoder.encode(req.getRequestURL() + "?" + req.getQueryString(),
-                            "UTF-8");
-    out.print("<a href=\"" + tailUrl + "\">Tail this file</a><br>");
-
-    out.print("<form action=\"/browseBlock.jsp\" method=GET>");
-    out.print("<b>Chunk size to view (in bytes, up to file's DFS block size): </b>");
-    out.print("<input type=\"hidden\" name=\"blockId\" value=\"" + blockId +
-              "\">");
-    out.print("<input type=\"hidden\" name=\"blockSize\" value=\"" + 
-              blockSize + "\">");
-    out.print("<input type=\"hidden\" name=\"startOffset\" value=\"" + 
-              startOffset + "\">");
-    out.print("<input type=\"hidden\" name=\"filename\" value=\"" + filename +
-              "\">");
-    out.print("<input type=\"hidden\" name=\"datanodePort\" value=\"" + 
-              datanodePort+ "\">");
-    out.print("<input type=\"hidden\" name=\"namenodeInfoPort\" value=\"" +
-              namenodeInfoPort + "\">");
-    out.print("<input type=\"text\" name=\"chunkSizeToView\" value=" +
-              chunkSizeToView + " size=10 maxlength=10>");
-    out.print("&nbsp;&nbsp;<input type=\"submit\" name=\"submit\" value=\"Refresh\">");
-    out.print("</form>");
-    out.print("<hr>"); 
-    out.print("<a name=\"blockDetails\"></a>");
-    out.print("<B>Total number of blocks: "+blocks.size()+"</B><br>");
-    //generate a table and dump the info
-    out.println("\n<table>");
-    for (LocatedBlock cur : blocks) {
-      out.print("<tr>");
-      final String blockidstring = Long.toString(cur.getBlock().getBlockId());
-      blockSize = cur.getBlock().getNumBytes();
-      out.print("<td>"+blockidstring+":</td>");
-      DatanodeInfo[] locs = cur.getLocations();
-      for(int j=0; j<locs.length; j++) {
-        String datanodeAddr = locs[j].getName();
-        datanodePort = Integer.parseInt(datanodeAddr.substring(
-                                        datanodeAddr.indexOf(':') + 1, 
-                                    datanodeAddr.length())); 
-        fqdn = InetAddress.getByName(locs[j].getHost()).getCanonicalHostName();
-        String blockUrl = "http://"+ fqdn + ":" +
-                        locs[j].getInfoPort() +
-                        "/browseBlock.jsp?blockId=" + blockidstring +
-                        "&blockSize=" + blockSize +
-               "&filename=" + URLEncoder.encode(filename, "UTF-8")+ 
-                        "&datanodePort=" + datanodePort + 
-                        "&genstamp=" + cur.getBlock().getGenerationStamp() + 
-                        "&namenodeInfoPort=" + namenodeInfoPort +
-                        "&chunkSizeToView=" + chunkSizeToView;
-        out.print("<td>&nbsp</td>" 
-          + "<td><a href=\"" + blockUrl + "\">" + datanodeAddr + "</a></td>");
-      }
-      out.println("</tr>");
-    }
-    out.println("</table>");
-    out.print("<hr>");
-    String namenodeHost = datanode.getNameNodeAddr().getHostName();
-    out.print("<br><a href=\"http://" + 
-              InetAddress.getByName(namenodeHost).getCanonicalHostName() + ":" +
-              namenodeInfoPort + "/dfshealth.jsp\">Go back to DFS home</a>");
-    dfs.close();
-  }
-
-  public void generateFileChunks(JspWriter out, HttpServletRequest req) 
-    throws IOException {
-    long startOffset = 0;
-    int datanodePort = 0; 
-
-    String namenodeInfoPortStr = req.getParameter("namenodeInfoPort");
-    int namenodeInfoPort = -1;
-    if (namenodeInfoPortStr != null)
-      namenodeInfoPort = Integer.parseInt(namenodeInfoPortStr);
-
-    final String filename = JspHelper.validatePath(
-        req.getParameter("filename"));
-    if (filename == null) {
-      out.print("Invalid input (filename absent)");
-      return;
-    }
-    
-    final Long blockId = JspHelper.validateLong(req.getParameter("blockId"));
-    if (blockId == null) {
-      out.print("Invalid input (blockId absent)");
-      return;
-    }
-
-    final DFSClient dfs = new DFSClient(datanode.getNameNodeAddr(), JspHelper.conf);
-    
-    AccessToken accessToken = AccessToken.DUMMY_TOKEN;
-    if (JspHelper.conf
-        .getBoolean(AccessTokenHandler.STRING_ENABLE_ACCESS_TOKEN, false)) {
-      List<LocatedBlock> blks = dfs.namenode.getBlockLocations(filename, 0,
-          Long.MAX_VALUE).getLocatedBlocks();
-      if (blks == null || blks.size() == 0) {
-        out.print("Can't locate file blocks");
-        dfs.close();
-        return;
-      }
-      for (int i = 0; i < blks.size(); i++) {
-        if (blks.get(i).getBlock().getBlockId() == blockId) {
-          accessToken = blks.get(i).getAccessToken();
-          break;
-        }
-      }
-    }
-    
-    final Long genStamp = JspHelper.validateLong(req.getParameter("genstamp"));
-    if (genStamp == null) {
-      out.print("Invalid input (genstamp absent)");
-      return;
-    }
-
-    String blockSizeStr;
-    long blockSize = 0;
-    blockSizeStr = req.getParameter("blockSize"); 
-    if (blockSizeStr == null) {
-      out.print("Invalid input (blockSize absent)");
-      return;
-    }
-    blockSize = Long.parseLong(blockSizeStr);
-    
-    final int chunkSizeToView = JspHelper.string2ChunkSizeToView(req.getParameter("chunkSizeToView"));
-
-    String startOffsetStr = req.getParameter("startOffset");
-    if (startOffsetStr == null || Long.parseLong(startOffsetStr) < 0)
-      startOffset = 0;
-    else startOffset = Long.parseLong(startOffsetStr);
-
-    String datanodePortStr = req.getParameter("datanodePort");
-    if (datanodePortStr == null) {
-      out.print("Invalid input (datanodePort absent)");
-      return;
-    }
-    datanodePort = Integer.parseInt(datanodePortStr);
-    out.print("<h3>File: ");
-    JspHelper.printPathWithLinks(filename, out, namenodeInfoPort);
-    out.print("</h3><hr>");
-    String parent = new File(filename).getParent();
-    JspHelper.printGotoForm(out, namenodeInfoPort, parent);
-    out.print("<hr>");
-    out.print("<a href=\"http://" + req.getServerName() + ":" + 
-              req.getServerPort() + 
-              "/browseDirectory.jsp?dir=" + 
-              URLEncoder.encode(parent, "UTF-8") +
-              "&namenodeInfoPort=" + namenodeInfoPort + 
-              "\"><i>Go back to dir listing</i></a><br>");
-    out.print("<a href=\"#viewOptions\">Advanced view/download options</a><br>");
-    out.print("<hr>");
-
-    //Determine the prev & next blocks
-    long nextStartOffset = 0;
-    long nextBlockSize = 0;
-    String nextBlockIdStr = null;
-    String nextGenStamp = null;
-    String nextHost = req.getServerName();
-    int nextPort = req.getServerPort();
-    int nextDatanodePort = datanodePort;
-    //determine data for the next link
-    if (startOffset + chunkSizeToView >= blockSize) {
-      //we have to go to the next block from this point onwards
-      List<LocatedBlock> blocks = 
-        dfs.namenode.getBlockLocations(filename, 0, Long.MAX_VALUE).getLocatedBlocks();
-      for (int i = 0; i < blocks.size(); i++) {
-        if (blocks.get(i).getBlock().getBlockId() == blockId) {
-          if (i != blocks.size() - 1) {
-            LocatedBlock nextBlock = blocks.get(i+1);
-            nextBlockIdStr = Long.toString(nextBlock.getBlock().getBlockId());
-            nextGenStamp = Long.toString(nextBlock.getBlock().getGenerationStamp());
-            nextStartOffset = 0;
-            nextBlockSize = nextBlock.getBlock().getNumBytes();
-            DatanodeInfo d = JspHelper.bestNode(nextBlock);
-            String datanodeAddr = d.getName();
-            nextDatanodePort = Integer.parseInt(
-                                      datanodeAddr.substring(
-                                           datanodeAddr.indexOf(':') + 1, 
-                                      datanodeAddr.length())); 
-            nextHost = InetAddress.getByName(d.getHost()).getCanonicalHostName();
-            nextPort = d.getInfoPort(); 
-          }
-        }
-      }
-    } 
-    else {
-      //we are in the same block
-      nextBlockIdStr = blockId.toString();
-      nextStartOffset = startOffset + chunkSizeToView;
-      nextBlockSize = blockSize;
-      nextGenStamp = genStamp.toString();
-    }
-    String nextUrl = null;
-    if (nextBlockIdStr != null) {
-      nextUrl = "http://" + nextHost + ":" + 
-                nextPort + 
-                "/browseBlock.jsp?blockId=" + nextBlockIdStr +
-                "&blockSize=" + nextBlockSize + "&startOffset=" + 
-                nextStartOffset + 
-                "&genstamp=" + nextGenStamp +
-                "&filename=" + URLEncoder.encode(filename, "UTF-8") +
-                "&chunkSizeToView=" + chunkSizeToView + 
-                "&datanodePort=" + nextDatanodePort +
-                "&namenodeInfoPort=" + namenodeInfoPort;
-      out.print("<a href=\"" + nextUrl + "\">View Next chunk</a>&nbsp;&nbsp;");        
-    }
-    //determine data for the prev link
-    String prevBlockIdStr = null;
-    String prevGenStamp = null;
-    long prevStartOffset = 0;
-    long prevBlockSize = 0;
-    String prevHost = req.getServerName();
-    int prevPort = req.getServerPort();
-    int prevDatanodePort = datanodePort;
-    if (startOffset == 0) {
-      List<LocatedBlock> blocks = 
-        dfs.namenode.getBlockLocations(filename, 0, Long.MAX_VALUE).getLocatedBlocks();
-      for (int i = 0; i < blocks.size(); i++) {
-        if (blocks.get(i).getBlock().getBlockId() == blockId) {
-          if (i != 0) {
-            LocatedBlock prevBlock = blocks.get(i-1);
-            prevBlockIdStr = Long.toString(prevBlock.getBlock().getBlockId());
-            prevGenStamp = Long.toString(prevBlock.getBlock().getGenerationStamp());
-            prevStartOffset = prevBlock.getBlock().getNumBytes() - chunkSizeToView;
-            if (prevStartOffset < 0)
-              prevStartOffset = 0;
-            prevBlockSize = prevBlock.getBlock().getNumBytes();
-            DatanodeInfo d = JspHelper.bestNode(prevBlock);
-            String datanodeAddr = d.getName();
-            prevDatanodePort = Integer.parseInt(
-                                      datanodeAddr.substring(
-                                          datanodeAddr.indexOf(':') + 1, 
-                                      datanodeAddr.length())); 
-            prevHost = InetAddress.getByName(d.getHost()).getCanonicalHostName();
-            prevPort = d.getInfoPort();
-          }
-        }
-      }
-    }
-    else {
-      //we are in the same block
-      prevBlockIdStr = blockId.toString();
-      prevStartOffset = startOffset - chunkSizeToView;
-      if (prevStartOffset < 0) prevStartOffset = 0;
-      prevBlockSize = blockSize;
-      prevGenStamp = genStamp.toString();
-    }
-
-    String prevUrl = null;
-    if (prevBlockIdStr != null) {
-      prevUrl = "http://" + prevHost + ":" + 
-                prevPort + 
-                "/browseBlock.jsp?blockId=" + prevBlockIdStr + 
-                "&blockSize=" + prevBlockSize + "&startOffset=" + 
-                prevStartOffset + 
-                "&filename=" + URLEncoder.encode(filename, "UTF-8") + 
-                "&chunkSizeToView=" + chunkSizeToView +
-                "&genstamp=" + prevGenStamp +
-                "&datanodePort=" + prevDatanodePort +
-                "&namenodeInfoPort=" + namenodeInfoPort;
-      out.print("<a href=\"" + prevUrl + "\">View Prev chunk</a>&nbsp;&nbsp;");
-    }
-    out.print("<hr>");
-    out.print("<textarea cols=\"100\" rows=\"25\" wrap=\"virtual\" style=\"width:100%\" READONLY>");
-    try {
-    JspHelper.streamBlockInAscii(
-            new InetSocketAddress(req.getServerName(), datanodePort), blockId, 
-            accessToken, genStamp, blockSize, startOffset, chunkSizeToView, out);
-    } catch (Exception e){
-        out.print(e);
-    }
-    out.print("</textarea>");
-    dfs.close();
-  }
-
+  //for java.io.Serializable
+  private static final long serialVersionUID = 1L;
 %>
+
 <html>
 <head>
 <%JspHelper.createTitle(out, request, request.getParameter("filename")); %>
 </head>
 <body onload="document.goto.dir.focus()">
-<% 
-   generateFileChunks(out,request);
-%>
+<% DatanodeJspHelper.generateFileChunks(out,request); %>
 <hr>
-<% 
-   generateFileDetails(out,request);
-%>
+<% DatanodeJspHelper.generateFileDetails(out,request); %>
 
 <h2>Local logs</h2>
 <a href="/logs/">Log</a> directory

+ 6 - 132
src/webapps/datanode/browseDirectory.jsp

@@ -19,140 +19,14 @@
 %>
 <%@ page
   contentType="text/html; charset=UTF-8"
-  import="javax.servlet.*"
-  import="javax.servlet.http.*"
-  import="java.io.*"
-  import="java.util.*"
-  import="java.net.*"
+  import="java.io.IOException"
 
-  import="org.apache.hadoop.fs.*"
-  import="org.apache.hadoop.hdfs.*"
-  import="org.apache.hadoop.hdfs.server.namenode.*"
-  import="org.apache.hadoop.hdfs.protocol.*"
-  import="org.apache.hadoop.util.*"
+  import="org.apache.hadoop.hdfs.server.common.JspHelper"
+  import="org.apache.hadoop.util.ServletUtil"
 %>
 <%!
-  static final DataNode datanode = DataNode.getDataNode();
-  
-  public void generateDirectoryStructure( JspWriter out, 
-                                          HttpServletRequest req,
-                                          HttpServletResponse resp) 
-    throws IOException {
-    final String dir = JspHelper.validatePath(req.getParameter("dir"));
-    if (dir == null) {
-      out.print("Invalid input");
-      return;
-    }
-    
-    String namenodeInfoPortStr = req.getParameter("namenodeInfoPort");
-    int namenodeInfoPort = -1;
-    if (namenodeInfoPortStr != null)
-      namenodeInfoPort = Integer.parseInt(namenodeInfoPortStr);
-    
-    final DFSClient dfs = new DFSClient(datanode.getNameNodeAddr(), JspHelper.conf);
-    String target = dir;
-    final FileStatus targetStatus = dfs.getFileInfo(target);
-    if (targetStatus == null) { // not exists
-      out.print("<h3>File or directory : " + target + " does not exist</h3>");
-      JspHelper.printGotoForm(out, namenodeInfoPort, target);
-    }
-    else {
-      if( !targetStatus.isDir() ) { // a file
-        List<LocatedBlock> blocks = 
-          dfs.namenode.getBlockLocations(dir, 0, 1).getLocatedBlocks();
-	      
-        LocatedBlock firstBlock = null;
-        DatanodeInfo [] locations = null;
-        if (blocks.size() > 0) {
-          firstBlock = blocks.get(0);
-          locations = firstBlock.getLocations();
-        }
-        if (locations == null || locations.length == 0) {
-          out.print("Empty file");
-        } else {
-          DatanodeInfo chosenNode = JspHelper.bestNode(firstBlock);
-          String fqdn = InetAddress.getByName(chosenNode.getHost()).
-            getCanonicalHostName();
-          String datanodeAddr = chosenNode.getName();
-          int datanodePort = Integer.parseInt(
-                                              datanodeAddr.substring(
-                                                                     datanodeAddr.indexOf(':') + 1, 
-                                                                     datanodeAddr.length())); 
-          String redirectLocation = "http://"+fqdn+":" +
-            chosenNode.getInfoPort() + 
-            "/browseBlock.jsp?blockId=" +
-            firstBlock.getBlock().getBlockId() +
-            "&blockSize=" + firstBlock.getBlock().getNumBytes() +
-            "&genstamp=" + firstBlock.getBlock().getGenerationStamp() +
-            "&filename=" + URLEncoder.encode(dir, "UTF-8") + 
-            "&datanodePort=" + datanodePort + 
-            "&namenodeInfoPort=" + namenodeInfoPort;
-          resp.sendRedirect(redirectLocation);
-        }
-        return;
-      }
-      // directory
-      FileStatus[] files = dfs.listPaths(target);
-      //generate a table and dump the info
-      String [] headings = { "Name", "Type", "Size", "Replication", 
-                              "Block Size", "Modification Time",
-                              "Permission", "Owner", "Group" };
-      out.print("<h3>Contents of directory ");
-      JspHelper.printPathWithLinks(dir, out, namenodeInfoPort);
-      out.print("</h3><hr>");
-      JspHelper.printGotoForm(out, namenodeInfoPort, dir);
-      out.print("<hr>");
-	
-      File f = new File(dir);
-      String parent;
-      if ((parent = f.getParent()) != null)
-        out.print("<a href=\"" + req.getRequestURL() + "?dir=" + parent +
-                  "&namenodeInfoPort=" + namenodeInfoPort +
-                  "\">Go to parent directory</a><br>");
-	
-      if (files == null || files.length == 0) {
-        out.print("Empty directory");
-      }
-      else {
-        JspHelper.addTableHeader(out);
-        int row=0;
-        JspHelper.addTableRow(out, headings, row++);
-        String cols [] = new String[headings.length];
-        for (int i = 0; i < files.length; i++) {
-          //Get the location of the first block of the file
-          if (files[i].getPath().toString().endsWith(".crc")) continue;
-          if (!files[i].isDir()) {
-            cols[1] = "file";
-            cols[2] = StringUtils.byteDesc(files[i].getLen());
-            cols[3] = Short.toString(files[i].getReplication());
-            cols[4] = StringUtils.byteDesc(files[i].getBlockSize());
-          }
-          else {
-            cols[1] = "dir";
-            cols[2] = "";
-            cols[3] = "";
-            cols[4] = "";
-          }
-          String datanodeUrl = req.getRequestURL()+"?dir="+
-              URLEncoder.encode(files[i].getPath().toString(), "UTF-8") + 
-              "&namenodeInfoPort=" + namenodeInfoPort;
-          cols[0] = "<a href=\""+datanodeUrl+"\">"+files[i].getPath().getName()+"</a>";
-          cols[5] = FsShell.dateForm.format(new Date((files[i].getModificationTime())));
-          cols[6] = files[i].getPermission().toString();
-          cols[7] = files[i].getOwner();
-          cols[8] = files[i].getGroup();
-          JspHelper.addTableRow(out, cols, row++);
-        }
-        JspHelper.addTableFooter(out);
-      }
-    } 
-    String namenodeHost = datanode.getNameNodeAddr().getHostName();
-    out.print("<br><a href=\"http://" + 
-              InetAddress.getByName(namenodeHost).getCanonicalHostName() + ":" +
-              namenodeInfoPort + "/dfshealth.jsp\">Go back to DFS home</a>");
-    dfs.close();
-  }
-
+  //for java.io.Serializable
+  private static final long serialVersionUID = 1L;
 %>
 
 <html>
@@ -171,7 +45,7 @@ body
 <body onload="document.goto.dir.focus()">
 <% 
   try {
-    generateDirectoryStructure(out,request,response);
+    DatanodeJspHelper.generateDirectoryStructure(out,request,response);
   }
   catch(IOException ioe) {
     String msg = ioe.getLocalizedMessage();

+ 5 - 96
src/webapps/datanode/tail.jsp

@@ -19,111 +19,20 @@
 %>
 <%@ page
   contentType="text/html; charset=UTF-8"
-  import="javax.servlet.*"
-  import="javax.servlet.http.*"
-  import="java.io.*"
-  import="java.util.*"
-  import="java.net.*"
-
-  import="org.apache.hadoop.hdfs.*"
-  import="org.apache.hadoop.hdfs.server.namenode.*"
-  import="org.apache.hadoop.hdfs.protocol.*"
-  import="org.apache.hadoop.security.AccessToken"
-  import="org.apache.hadoop.util.*"
-  import="org.apache.hadoop.net.NetUtils"
+  import="org.apache.hadoop.hdfs.server.common.JspHelper"
+  import="org.apache.hadoop.util.ServletUtil"
 %>
-
 <%!
-  static final DataNode datanode = DataNode.getDataNode();
-
-  public void generateFileChunks(JspWriter out, HttpServletRequest req) 
-    throws IOException {
-    final String referrer = JspHelper.validateURL(req.getParameter("referrer"));
-    boolean noLink = false;
-    if (referrer == null) {
-      noLink = true;
-    }
-
-    final String filename = JspHelper.validatePath(
-        req.getParameter("filename"));
-    if (filename == null) {
-      out.print("Invalid input (file name absent)");
-      return;
-    }
-
-    String namenodeInfoPortStr = req.getParameter("namenodeInfoPort");
-    int namenodeInfoPort = -1;
-    if (namenodeInfoPortStr != null)
-      namenodeInfoPort = Integer.parseInt(namenodeInfoPortStr);
-    
-    final int chunkSizeToView = JspHelper.string2ChunkSizeToView(req.getParameter("chunkSizeToView"));
-
-    if (!noLink) {
-      out.print("<h3>Tail of File: ");
-      JspHelper.printPathWithLinks(filename, out, namenodeInfoPort);
-	    out.print("</h3><hr>");
-      out.print("<a href=\"" + referrer + "\">Go Back to File View</a><hr>");
-    }
-    else {
-      out.print("<h3>" + filename + "</h3>");
-    }
-    out.print("<b>Chunk size to view (in bytes, up to file's DFS block size): </b>");
-    out.print("<input type=\"text\" name=\"chunkSizeToView\" value=" +
-              chunkSizeToView + " size=10 maxlength=10>");
-    out.print("&nbsp;&nbsp;<input type=\"submit\" name=\"submit\" value=\"Refresh\"><hr>");
-    out.print("<input type=\"hidden\" name=\"filename\" value=\"" + filename +
-              "\">");
-    out.print("<input type=\"hidden\" name=\"namenodeInfoPort\" value=\"" + namenodeInfoPort +
-    "\">");
-    if (!noLink)
-      out.print("<input type=\"hidden\" name=\"referrer\" value=\"" + 
-                referrer+ "\">");
-
-    //fetch the block from the datanode that has the last block for this file
-    final DFSClient dfs = new DFSClient(datanode.getNameNodeAddr(), JspHelper.conf);
-    List<LocatedBlock> blocks = 
-      dfs.namenode.getBlockLocations(filename, 0, Long.MAX_VALUE).getLocatedBlocks();
-    if (blocks == null || blocks.size() == 0) {
-      out.print("No datanodes contain blocks of file "+filename);
-      dfs.close();
-      return;
-    }
-    LocatedBlock lastBlk = blocks.get(blocks.size() - 1);
-    long blockSize = lastBlk.getBlock().getNumBytes();
-    long blockId = lastBlk.getBlock().getBlockId();
-    AccessToken accessToken = lastBlk.getAccessToken();
-    long genStamp = lastBlk.getBlock().getGenerationStamp();
-    DatanodeInfo chosenNode;
-    try {
-      chosenNode = JspHelper.bestNode(lastBlk);
-    } catch (IOException e) {
-      out.print(e.toString());
-      dfs.close();
-      return;
-    }      
-    InetSocketAddress addr = NetUtils.createSocketAddr(chosenNode.getName());
-    //view the last chunkSizeToView bytes while Tailing
-    final long startOffset = blockSize >= chunkSizeToView? blockSize - chunkSizeToView: 0;
-
-    out.print("<textarea cols=\"100\" rows=\"25\" wrap=\"virtual\" style=\"width:100%\" READONLY>");
-    JspHelper.streamBlockInAscii(addr, blockId, accessToken, genStamp, blockSize, startOffset, chunkSizeToView, out);
-    out.print("</textarea>");
-    dfs.close();
-  }
-
+  //for java.io.Serializable
+  private static final long serialVersionUID = 1L;
 %>
-
-
-
 <html>
 <head>
 <%JspHelper.createTitle(out, request, request.getParameter("filename")); %>
 </head>
 <body>
 <form action="/tail.jsp" method="GET">
-<% 
-   generateFileChunks(out,request);
-%>
+<% DatanodeJspHelper.generateFileChunksForTail(out,request); %>
 </form>
 <hr>
 

+ 11 - 232
src/webapps/hdfs/dfshealth.jsp

@@ -19,231 +19,14 @@
 %>
 <%@ page
   contentType="text/html; charset=UTF-8"
-  import="javax.servlet.*"
-  import="javax.servlet.http.*"
-  import="java.io.*"
-  import="java.util.*"
-  import="org.apache.hadoop.fs.*"
-  import="org.apache.hadoop.hdfs.*"
-  import="org.apache.hadoop.hdfs.server.namenode.*"
-  import="org.apache.hadoop.hdfs.server.datanode.*"
-  import="org.apache.hadoop.hdfs.server.common.Storage"
-  import="org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory"
-  import="org.apache.hadoop.hdfs.protocol.*"
-  import="org.apache.hadoop.util.*"
-  import="java.text.DateFormat"
-  import="java.lang.Math"
-  import="java.net.URLEncoder"
+  import="org.apache.hadoop.util.ServletUtil"
 %>
 <%!
-  int rowNum = 0;
-  int colNum = 0;
-
-  String rowTxt() { colNum = 0;
-      return "<tr class=\"" + (((rowNum++)%2 == 0)? "rowNormal" : "rowAlt")
-          + "\"> "; }
-  String colTxt() { return "<td id=\"col" + ++colNum + "\"> "; }
-  void counterReset () { colNum = 0; rowNum = 0 ; }
-
-  long diskBytes = 1024 * 1024 * 1024;
-  String diskByteStr = "GB";
-
-  String sorterField = null;
-  String sorterOrder = null;
-
-  String NodeHeaderStr(String name) {
-      String ret = "class=header";
-      String order = "ASC";
-      if ( name.equals( sorterField ) ) {
-          ret += sorterOrder;
-          if ( sorterOrder.equals("ASC") )
-              order = "DSC";
-      }
-      ret += " onClick=\"window.document.location=" +
-          "'/dfshealth.jsp?sorter/field=" + name + "&sorter/order=" +
-          order + "'\" title=\"sort on this column\"";
-      
-      return ret;
-  }
-      
-  public void generateNodeData( JspWriter out, DatanodeDescriptor d,
-                                    String suffix, boolean alive,
-                                    int nnHttpPort )
-    throws IOException {
-      
-    /* Say the datanode is dn1.hadoop.apache.org with ip 192.168.0.5
-       we use:
-       1) d.getHostName():d.getPort() to display.
-           Domain and port are stripped if they are common across the nodes.
-           i.e. "dn1"
-       2) d.getHost():d.Port() for "title".
-          i.e. "192.168.0.5:50010"
-       3) d.getHostName():d.getInfoPort() for url.
-          i.e. "http://dn1.hadoop.apache.org:50075/..."
-          Note that "d.getHost():d.getPort()" is what DFS clients use
-          to interact with datanodes.
-    */
-    // from nn_browsedfscontent.jsp:
-    String url = "http://" + d.getHostName() + ":" + d.getInfoPort() +
-                 "/browseDirectory.jsp?namenodeInfoPort=" +
-                 nnHttpPort + "&dir=" +
-                 URLEncoder.encode("/", "UTF-8");
-     
-    String name = d.getHostName() + ":" + d.getPort();
-    if ( !name.matches( "\\d+\\.\\d+.\\d+\\.\\d+.*" ) ) 
-        name = name.replaceAll( "\\.[^.:]*", "" );    
-    int idx = (suffix != null && name.endsWith( suffix )) ?
-        name.indexOf( suffix ) : -1;
-    
-    out.print( rowTxt() + "<td class=\"name\"><a title=\""
-               + d.getHost() + ":" + d.getPort() +
-               "\" href=\"" + url + "\">" +
-               (( idx > 0 ) ? name.substring(0, idx) : name) + "</a>" +
-               (( alive ) ? "" : "\n") );
-    if ( !alive )
-        return;
-    
-    long c = d.getCapacity();
-    long u = d.getDfsUsed();
-    long nu = d.getNonDfsUsed();
-    long r = d.getRemaining();
-    String percentUsed = StringUtils.limitDecimalTo2(d.getDfsUsedPercent());    
-    String percentRemaining = StringUtils.limitDecimalTo2(d.getRemainingPercent());    
-    
-    String adminState = (d.isDecommissioned() ? "Decommissioned" :
-                         (d.isDecommissionInProgress() ? "Decommission In Progress":
-                          "In Service"));
-    
-    long timestamp = d.getLastUpdate();
-    long currentTime = System.currentTimeMillis();
-    out.print("<td class=\"lastcontact\"> " +
-              ((currentTime - timestamp)/1000) +
-              "<td class=\"adminstate\">" +
-              adminState +
-              "<td align=\"right\" class=\"capacity\">" +
-              StringUtils.limitDecimalTo2(c*1.0/diskBytes) +
-              "<td align=\"right\" class=\"used\">" +
-              StringUtils.limitDecimalTo2(u*1.0/diskBytes) +      
-              "<td align=\"right\" class=\"nondfsused\">" +
-              StringUtils.limitDecimalTo2(nu*1.0/diskBytes) +      
-              "<td align=\"right\" class=\"remaining\">" +
-              StringUtils.limitDecimalTo2(r*1.0/diskBytes) +      
-              "<td align=\"right\" class=\"pcused\">" + percentUsed +
-              "<td class=\"pcused\">" +
-              ServletUtil.percentageGraph( (int)Double.parseDouble(percentUsed) , 100) +
-              "<td align=\"right\" class=\"pcremaining`\">" + percentRemaining +
-              "<td title=" + "\"blocks scheduled : " + d.getBlocksScheduled() + 
-              "\" class=\"blocks\">" + d.numBlocks() + "\n");
-  }
-  
-  
-  public void generateConfReport( JspWriter out,
-		  NameNode nn,
-		  HttpServletRequest request)
-  throws IOException {
-	  FSNamesystem fsn = nn.getNamesystem();
-	  long underReplicatedBlocks = fsn.getUnderReplicatedBlocks();
-	  FSImage fsImage = fsn.getFSImage();
-	  List<Storage.StorageDirectory> removedStorageDirs = fsImage.getRemovedStorageDirs();
-	  String storageDirsSizeStr="", removedStorageDirsSizeStr="", storageDirsStr="", removedStorageDirsStr="", storageDirsDiv="", removedStorageDirsDiv="";
-
-	  //FS Image storage configuration
-	  out.print("<h3> " + nn.getRole() + " Storage: </h3>");
-	  out.print("<div id=\"dfstable\"> <table border=1 cellpadding=10 cellspacing=0 title=\"NameNode Storage\">\n"+
-	  "<thead><tr><td><b>Storage Directory</b></td><td><b>Type</b></td><td><b>State</b></td></tr></thead>");
-	  
-	  StorageDirectory st =null;
-	  for (Iterator<StorageDirectory> it = fsImage.dirIterator(); it.hasNext();) {
-	      st = it.next();
-	      String dir = "" +  st.getRoot();
-		  String type = "" + st.getStorageDirType();
-		  out.print("<tr><td>"+dir+"</td><td>"+type+"</td><td>Active</td></tr>");
-	  }
-	  
-	  long storageDirsSize = removedStorageDirs.size();
-	  for(int i=0; i< storageDirsSize; i++){
-		  st = removedStorageDirs.get(i);
-		  String dir = "" +  st.getRoot();
-		  String type = "" + st.getStorageDirType();
-		  out.print("<tr><td>"+dir+"</td><td>"+type+"</td><td><font color=red>Failed</font></td></tr>");
-	  }
-	  
-	  out.print("</table></div><br>\n");
-  }
-
-
-  public void generateDFSHealthReport(JspWriter out,
-                                      NameNode nn,
-                                      HttpServletRequest request)
-                                      throws IOException {
-    FSNamesystem fsn = nn.getNamesystem();
-    ArrayList<DatanodeDescriptor> live = new ArrayList<DatanodeDescriptor>();
-    ArrayList<DatanodeDescriptor> dead = new ArrayList<DatanodeDescriptor>();
-    fsn.DFSNodesStatus(live, dead);
-
-    sorterField = request.getParameter("sorter/field");
-    sorterOrder = request.getParameter("sorter/order");
-    if ( sorterField == null )
-        sorterField = "name";
-    if ( sorterOrder == null )
-        sorterOrder = "ASC";
-
-    // Find out common suffix. Should this be before or after the sort?
-    String port_suffix = null;
-    if ( live.size() > 0 ) {
-        String name = live.get(0).getName();
-        int idx = name.indexOf(':');
-        if ( idx > 0 ) {
-            port_suffix = name.substring( idx );
-        }
-        
-        for ( int i=1; port_suffix != null && i < live.size(); i++ ) {
-            if ( live.get(i).getName().endsWith( port_suffix ) == false ) {
-                port_suffix = null;
-                break;
-            }
-        }
-    }
-        
-    counterReset();
-    long[] fsnStats = fsn.getStats(); 
-    long total = fsnStats[0];
-    long remaining = fsnStats[2];
-    long used = fsnStats[1];
-    long nonDFS = total - remaining - used;
-	nonDFS = nonDFS < 0 ? 0 : nonDFS; 
-    float percentUsed = total <= 0 
-        ? 0f : ((float)used * 100.0f)/(float)total;
-    float percentRemaining = total <= 0 
-        ? 100f : ((float)remaining * 100.0f)/(float)total;
-
-    out.print( "<div id=\"dfstable\"> <table>\n" +
-	       rowTxt() + colTxt() + "Configured Capacity" + colTxt() + ":" + colTxt() +
-	       StringUtils.byteDesc( total ) +
-	       rowTxt() + colTxt() + "DFS Used" + colTxt() + ":" + colTxt() +
-	       StringUtils.byteDesc( used ) +
-	       rowTxt() + colTxt() + "Non DFS Used" + colTxt() + ":" + colTxt() +
-	       StringUtils.byteDesc( nonDFS ) +
-	       rowTxt() + colTxt() + "DFS Remaining" + colTxt() + ":" + colTxt() +
-	       StringUtils.byteDesc( remaining ) +
-	       rowTxt() + colTxt() + "DFS Used%" + colTxt() + ":" + colTxt() +
-	       StringUtils.limitDecimalTo2(percentUsed) + " %" +
-	       rowTxt() + colTxt() + "DFS Remaining%" + colTxt() + ":" + colTxt() +
-	       StringUtils.limitDecimalTo2(percentRemaining) + " %" +
-	       rowTxt() + colTxt() +
-	       		"<a href=\"dfsnodelist.jsp?whatNodes=LIVE\">Live Nodes</a> " +
-	       		colTxt() + ":" + colTxt() + live.size() +
-	       rowTxt() + colTxt() +
-	       		"<a href=\"dfsnodelist.jsp?whatNodes=DEAD\">Dead Nodes</a> " +
-	       		colTxt() + ":" + colTxt() + dead.size() +
-               "</table></div><br>\n" );
-    
-    if (live.isEmpty() && dead.isEmpty()) {
-        out.print("There are no datanodes in the cluster");
-    }
-  }%>
-
+  //for java.io.Serializable
+  private static final long serialVersionUID = 1L;
+%>
 <%
+  final NamenodeJspHelper.HealthJsp healthjsp  = new NamenodeJspHelper.HealthJsp();
   NameNode nn = (NameNode)application.getAttribute("name.node");
   FSNamesystem fsn = nn.getNamesystem();
   String namenodeRole = nn.getRole().toString();
@@ -257,24 +40,20 @@
     
 <body>
 <h1><%=namenodeRole%> '<%=namenodeLabel%>'</h1>
-<%= JspHelper.getVersionTable(fsn) %>
+<%= NamenodeJspHelper.getVersionTable(fsn) %>
 <br />
 <b><a href="/nn_browsedfscontent.jsp">Browse the filesystem</a></b><br>
 <b><a href="/logs/"><%=namenodeRole%> Logs</a></b>
 
 <hr>
 <h3>Cluster Summary</h3>
-<b> <%= JspHelper.getSafeModeText(fsn)%> </b>
-<b> <%= JspHelper.getInodeLimitText(fsn)%> </b>
-<a class="warning"> <%= JspHelper.getWarningText(fsn)%></a>
+<b> <%= NamenodeJspHelper.getSafeModeText(fsn)%> </b>
+<b> <%= NamenodeJspHelper.getInodeLimitText(fsn)%> </b>
+<a class="warning"><%= NamenodeJspHelper.getWarningText(fsn)%></a>
 
-<%
-    generateDFSHealthReport(out, nn, request); 
-%>
+<% healthjsp.generateHealthReport(out, nn, request); %>
 <hr>
-<%
-	generateConfReport(out, nn, request);
-%>
+<% healthjsp.generateConfReport(out, nn, request); %>
 <%
 out.println(ServletUtil.htmlFooter());
 %>

+ 8 - 231
src/webapps/hdfs/dfsnodelist.jsp

@@ -18,236 +18,15 @@
  */
 %>
 <%@ page
-contentType="text/html; charset=UTF-8"
-	import="javax.servlet.*"
-	import="javax.servlet.http.*"
-	import="java.io.*"
-	import="java.util.*"
-	import="org.apache.hadoop.fs.*"
-	import="org.apache.hadoop.hdfs.*"
-	import="org.apache.hadoop.hdfs.server.common.*"
-	import="org.apache.hadoop.hdfs.server.namenode.*"
-	import="org.apache.hadoop.hdfs.server.datanode.*"
-	import="org.apache.hadoop.hdfs.protocol.*"
-	import="org.apache.hadoop.util.*"
-	import="java.text.DateFormat"
-	import="java.lang.Math"
-	import="java.net.URLEncoder"
+  contentType="text/html; charset=UTF-8"
+  import="org.apache.hadoop.util.ServletUtil"
 %>
 <%!
-	int rowNum = 0;
-	int colNum = 0;
-
-	String rowTxt() { colNum = 0;
-	return "<tr class=\"" + (((rowNum++)%2 == 0)? "rowNormal" : "rowAlt")
-	+ "\"> "; }
-	String colTxt() { return "<td id=\"col" + ++colNum + "\"> "; }
-	void counterReset () { colNum = 0; rowNum = 0 ; }
-
-	long diskBytes = 1024 * 1024 * 1024;
-	String diskByteStr = "GB";
-
-	String sorterField = null;
-	String sorterOrder = null;
-	String whatNodes = "LIVE";
-
-String NodeHeaderStr(String name) {
-	String ret = "class=header";
-	String order = "ASC";
-	if ( name.equals( sorterField ) ) {
-		ret += sorterOrder;
-		if ( sorterOrder.equals("ASC") )
-			order = "DSC";
-	}
-	ret += " onClick=\"window.document.location=" +
-	"'/dfsnodelist.jsp?whatNodes="+whatNodes+"&sorter/field=" + name + "&sorter/order=" +
-	order + "'\" title=\"sort on this column\"";
-
-	return ret;
-}
-
-public void generateNodeData( JspWriter out, DatanodeDescriptor d,
-		String suffix, boolean alive,
-		int nnHttpPort )
-throws IOException {
-
-	/* Say the datanode is dn1.hadoop.apache.org with ip 192.168.0.5
-we use:
-1) d.getHostName():d.getPort() to display.
-Domain and port are stripped if they are common across the nodes.
-i.e. "dn1"
-2) d.getHost():d.Port() for "title".
-i.e. "192.168.0.5:50010"
-3) d.getHostName():d.getInfoPort() for url.
-i.e. "http://dn1.hadoop.apache.org:50075/..."
-Note that "d.getHost():d.getPort()" is what DFS clients use
-to interact with datanodes.
-	 */
-	// from nn_browsedfscontent.jsp:
-	String url = "http://" + d.getHostName() + ":" + d.getInfoPort() +
-	"/browseDirectory.jsp?namenodeInfoPort=" +
-	nnHttpPort + "&dir=" +
-	URLEncoder.encode("/", "UTF-8");
-
-	String name = d.getHostName() + ":" + d.getPort();
-	if ( !name.matches( "\\d+\\.\\d+.\\d+\\.\\d+.*" ) ) 
-		name = name.replaceAll( "\\.[^.:]*", "" );    
-	int idx = (suffix != null && name.endsWith( suffix )) ?
-			name.indexOf( suffix ) : -1;
-
-			out.print( rowTxt() + "<td class=\"name\"><a title=\""
-					+ d.getHost() + ":" + d.getPort() +
-					"\" href=\"" + url + "\">" +
-					(( idx > 0 ) ? name.substring(0, idx) : name) + "</a>" +
-					(( alive ) ? "" : "\n") );
-			if ( !alive )
-				return;
-
-			long c = d.getCapacity();
-			long u = d.getDfsUsed();
-			long nu = d.getNonDfsUsed();
-			long r = d.getRemaining();
-			String percentUsed = StringUtils.limitDecimalTo2(d.getDfsUsedPercent());    
-			String percentRemaining = StringUtils.limitDecimalTo2(d.getRemainingPercent());    
-
-			String adminState = (d.isDecommissioned() ? "Decommissioned" :
-				(d.isDecommissionInProgress() ? "Decommission In Progress":
-				"In Service"));
-
-			long timestamp = d.getLastUpdate();
-			long currentTime = System.currentTimeMillis();
-			out.print("<td class=\"lastcontact\"> " +
-					((currentTime - timestamp)/1000) +
-					"<td class=\"adminstate\">" +
-					adminState +
-					"<td align=\"right\" class=\"capacity\">" +
-					StringUtils.limitDecimalTo2(c*1.0/diskBytes) +
-					"<td align=\"right\" class=\"used\">" +
-					StringUtils.limitDecimalTo2(u*1.0/diskBytes) +      
-					"<td align=\"right\" class=\"nondfsused\">" +
-					StringUtils.limitDecimalTo2(nu*1.0/diskBytes) +      
-					"<td align=\"right\" class=\"remaining\">" +
-					StringUtils.limitDecimalTo2(r*1.0/diskBytes) +      
-					"<td align=\"right\" class=\"pcused\">" + percentUsed +
-					"<td class=\"pcused\">" +
-					ServletUtil.percentageGraph( (int)Double.parseDouble(percentUsed) , 100) +
-					"<td align=\"right\" class=\"pcremaining`\">" + percentRemaining +
-					"<td title=" + "\"blocks scheduled : " + d.getBlocksScheduled() + 
-					"\" class=\"blocks\">" + d.numBlocks() + "\n");
-}
-
-
-
-public void generateDFSNodesList(JspWriter out, 
-		NameNode nn,
-		HttpServletRequest request)
-throws IOException {
-	ArrayList<DatanodeDescriptor> live = new ArrayList<DatanodeDescriptor>();    
-	ArrayList<DatanodeDescriptor> dead = new ArrayList<DatanodeDescriptor>();
-	nn.getNamesystem().DFSNodesStatus(live, dead);
-
-	whatNodes = request.getParameter("whatNodes"); // show only live or only dead nodes
-	sorterField = request.getParameter("sorter/field");
-	sorterOrder = request.getParameter("sorter/order");
-	if ( sorterField == null )
-		sorterField = "name";
-	if ( sorterOrder == null )
-		sorterOrder = "ASC";
-
-	JspHelper.sortNodeList(live, sorterField, sorterOrder);
-	JspHelper.sortNodeList(dead, "name", "ASC");
-
-	// Find out common suffix. Should this be before or after the sort?
-	String port_suffix = null;
-	if ( live.size() > 0 ) {
-		String name = live.get(0).getName();
-		int idx = name.indexOf(':');
-		if ( idx > 0 ) {
-			port_suffix = name.substring( idx );
-		}
-
-		for ( int i=1; port_suffix != null && i < live.size(); i++ ) {
-			if ( live.get(i).getName().endsWith( port_suffix ) == false ) {
-				port_suffix = null;
-				break;
-			}
-		}
-	}
-
-	counterReset();
-
-	try {
-		Thread.sleep(1000);
-	} catch (InterruptedException e) {}
-
-	if (live.isEmpty() && dead.isEmpty()) {
-		out.print("There are no datanodes in the cluster");
-	}
-	else {
-
-		int nnHttpPort = nn.getHttpAddress().getPort();
-		out.print( "<div id=\"dfsnodetable\"> ");
-		if(whatNodes.equals("LIVE")) {
-
-			out.print( 
-					"<a name=\"LiveNodes\" id=\"title\">" +
-					"Live Datanodes : " + live.size() + "</a>" +
-			"<br><br>\n<table border=1 cellspacing=0>\n" );
-
-			counterReset();
-
-			if ( live.size() > 0 ) {
-
-				if ( live.get(0).getCapacity() > 1024 * diskBytes ) {
-					diskBytes *= 1024;
-					diskByteStr = "TB";
-				}
-
-				out.print( "<tr class=\"headerRow\"> <th " +
-						NodeHeaderStr("name") + "> Node <th " +
-						NodeHeaderStr("lastcontact") + "> Last <br>Contact <th " +
-						NodeHeaderStr("adminstate") + "> Admin State <th " +
-						NodeHeaderStr("capacity") + "> Configured <br>Capacity (" + 
-						diskByteStr + ") <th " + 
-						NodeHeaderStr("used") + "> Used <br>(" + 
-						diskByteStr + ") <th " + 
-						NodeHeaderStr("nondfsused") + "> Non DFS <br>Used (" + 
-						diskByteStr + ") <th " + 
-						NodeHeaderStr("remaining") + "> Remaining <br>(" + 
-						diskByteStr + ") <th " + 
-						NodeHeaderStr("pcused") + "> Used <br>(%) <th " + 
-						NodeHeaderStr("pcused") + "> Used <br>(%) <th " +
-						NodeHeaderStr("pcremaining") + "> Remaining <br>(%) <th " +
-						NodeHeaderStr("blocks") + "> Blocks\n" );
-
-				JspHelper.sortNodeList(live, sorterField, sorterOrder);
-				for ( int i=0; i < live.size(); i++ ) {
-					generateNodeData(out, live.get(i), port_suffix, true, nnHttpPort);
-				}
-			}
-			out.print("</table>\n");
-		} else {
-
-			out.print("<br> <a name=\"DeadNodes\" id=\"title\"> " +
-					" Dead Datanodes : " +dead.size() + "</a><br><br>\n");
-
-			if ( dead.size() > 0 ) {
-				out.print( "<table border=1 cellspacing=0> <tr id=\"row1\"> " +
-				"<td> Node \n" );
-
-				JspHelper.sortNodeList(dead, "name", "ASC");
-				for ( int i=0; i < dead.size() ; i++ ) {
-					generateNodeData(out, dead.get(i), port_suffix, false, nnHttpPort);
-				}
-
-				out.print("</table>\n");
-			}
-		}
-		out.print("</div>");
-	}
-}%>
-
+  //for java.io.Serializable
+  private static final long serialVersionUID = 1L;
+%>
 <%
+final NamenodeJspHelper.NodeListJsp nodelistjsp = new NamenodeJspHelper.NodeListJsp();
 NameNode nn = (NameNode)application.getAttribute("name.node");
 String namenodeRole = nn.getRole().toString();
 FSNamesystem fsn = nn.getNamesystem();
@@ -261,15 +40,13 @@ String namenodeLabel = nn.getNameNodeAddress().getHostName() + ":" + nn.getNameN
   
 <body>
 <h1><%=namenodeRole%> '<%=namenodeLabel%>'</h1>
-<%= JspHelper.getVersionTable(fsn) %>
+<%= NamenodeJspHelper.getVersionTable(fsn) %>
 <br />
 <b><a href="/nn_browsedfscontent.jsp">Browse the filesystem</a></b><br>
 <b><a href="/logs/"><%=namenodeRole%> Logs</a></b><br>
 <b><a href=/dfshealth.jsp> Go back to DFS home</a></b>
 <hr>
-<%
-	generateDFSNodesList(out, nn, request); 
-%>
+<% nodelistjsp.generateNodesList(out, nn, request); %>
 
 <%
 out.println(ServletUtil.htmlFooter());

+ 4 - 37
src/webapps/hdfs/nn_browsedfscontent.jsp

@@ -19,45 +19,12 @@
 %>
 <%@ page
   contentType="text/html; charset=UTF-8"
-  import="javax.servlet.*"
-  import="javax.servlet.http.*"
-  import="java.io.*"
-  import="java.util.*"
-  import="org.apache.hadoop.hdfs.*"
-  import="org.apache.hadoop.hdfs.server.namenode.*"
-  import="org.apache.hadoop.hdfs.server.datanode.*"
-  import="org.apache.hadoop.hdfs.protocol.*"
-  import="org.apache.hadoop.util.*"
-  import="java.text.DateFormat"
-  import="java.net.InetAddress"
-  import="java.net.URLEncoder"
+  import="org.apache.hadoop.util.ServletUtil"
 %>
 <%!
-  public void redirectToRandomDataNode(
-                            NameNode nn, 
-                            HttpServletResponse resp) throws IOException {
-    FSNamesystem fsn = nn.getNamesystem();
-    String datanode = fsn.randomDataNode();
-    String redirectLocation;
-    String nodeToRedirect;
-    int redirectPort;
-    if (datanode != null) {
-      redirectPort = Integer.parseInt(datanode.substring(datanode.indexOf(':') + 1));
-      nodeToRedirect = datanode.substring(0, datanode.indexOf(':'));
-    }
-    else {
-      nodeToRedirect = nn.getHttpAddress().getHostName();
-      redirectPort = nn.getHttpAddress().getPort();
-    }
-    String fqdn = InetAddress.getByName(nodeToRedirect).getCanonicalHostName();
-    redirectLocation = "http://" + fqdn + ":" + redirectPort + 
-                       "/browseDirectory.jsp?namenodeInfoPort=" + 
-                       nn.getHttpAddress().getPort() +
-                       "&dir=" + URLEncoder.encode("/", "UTF-8");
-    resp.sendRedirect(redirectLocation);
-  }
+  //for java.io.Serializable
+  private static final long serialVersionUID = 1L;
 %>
-
 <html>
 
 <title></title>
@@ -65,7 +32,7 @@
 <body>
 <% 
   NameNode nn = (NameNode)application.getAttribute("name.node");
-  redirectToRandomDataNode(nn, response); 
+  NamenodeJspHelper.redirectToRandomDataNode(nn, response); 
 %>
 <hr>
 

+ 6 - 1
src/webapps/secondary/status.jsp

@@ -19,7 +19,12 @@
 %>
 <%@ page
   contentType="text/html; charset=UTF-8"
-  import="org.apache.hadoop.util.*"
+  import="org.apache.hadoop.hdfs.server.common.JspHelper"
+  import="org.apache.hadoop.util.ServletUtil"
+%>
+<%!
+  //for java.io.Serializable
+  private static final long serialVersionUID = 1L;
 %>
 
 <html>

برخی فایل ها در این مقایسه diff نمایش داده نمی شوند زیرا تعداد فایل ها بسیار زیاد است