You are viewing a plain text version of this content. The canonical link for it is here.
Posted to hdfs-commits@hadoop.apache.org by sh...@apache.org on 2009/12/11 04:32:24 UTC

svn commit: r889494 - in /hadoop/hdfs/trunk: ./ src/java/ src/java/org/apache/hadoop/hdfs/server/common/ src/java/org/apache/hadoop/hdfs/server/namenode/ src/test/hdfs/org/apache/hadoop/hdfs/ src/test/hdfs/org/apache/hadoop/hdfs/server/common/ src/test...

Author: shv
Date: Fri Dec 11 03:32:23 2009
New Revision: 889494

URL: http://svn.apache.org/viewvc?rev=889494&view=rev
Log:
HDFS-456. Fix URI generation for windows file paths. Contributed by Konstantin Shvachko.

Added:
    hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/server/common/TestGetUriFromString.java   (with props)
Modified:
    hadoop/hdfs/trunk/CHANGES.txt
    hadoop/hdfs/trunk/src/java/hdfs-default.xml
    hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/common/Util.java
    hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java
    hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
    hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/MiniDFSCluster.java
    hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/TestHDFSServerPorts.java
    hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestStartup.java
    hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestStorageRestore.java

Modified: hadoop/hdfs/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/hdfs/trunk/CHANGES.txt?rev=889494&r1=889493&r2=889494&view=diff
==============================================================================
--- hadoop/hdfs/trunk/CHANGES.txt (original)
+++ hadoop/hdfs/trunk/CHANGES.txt Fri Dec 11 03:32:23 2009
@@ -563,6 +563,8 @@
     HDFS-823. CheckPointer should use addInternalServlet for image-fetching
     servlet (jghoman)
 
+    HDFS-456. Fix URI generation for windows file paths. (shv)
+
 Release 0.20.2 - Unreleased
 
   IMPROVEMENTS

Modified: hadoop/hdfs/trunk/src/java/hdfs-default.xml
URL: http://svn.apache.org/viewvc/hadoop/hdfs/trunk/src/java/hdfs-default.xml?rev=889494&r1=889493&r2=889494&view=diff
==============================================================================
--- hadoop/hdfs/trunk/src/java/hdfs-default.xml (original)
+++ hadoop/hdfs/trunk/src/java/hdfs-default.xml Fri Dec 11 03:32:23 2009
@@ -169,7 +169,7 @@
 
 <property>
   <name>dfs.namenode.name.dir</name>
-  <value>${hadoop.tmp.dir}/dfs/name</value>
+  <value>file://${hadoop.tmp.dir}/dfs/name</value>
   <description>Determines where on the local filesystem the DFS name node
       should store the name table(fsimage).  If this is a comma-delimited list
       of directories then the name table is replicated in all of the
@@ -447,7 +447,7 @@
 
 <property>
   <name>dfs.namenode.checkpoint.dir</name>
-  <value>${hadoop.tmp.dir}/dfs/namesecondary</value>
+  <value>file://${hadoop.tmp.dir}/dfs/namesecondary</value>
   <description>Determines where on the local filesystem the DFS secondary
       name node should store the temporary images to merge.
       If this is a comma-delimited list of directories then the image is

Modified: hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/common/Util.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/common/Util.java?rev=889494&r1=889493&r2=889494&view=diff
==============================================================================
--- hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/common/Util.java (original)
+++ hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/common/Util.java Fri Dec 11 03:32:23 2009
@@ -17,7 +17,19 @@
  */
 package org.apache.hadoop.hdfs.server.common;
 
+import java.io.File;
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.ArrayList;
+import java.util.Collection;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
 public final class Util {
+  private final static Log LOG = LogFactory.getLog(Util.class.getName());
+
   /**
    * Current system time.
    * @return current time in msec.
@@ -25,4 +37,58 @@
   public static long now() {
     return System.currentTimeMillis();
   }
-}
\ No newline at end of file
+  
+  /**
+   * Interprets the passed string as a URI. In case of error it 
+   * assumes the specified string is a file.
+   *
+   * @param s the string to interpret
+   * @return the resulting URI 
+   * @throws IOException 
+   */
+  public static URI stringAsURI(String s) throws IOException {
+    URI u = null;
+    // try to make a URI
+    try {
+      u = new URI(s);
+    } catch (URISyntaxException e){
+      LOG.warn("Path " + s + " should be specified as a URI " +
+      "in configuration files. Please update hdfs configuration.", e);
+    }
+
+    // if URI is null or scheme is undefined, then assume it's file://
+    if(u == null || u.getScheme() == null){
+      u = fileAsURI(new File(s));
+    }
+    return u;
+  }
+
+  /**
+   * Converts the passed File to a URI.
+   *
+   * @param f the file to convert
+   * @return the resulting URI 
+   * @throws IOException 
+   */
+  public static URI fileAsURI(File f) throws IOException {
+    return f.getCanonicalFile().toURI();
+  }
+
+  /**
+   * Converts a collection of strings into a collection of URIs.
+   * @param names collection of strings to convert to URIs
+   * @return collection of URIs
+   */
+  public static Collection<URI> stringCollectionAsURIs(
+                                  Collection<String> names) {
+    Collection<URI> uris = new ArrayList<URI>(names.size());
+    for(String name : names) {
+      try {
+        uris.add(stringAsURI(name));
+      } catch (IOException e) {
+        LOG.error("Error while processing URI: " + name, e);
+      }
+    }
+    return uris;
+  }
+}

Modified: hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java?rev=889494&r1=889493&r2=889494&view=diff
==============================================================================
--- hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java (original)
+++ hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java Fri Dec 11 03:32:23 2009
@@ -56,6 +56,7 @@
 import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.common.StorageInfo;
 import org.apache.hadoop.hdfs.server.common.UpgradeManager;
+import org.apache.hadoop.hdfs.server.common.Util;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants.NamenodeRole;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants.NodeType;
@@ -305,11 +306,10 @@
     for ( ;it.hasNext(); ) {
       StorageDirectory sd = it.next();
       try {
-        list.add(new URI("file://" + sd.getRoot().getAbsolutePath()));
-      } catch (Exception e) {
+        list.add(Util.fileAsURI(sd.getRoot()));
+      } catch (IOException e) {
         throw new IOException("Exception while processing " +
-            "StorageDirectory " + sd.getRoot().getAbsolutePath() + ". The"
-            + " full error message is " + e.getMessage());
+            "StorageDirectory " + sd.getRoot(), e);
       }
     }
     return list;
@@ -1907,24 +1907,9 @@
     if (dirNames.size() == 0 && defaultValue != null) {
       dirNames.add(defaultValue);
     }
-    Collection<URI> dirs = new ArrayList<URI>(dirNames.size());
-    for(String name : dirNames) {
-      try {
-        // process value as URI 
-        URI u = new URI(name);
-        // if scheme is undefined, then assume it's file://
-        if(u.getScheme() == null)
-          u = new URI("file://" + new File(name).getAbsolutePath());
-        // check that scheme is not null (trivial) and supported
-        checkSchemeConsistency(u);
-        dirs.add(u);
-      } catch (Exception e) {
-        LOG.error("Error while processing URI: " + name, e);
-      }
-    }
-    return dirs;
+    return Util.stringCollectionAsURIs(dirNames);
   }
-  
+
   static Collection<URI> getCheckpointEditsDirs(Configuration conf,
       String defaultName) {
     Collection<String> dirNames = 
@@ -1932,23 +1917,7 @@
     if (dirNames.size() == 0 && defaultName != null) {
       dirNames.add(defaultName);
     }
-    Collection<URI> dirs = new ArrayList<URI>(dirNames.size());
-    for(String name : dirNames) {
-      try {
-        // process value as URI 
-        URI u = new URI(name);
-        // if scheme is undefined, then assume it's file://
-        if(u.getScheme() == null)
-          u = new URI("file://" + new File(name).getAbsolutePath());
-        // check that scheme is not null (trivial) and supported
-        checkSchemeConsistency(u);
-        dirs.add(u);
-      } catch (Exception e) {
-        LOG.error("Error while processing URI: " + name + 
-            ". The error message was: " + e.getMessage());
-      }
-    }
-    return dirs;    
+    return Util.stringCollectionAsURIs(dirNames);
   }
 
   static private final DeprecatedUTF8 U_STR = new DeprecatedUTF8();

Modified: hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java?rev=889494&r1=889493&r2=889494&view=diff
==============================================================================
--- hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java (original)
+++ hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java Fri Dec 11 03:32:23 2009
@@ -29,6 +29,7 @@
 import org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.common.UpgradeStatusReport;
+import org.apache.hadoop.hdfs.server.common.Util;
 import org.apache.hadoop.hdfs.server.namenode.metrics.FSNamesystemMBean;
 import org.apache.hadoop.hdfs.server.namenode.metrics.FSNamesystemMetrics;
 import org.apache.hadoop.security.AccessControlException;
@@ -339,26 +340,8 @@
           "\n\t\t- use Backup Node as a persistent and up-to-date storage " +
           "of the file system meta-data.");
     } else if (dirNames.isEmpty())
-      dirNames.add("/tmp/hadoop/dfs/name");
-    Collection<URI> dirs = new ArrayList<URI>(dirNames.size());
-    for(String name : dirNames) {
-      try {
-        URI u = new URI(name);
-        // If the scheme was not declared, default to file://
-        // and use the absolute path of the file, then warn the user 
-        if(u.getScheme() == null) {
-          u = new URI("file://" + new File(name).getAbsolutePath());
-          LOG.warn("Scheme is undefined for " + name);
-          LOG.warn("Please check your file system configuration in " +
-          		"hdfs-site.xml");
-        }
-        dirs.add(u);
-      } catch (Exception e) {
-        LOG.error("Error while processing URI: " + name + 
-            ". The error message was: " + e.getMessage());
-      }
-    }
-    return dirs;
+      dirNames.add("file:///tmp/hadoop/dfs/name");
+    return Util.stringCollectionAsURIs(dirNames);
   }
 
   public static Collection<URI> getNamespaceEditsDirs(Configuration conf) {

Modified: hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/MiniDFSCluster.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/MiniDFSCluster.java?rev=889494&r1=889493&r2=889494&view=diff
==============================================================================
--- hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/MiniDFSCluster.java (original)
+++ hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/MiniDFSCluster.java Fri Dec 11 03:32:23 2009
@@ -37,6 +37,7 @@
 import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.FSConstants.DatanodeReportType;
+import static org.apache.hadoop.hdfs.server.common.Util.fileAsURI;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.FSDatasetInterface;
@@ -258,10 +259,12 @@
     FileSystem.setDefaultUri(conf, "hdfs://localhost:"+ Integer.toString(nameNodePort));
     conf.set(DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY, "127.0.0.1:0");  
     if (manageNameDfsDirs) {
-      conf.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY, new File(base_dir, "name1").getPath()+","+
-               new File(base_dir, "name2").getPath());
-      conf.set(DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_DIR_KEY, new File(base_dir, "namesecondary1").
-                getPath()+"," + new File(base_dir, "namesecondary2").getPath());
+      conf.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY,
+          fileAsURI(new File(base_dir, "name1"))+","+
+          fileAsURI(new File(base_dir, "name2")));
+      conf.set(DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_DIR_KEY,
+          fileAsURI(new File(base_dir, "namesecondary1"))+","+
+          fileAsURI(new File(base_dir, "namesecondary2")));
     }
     
     int replication = conf.getInt("dfs.replication", 3);
@@ -716,8 +719,8 @@
 
   /**
    * Restart a datanode, on the same port if requested
-   * @param dnprop, the datanode to restart
-   * @param keepPort, whether to use the same port 
+   * @param dnprop the datanode to restart
+   * @param keepPort whether to use the same port 
    * @return true if restarting is successful
    * @throws IOException
    */
@@ -957,7 +960,6 @@
 
   /**
    * Access to the data directory used for Datanodes
-   * @throws IOException 
    */
   public String getDataDirectory() {
     return data_dir.getAbsolutePath();

Modified: hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/TestHDFSServerPorts.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/TestHDFSServerPorts.java?rev=889494&r1=889493&r2=889494&view=diff
==============================================================================
--- hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/TestHDFSServerPorts.java (original)
+++ hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/TestHDFSServerPorts.java Fri Dec 11 03:32:23 2009
@@ -28,6 +28,7 @@
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
+import static org.apache.hadoop.hdfs.server.common.Util.fileAsURI;
 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;
@@ -92,7 +93,8 @@
       throw new IOException("Could not delete hdfs directory '" + hdfsDir + "'");
     }
     config = new HdfsConfiguration();
-    config.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY, new File(hdfsDir, "name1").getPath());
+    config.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY,
+        fileAsURI(new File(hdfsDir, "name1")).toString());
     FileSystem.setDefaultUri(config, "hdfs://"+NAME_NODE_HOST + "0");
     config.set(DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY, NAME_NODE_HTTP_HOST + "0");
     NameNode.format(config);
@@ -120,7 +122,8 @@
     assertTrue(currDir2.mkdirs());
     assertTrue(currDir3.mkdirs());
     
-    conf.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY, new File(hdfsDir, "name2").getPath());
+    conf.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY,
+        fileAsURI(new File(hdfsDir, "name2")).toString());
     conf.set(DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_KEY, "${dfs.name.dir}");
     
     // Start BackupNode
@@ -246,7 +249,8 @@
 
       // start another namenode on the same port
       Configuration conf2 = new HdfsConfiguration(config);
-      conf2.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY, new File(hdfsDir, "name2").getPath());
+      conf2.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY,
+          fileAsURI(new File(hdfsDir, "name2")).toString());
       NameNode.format(conf2);
       boolean started = canStartNameNode(conf2);
       assertFalse(started); // should fail

Added: hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/server/common/TestGetUriFromString.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/server/common/TestGetUriFromString.java?rev=889494&view=auto
==============================================================================
--- hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/server/common/TestGetUriFromString.java (added)
+++ hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/server/common/TestGetUriFromString.java Fri Dec 11 03:32:23 2009
@@ -0,0 +1,92 @@
+/**
+ * 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.IOException;
+import java.net.URI;
+
+import junit.framework.TestCase;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+/**
+ * This is a unit test, which tests {@link Util#stringAsURI(String)}
+ * for Windows and Unix style file paths.
+ */
+public class TestGetUriFromString extends TestCase {
+  private static final Log LOG = LogFactory.getLog(TestGetUriFromString.class);
+
+  private static final String RELATIVE_FILE_PATH = "relativeFilePath";
+  private static final String ABSOLUTE_PATH_UNIX = "/tmp/file1";
+  private static final String ABSOLUTE_PATH_WINDOWS =
+    "C:\\Documents and Settings\\All Users";
+  private static final String URI_FILE_SCHEMA = "file";
+  private static final String URI_PATH_UNIX = "/var/www";
+  private static final String URI_PATH_WINDOWS =
+    "/C:/Documents%20and%20Settings/All%20Users";
+  private static final String URI_UNIX = URI_FILE_SCHEMA + "://"
+      + URI_PATH_UNIX;
+  private static final String URI_WINDOWS = URI_FILE_SCHEMA + "://"
+      + URI_PATH_WINDOWS;
+
+  /**
+   * Test for a relative path, os independent
+   * @throws IOException 
+   */
+  public void testRelativePathAsURI() throws IOException {
+    URI u = Util.stringAsURI(RELATIVE_FILE_PATH);
+    LOG.info("Uri: " + u);
+    assertNotNull(u);
+  }
+
+  /**
+   * Test for an OS dependent absolute paths.
+   * @throws IOException 
+   */
+  public void testAbsolutePathAsURI() throws IOException {
+    URI u = null;
+    u = Util.stringAsURI(ABSOLUTE_PATH_WINDOWS);
+    assertNotNull(
+        "Uri should not be null for Windows path" + ABSOLUTE_PATH_WINDOWS, u);
+    assertEquals(URI_FILE_SCHEMA, u.getScheme());
+    u = Util.stringAsURI(ABSOLUTE_PATH_UNIX);
+    assertNotNull("Uri should not be null for Unix path" + ABSOLUTE_PATH_UNIX, u);
+    assertEquals(URI_FILE_SCHEMA, u.getScheme());
+  }
+
+  /**
+   * Test for a URI
+   * @throws IOException 
+   */
+  public void testURI() throws IOException {
+    LOG.info("Testing correct Unix URI: " + URI_UNIX);
+    URI u = Util.stringAsURI(URI_UNIX);
+    LOG.info("Uri: " + u);    
+    assertNotNull("Uri should not be null at this point", u);
+    assertEquals(URI_FILE_SCHEMA, u.getScheme());
+    assertEquals(URI_PATH_UNIX, u.getPath());
+
+    LOG.info("Testing correct windows URI: " + URI_WINDOWS);
+    u = Util.stringAsURI(URI_WINDOWS);
+    LOG.info("Uri: " + u);
+    assertNotNull("Uri should not be null at this point", u);
+    assertEquals(URI_FILE_SCHEMA, u.getScheme());
+    assertEquals(URI_PATH_WINDOWS.replace("%20", " "), u.getPath());
+  }
+}

Propchange: hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/server/common/TestGetUriFromString.java
------------------------------------------------------------------------------
    svn:mime-type = text/plain

Modified: hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestStartup.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestStartup.java?rev=889494&r1=889493&r2=889494&view=diff
==============================================================================
--- hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestStartup.java (original)
+++ hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestStartup.java Fri Dec 11 03:32:23 2009
@@ -36,6 +36,7 @@
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
+import static org.apache.hadoop.hdfs.server.common.Util.fileAsURI;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
 import org.apache.hadoop.hdfs.server.namenode.FSImage.NameNodeDirType;
@@ -74,18 +75,18 @@
 
   protected void setUp() throws Exception {
     config = new HdfsConfiguration();
-    String baseDir = System.getProperty("test.build.data", "/tmp");
+    hdfsDir = new File(MiniDFSCluster.getBaseDirectory());
 
-    hdfsDir = new File(baseDir, "dfs");
     if ( hdfsDir.exists() && !FileUtil.fullyDelete(hdfsDir) ) {
       throw new IOException("Could not delete hdfs directory '" + hdfsDir + "'");
     }
     LOG.info("--hdfsdir is " + hdfsDir.getAbsolutePath());
-    config.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY, new File(hdfsDir, "name").getPath());
-    config.set(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY, new File(hdfsDir, "data").getPath());
-
-    config.set(DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_DIR_KEY,new File(hdfsDir, "secondary").getPath());
-    //config.set("fs.default.name", "hdfs://"+ NAME_NODE_HOST + "0");
+    config.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY,
+        fileAsURI(new File(hdfsDir, "name")).toString());
+    config.set(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY,
+        new File(hdfsDir, "data").getPath());
+    config.set(DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_DIR_KEY,
+        fileAsURI(new File(hdfsDir, "secondary")).toString());
     
     FileSystem.setDefaultUri(config, "hdfs://"+NAME_NODE_HOST + "0");
   }
@@ -231,11 +232,15 @@
   public void testChkpointStartup2() throws IOException{
     LOG.info("--starting checkpointStartup2 - same directory for checkpoint");
     // different name dirs
-    config.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY, new File(hdfsDir, "name").getPath());
-    config.set(DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_KEY, new File(hdfsDir, "edits").getPath());
+    config.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY,
+        fileAsURI(new File(hdfsDir, "name")).toString());
+    config.set(DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_KEY,
+        fileAsURI(new File(hdfsDir, "edits")).toString());
     // same checkpoint dirs
-    config.set(DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_EDITS_DIR_KEY, new File(hdfsDir, "chkpt").getPath());
-    config.set(DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_DIR_KEY, new File(hdfsDir, "chkpt").getPath());
+    config.set(DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_EDITS_DIR_KEY,
+        fileAsURI(new File(hdfsDir, "chkpt")).toString());
+    config.set(DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_DIR_KEY,
+        fileAsURI(new File(hdfsDir, "chkpt")).toString());
 
     createCheckPoint();
 
@@ -253,11 +258,15 @@
     //setUpConfig();
     LOG.info("--starting testStartup Recovery");
     // different name dirs
-    config.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY, new File(hdfsDir, "name").getPath());
-    config.set(DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_KEY, new File(hdfsDir, "edits").getPath());
+    config.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY,
+        fileAsURI(new File(hdfsDir, "name")).toString());
+    config.set(DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_KEY,
+        fileAsURI(new File(hdfsDir, "edits")).toString());
     // same checkpoint dirs
-    config.set(DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_EDITS_DIR_KEY, new File(hdfsDir, "chkpt_edits").getPath());
-    config.set(DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_DIR_KEY, new File(hdfsDir, "chkpt").getPath());
+    config.set(DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_EDITS_DIR_KEY,
+        fileAsURI(new File(hdfsDir, "chkpt_edits")).toString());
+    config.set(DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_DIR_KEY,
+        fileAsURI(new File(hdfsDir, "chkpt")).toString());
 
     createCheckPoint();
     corruptNameNodeFiles();
@@ -274,11 +283,15 @@
     //setUpConfig();
     LOG.info("--starting SecondNN startup test");
     // different name dirs
-    config.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY, new File(hdfsDir, "name").getPath());
-    config.set(DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_KEY, new File(hdfsDir, "name").getPath());
+    config.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY,
+        fileAsURI(new File(hdfsDir, "name")).toString());
+    config.set(DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_KEY,
+        fileAsURI(new File(hdfsDir, "name")).toString());
     // same checkpoint dirs
-    config.set(DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_EDITS_DIR_KEY, new File(hdfsDir, "chkpt_edits").getPath());
-    config.set(DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_DIR_KEY, new File(hdfsDir, "chkpt").getPath());
+    config.set(DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_EDITS_DIR_KEY,
+        fileAsURI(new File(hdfsDir, "chkpt_edits")).toString());
+    config.set(DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_DIR_KEY,
+        fileAsURI(new File(hdfsDir, "chkpt")).toString());
 
     LOG.info("--starting NN ");
     MiniDFSCluster cluster = null;

Modified: hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestStorageRestore.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestStorageRestore.java?rev=889494&r1=889493&r2=889494&view=diff
==============================================================================
--- hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestStorageRestore.java (original)
+++ hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestStorageRestore.java Fri Dec 11 03:32:23 2009
@@ -132,8 +132,8 @@
     Iterator<StorageDirectory> it = fi.dirIterator();
     while(it.hasNext()) {
       StorageDirectory sd = it.next();
-      if(sd.getRoot().getAbsolutePath().equals(path2.getAbsolutePath()) ||
-          sd.getRoot().getAbsolutePath().equals(path3.getAbsolutePath())) {
+      if(sd.getRoot().getCanonicalPath().equals(path2.getCanonicalPath()) ||
+          sd.getRoot().getCanonicalPath().equals(path3.getCanonicalPath())) {
         al.add(sd);
       }
     }
@@ -164,7 +164,7 @@
   /**
    * This function returns a md5 hash of a file.
    * 
-   * @param FileToMd5
+   * @param file input file
    * @return The md5 string
    */
   public String getFileMD5(File file) throws Exception {
@@ -189,7 +189,7 @@
   /**
    * read currentCheckpointTime directly from the file
    * @param currDir
-   * @return
+   * @return the checkpoint time
    * @throws IOException
    */
   long readCheckpointTime(File currDir) throws IOException {