Browse Source

HDFS-396. NameNode image and edits directories are specified as URIs.
(Luca Telloli via rangadi)

git-svn-id: https://svn.apache.org/repos/asf/hadoop/hdfs/trunk@787781 13f79535-47bb-0310-9956-ffa450edef68

Raghu Angadi 16 years ago
parent
commit
f652fefcf7

+ 3 - 0
CHANGES.txt

@@ -12,6 +12,9 @@ Trunk (unreleased changes)
     HDFS-381. Remove blocks from DataNode maps when corresponding file
     is deleted. (Suresh Srinivas via rangadi)
 
+    HDFS-396. NameNode image and edits directories are specified as URIs.
+    (Luca Telloli via rangadi)
+
   BUG FIXES
     HDFS-76. Better error message to users when commands fail because of 
     lack of quota. Allow quota to be set even if the limit is lower than

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

@@ -20,6 +20,7 @@ package org.apache.hadoop.hdfs.server.namenode;
 import java.io.DataInputStream;
 import java.io.File;
 import java.io.IOException;
+import java.net.URI;
 import java.util.Collection;
 import java.util.Iterator;
 
@@ -66,12 +67,12 @@ public class BackupStorage extends FSImage {
    * Read VERSION and fstime files if exist.<br>
    * Do not load image or edits.
    * 
-   * @param imageDirs list of image directories.
-   * @param editsDirs list of edits directories.
+   * @param imageDirs list of image directories as URI.
+   * @param editsDirs list of edits directories URI.
    * @throws IOException if the node should shutdown.
    */
-  void recoverCreateRead(Collection<File> imageDirs,
-                         Collection<File> editsDirs) throws IOException {
+  void recoverCreateRead(Collection<URI> imageDirs,
+                         Collection<URI> editsDirs) throws IOException {
     setStorageDirectories(imageDirs, editsDirs);
     this.checkpointTime = 0L;
     for(Iterator<StorageDirectory> it = dirIterator(); it.hasNext();) {

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

@@ -18,6 +18,7 @@
 package org.apache.hadoop.hdfs.server.namenode;
 
 import java.io.*;
+import java.net.URI;
 import java.util.*;
 
 import org.apache.hadoop.conf.Configuration;
@@ -85,8 +86,8 @@ class FSDirectory implements Closeable {
     directoryMetrics.setTag("sessionId", conf.get("session.id"));
   }
 
-  void loadFSImage(Collection<File> dataDirs,
-                   Collection<File> editsDirs,
+  void loadFSImage(Collection<URI> dataDirs,
+                   Collection<URI> editsDirs,
                    StartupOption startOpt) throws IOException {
     // format before starting up if requested
     if (startOpt == StartupOption.FORMAT) {

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

@@ -28,6 +28,7 @@ import java.io.FileInputStream;
 import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.RandomAccessFile;
+import java.net.URI;
 import java.nio.ByteBuffer;
 import java.text.SimpleDateFormat;
 import java.util.ArrayList;
@@ -57,6 +58,7 @@ import org.apache.hadoop.hdfs.server.common.HdfsConstants.NamenodeRole;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants.NodeType;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.namenode.BlocksMap.BlockInfo;
+import org.apache.hadoop.hdfs.server.namenode.JournalStream.JournalType;
 import org.apache.hadoop.hdfs.server.protocol.CheckpointCommand;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeCommand;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
@@ -127,12 +129,13 @@ public class FSImage extends Storage {
    * list of failed (and thus removed) storages
    */
   protected List<StorageDirectory> removedStorageDirs = new ArrayList<StorageDirectory>();
-  
+    
   /**
-   * Directories for importing an image from a checkpoint.
+   * URIs for importing an image from a checkpoint. In the default case,
+   * URIs will represent directories. 
    */
-  private Collection<File> checkpointDirs;
-  private Collection<File> checkpointEditsDirs;
+  private Collection<URI> checkpointDirs;
+  private Collection<URI> checkpointEditsDirs;
 
   /**
    * Can fs-image be rolled?
@@ -158,8 +161,10 @@ public class FSImage extends Storage {
   }
 
   /**
+   * @throws IOException 
    */
-  FSImage(Collection<File> fsDirs, Collection<File> fsEditsDirs) {
+  FSImage(Collection<URI> fsDirs, Collection<URI> fsEditsDirs) 
+      throws IOException {
     this();
     setStorageDirectories(fsDirs, fsEditsDirs);
   }
@@ -170,11 +175,12 @@ public class FSImage extends Storage {
 
   /**
    * Represents an Image (image and edit file).
+   * @throws IOException 
    */
-  FSImage(File imageDir) {
+  FSImage(URI imageDir) throws IOException {
     this();
-    ArrayList<File> dirs = new ArrayList<File>(1);
-    ArrayList<File> editsDirs = new ArrayList<File>(1);
+    ArrayList<URI> dirs = new ArrayList<URI>(1);
+    ArrayList<URI> editsDirs = new ArrayList<URI>(1);
     dirs.add(imageDir);
     editsDirs.add(imageDir);
     setStorageDirectories(dirs, editsDirs);
@@ -197,14 +203,16 @@ public class FSImage extends Storage {
     return restoreFailedStorage;
   }
   
-  void setStorageDirectories(Collection<File> fsNameDirs,
-                             Collection<File> fsEditsDirs) {
+  void setStorageDirectories(Collection<URI> fsNameDirs,
+                             Collection<URI> fsEditsDirs) throws IOException {
     this.storageDirs = new ArrayList<StorageDirectory>();
     this.removedStorageDirs = new ArrayList<StorageDirectory>();
+    
    // Add all name dirs with appropriate NameNodeDirType 
-    for (File dirName : fsNameDirs) {
+    for (URI dirName : fsNameDirs) {
+      checkSchemeConsistency(dirName);
       boolean isAlsoEdits = false;
-      for (File editsDirName : fsEditsDirs) {
+      for (URI editsDirName : fsEditsDirs) {
         if (editsDirName.compareTo(dirName) == 0) {
           isAlsoEdits = true;
           fsEditsDirs.remove(editsDirName);
@@ -214,18 +222,49 @@ public class FSImage extends Storage {
       NameNodeDirType dirType = (isAlsoEdits) ?
                           NameNodeDirType.IMAGE_AND_EDITS :
                           NameNodeDirType.IMAGE;
-      this.addStorageDir(new StorageDirectory(dirName, dirType));
+      // Add to the list of storage directories, only if the 
+      // URI is of type file://
+      if(dirName.getScheme().compareTo(JournalType.FILE.name().toLowerCase()) 
+          == 0){
+        this.addStorageDir(new StorageDirectory(new File(dirName.getPath()), 
+            dirType));
+      }
     }
     
     // Add edits dirs if they are different from name dirs
-    for (File dirName : fsEditsDirs) {
-      this.addStorageDir(new StorageDirectory(dirName, 
+    for (URI dirName : fsEditsDirs) {
+      checkSchemeConsistency(dirName);
+      // Add to the list of storage directories, only if the 
+      // URI is of type file://
+      if(dirName.getScheme().compareTo(JournalType.FILE.name().toLowerCase())
+          == 0)
+        this.addStorageDir(new StorageDirectory(new File(dirName.getPath()), 
                     NameNodeDirType.EDITS));
     }
   }
 
-  void setCheckpointDirectories(Collection<File> dirs,
-                                Collection<File> editsDirs) {
+  /* 
+   * Checks the consistency of a URI, in particular if the scheme 
+   * is specified and is supported by a concrete implementation 
+   */
+  static void checkSchemeConsistency(URI u) throws IOException {
+    String scheme = u.getScheme();
+    // the URI should have a proper scheme
+    if(scheme == null)
+      throw new IOException("Undefined scheme for " + u);
+    else {
+      try {
+        // the scheme should be enumerated as JournalType
+        JournalType.valueOf(scheme.toUpperCase());
+      } catch (IllegalArgumentException iae){
+        throw new IOException("Unknown scheme " + scheme + 
+            ". It should correspond to a JournalType enumeration value");
+      }
+    }
+  };
+  
+  void setCheckpointDirectories(Collection<URI> dirs,
+                                Collection<URI> editsDirs) {
     checkpointDirs = dirs;
     checkpointEditsDirs = editsDirs;
   }
@@ -235,7 +274,7 @@ public class FSImage extends Storage {
   }
   
   List<StorageDirectory> getRemovedStorageDirs() {
-	  return this.removedStorageDirs;
+    return this.removedStorageDirs;
   }
   
   File getEditFile(StorageDirectory sd) {
@@ -256,21 +295,39 @@ public class FSImage extends Storage {
     return list;
   }
 
-  Collection<File> getDirectories(NameNodeDirType dirType) {
-    ArrayList<File> list = new ArrayList<File>();
+  Collection<URI> getDirectories(NameNodeDirType dirType) 
+      throws IOException {
+    ArrayList<URI> list = new ArrayList<URI>();
     Iterator<StorageDirectory> it = (dirType == null) ? dirIterator() :
                                     dirIterator(dirType);
     for ( ;it.hasNext(); ) {
-      list.add(it.next().getRoot());
+      StorageDirectory sd = it.next();
+      try {
+        list.add(new URI("file://" + sd.getRoot().getAbsolutePath()));
+      } catch (Exception e) {
+        throw new IOException("Exception while processing " +
+            "StorageDirectory " + sd.getRoot().getAbsolutePath() + ". The"
+            + " full error message is " + e.getMessage());
+      }
     }
     return list;
   }
 
-  Collection<File> getImageDirectories() {
+  /**
+   * Retrieve current directories of type IMAGE
+   * @return Collection of URI representing image directories 
+   * @throws IOException in case of URI processing error
+   */
+  Collection<URI> getImageDirectories() throws IOException {
     return getDirectories(NameNodeDirType.IMAGE);
   }
 
-  Collection<File> getEditsDirectories() {
+  /**
+   * Retrieve current directories of type EDITS
+   * @return Collection of URI representing edits directories 
+   * @throws IOException in case of URI processing error
+   */
+  Collection<URI> getEditsDirectories() throws IOException {
     return getDirectories(NameNodeDirType.EDITS);
   }
 
@@ -300,8 +357,8 @@ public class FSImage extends Storage {
    * @throws IOException
    * @return true if the image needs to be saved or false otherwise
    */
-  boolean recoverTransitionRead(Collection<File> dataDirs,
-                             Collection<File> editsDirs,
+  boolean recoverTransitionRead(Collection<URI> dataDirs,
+                                 Collection<URI> editsDirs,
                                 StartupOption startOpt
                                 ) throws IOException {
     assert startOpt != StartupOption.FORMAT : 
@@ -740,7 +797,7 @@ public class FSImage extends Storage {
         }
       }
     }
-    // if there are some edit log streams to remove		
+    // if there are some edit log streams to remove    
     if(propagate && al != null) 
       editLog.processIOError(al, false);
     
@@ -1667,7 +1724,7 @@ public class FSImage extends Storage {
       return; //nothing to restore
     
     LOG.info("FSImage.attemptRestoreRemovedStorage: check removed(failed) " +
-    		"storarge. removedStorages size = " + removedStorageDirs.size());
+        "storarge. removedStorages size = " + removedStorageDirs.size());
     for(Iterator<StorageDirectory> it = this.removedStorageDirs.iterator(); it.hasNext();) {
       StorageDirectory sd = it.next();
       File root = sd.getRoot();
@@ -1823,31 +1880,63 @@ public class FSImage extends Storage {
         + FSConstants.LAYOUT_VERSION + " is initialized.");
   }
 
-  static Collection<File> getCheckpointDirs(Configuration conf,
-                                            String defaultName) {
+  /**
+   * Retrieve checkpoint dirs from configuration.
+   *  
+   * @param conf, the Configuration
+   * @param defaultValue, a default value for the attribute, if null
+   * @return a Collection of URIs representing the values in 
+   * fs.checkpoint.dir configuration property
+   */
+  static Collection<URI> getCheckpointDirs(Configuration conf,
+      String defaultValue) {
     Collection<String> dirNames = conf.getStringCollection("fs.checkpoint.dir");
-    if (dirNames.size() == 0 && defaultName != null) {
-      dirNames.add(defaultName);
+    if (dirNames.size() == 0 && defaultValue != null) {
+      dirNames.add(defaultValue);
     }
-    Collection<File> dirs = new ArrayList<File>(dirNames.size());
+    Collection<URI> dirs = new ArrayList<URI>(dirNames.size());
     for(String name : dirNames) {
-      dirs.add(new File(name));
+      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;
   }
   
-  static Collection<File> getCheckpointEditsDirs(Configuration conf,
-                                                 String defaultName) {
+  static Collection<URI> getCheckpointEditsDirs(Configuration conf,
+      String defaultName) {
     Collection<String> dirNames = 
-                conf.getStringCollection("fs.checkpoint.edits.dir");
- if (dirNames.size() == 0 && defaultName != null) {
-   dirNames.add(defaultName);
- }
- Collection<File> dirs = new ArrayList<File>(dirNames.size());
- for(String name : dirNames) {
-   dirs.add(new File(name));
- }
- return dirs;    
+      conf.getStringCollection("fs.checkpoint.edits.dir");
+    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;    
   }
 
   static private final DeprecatedUTF8 U_STR = new DeprecatedUTF8();

+ 21 - 6
src/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

@@ -69,6 +69,7 @@ import java.io.IOException;
 import java.io.PrintWriter;
 import java.io.DataOutputStream;
 import java.net.InetAddress;
+import java.net.URI;
 import java.util.*;
 import java.util.Map.Entry;
 
@@ -301,11 +302,11 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean {
     }
   }
 
-  public static Collection<File> getNamespaceDirs(Configuration conf) {
+  public static Collection<URI> getNamespaceDirs(Configuration conf) {
     return getStorageDirs(conf, "dfs.name.dir");
   }
 
-  public static Collection<File> getStorageDirs(Configuration conf,
+  public static Collection<URI> getStorageDirs(Configuration conf,
                                                 String propertyName) {
     Collection<String> dirNames = conf.getStringCollection(propertyName);
     StartupOption startOpt = NameNode.getStartupOption(conf);
@@ -331,14 +332,28 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean {
           "of the file system meta-data.");
     } else if (dirNames.isEmpty())
       dirNames.add("/tmp/hadoop/dfs/name");
-    Collection<File> dirs = new ArrayList<File>(dirNames.size());
+    Collection<URI> dirs = new ArrayList<URI>(dirNames.size());
     for(String name : dirNames) {
-      dirs.add(new File(name));
+      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;
   }
 
-  public static Collection<File> getNamespaceEditsDirs(Configuration conf) {
+  public static Collection<URI> getNamespaceEditsDirs(Configuration conf) {
     return getStorageDirs(conf, "dfs.name.edits.dir");
   }
 
@@ -3687,7 +3702,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean {
     boolean regAllowed = getEditLog().checkBackupRegistration(registration);
     if(!regAllowed)
       throw new IOException("Registration is not allowed. " +
-      		"Another node is registered as a backup.");
+          "Another node is registered as a backup.");
   }
 
   /**

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

@@ -1030,11 +1030,11 @@ public class NameNode implements ClientProtocol, DatanodeProtocol,
   private static boolean format(Configuration conf,
                                 boolean isConfirmationNeeded
                                 ) throws IOException {
-    Collection<File> dirsToFormat = FSNamesystem.getNamespaceDirs(conf);
-    Collection<File> editDirsToFormat = 
+    Collection<URI> dirsToFormat = FSNamesystem.getNamespaceDirs(conf);
+    Collection<URI> editDirsToFormat = 
                  FSNamesystem.getNamespaceEditsDirs(conf);
-    for(Iterator<File> it = dirsToFormat.iterator(); it.hasNext();) {
-      File curDir = it.next();
+    for(Iterator<URI> it = dirsToFormat.iterator(); it.hasNext();) {
+      File curDir = new File(it.next().getPath());
       if (!curDir.exists())
         continue;
       if (isConfirmationNeeded) {
@@ -1056,8 +1056,8 @@ public class NameNode implements ClientProtocol, DatanodeProtocol,
   private static boolean finalize(Configuration conf,
                                boolean isConfirmationNeeded
                                ) throws IOException {
-    Collection<File> dirsToFormat = FSNamesystem.getNamespaceDirs(conf);
-    Collection<File> editDirsToFormat = 
+    Collection<URI> dirsToFormat = FSNamesystem.getNamespaceDirs(conf);
+    Collection<URI> editDirsToFormat = 
                                FSNamesystem.getNamespaceEditsDirs(conf);
     FSNamesystem nsys = new FSNamesystem(new FSImage(dirsToFormat,
                                          editDirsToFormat), conf);

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

@@ -79,8 +79,8 @@ public class SecondaryNameNode implements Runnable {
   private int infoPort;
   private String infoBindAddress;
 
-  private Collection<File> checkpointDirs;
-  private Collection<File> checkpointEditsDirs;
+  private Collection<URI> checkpointDirs;
+  private Collection<URI> checkpointEditsDirs;
   private long checkpointPeriod;	// in seconds
   private long checkpointSize;    // size (in MB) of current Edit Log
 
@@ -489,10 +489,10 @@ public class SecondaryNameNode implements Runnable {
      * @param editsDirs
      * @throws IOException
      */
-    void recoverCreate(Collection<File> dataDirs,
-                       Collection<File> editsDirs) throws IOException {
-      Collection<File> tempDataDirs = new ArrayList<File>(dataDirs);
-      Collection<File> tempEditsDirs = new ArrayList<File>(editsDirs);
+    void recoverCreate(Collection<URI> dataDirs,
+                       Collection<URI> editsDirs) throws IOException {
+      Collection<URI> tempDataDirs = new ArrayList<URI>(dataDirs);
+      Collection<URI> tempEditsDirs = new ArrayList<URI>(editsDirs);
       this.storageDirs = new ArrayList<StorageDirectory>();
       setStorageDirectories(tempDataDirs, tempEditsDirs);
       for (Iterator<StorageDirectory> it = 

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

@@ -20,6 +20,7 @@ package org.apache.hadoop.hdfs;
 import java.io.File;
 import java.io.IOException;
 import java.net.InetSocketAddress;
+import java.net.URI;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.nio.channels.FileChannel;
@@ -704,14 +705,14 @@ public class MiniDFSCluster {
   /**
    * Get the directories where the namenode stores its image.
    */
-  public Collection<File> getNameDirs() {
+  public Collection<URI> getNameDirs() {
     return FSNamesystem.getNamespaceDirs(conf);
   }
 
   /**
    * Get the directories where the namenode stores its edits.
    */
-  public Collection<File> getNameEditsDirs() {
+  public Collection<URI> getNameEditsDirs() {
     return FSNamesystem.getNamespaceEditsDirs(conf);
   }
 

+ 13 - 4
src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/CreateEditsLog.java

@@ -19,6 +19,8 @@ package org.apache.hadoop.hdfs.server.namenode;
 
 import java.io.File;
 import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
 
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.PermissionStatus;
@@ -52,7 +54,7 @@ import org.apache.hadoop.hdfs.server.namenode.BlocksMap.BlockInfo;
 public class CreateEditsLog {
   static final String BASE_PATH = "/createdViaInjectingInEditsLog";
   static final String EDITS_DIR = "/tmp/EditsLogOut";
-  static String edits_dir = EDITS_DIR;
+  static String edits_dir = "file:// " + EDITS_DIR; // process as URI
   static final public long BLOCK_GENERATION_STAMP =
     GenerationStamp.FIRST_VALID_STAMP;
   
@@ -134,7 +136,8 @@ public class CreateEditsLog {
    * @param args
    * @throws IOException 
    */
-  public static void main(String[] args) throws IOException {
+  public static void main(String[] args) 
+      throws IOException {
 
 
 
@@ -194,8 +197,14 @@ public class CreateEditsLog {
         System.exit(-1);
       }
     }
-  
-    FSImage fsImage = new FSImage(new File(edits_dir));
+    
+    FSImage fsImage = null;
+    try {
+      fsImage = new FSImage(new URI(edits_dir));
+    } catch (URISyntaxException use) {
+      throw new IOException("Error while processing URI: " + edits_dir + 
+          ". The full error message was: " + use.getMessage());
+    }
     FileNameGenerator nameGenerator = new FileNameGenerator(BASE_PATH, 100);
 
 

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

@@ -75,7 +75,7 @@ public class TestBackupNode extends TestCase {
   }
 
   static String getBackupNodeDir(StartupOption t, int i) {
-    return BASE_DIR + "name" + t.getName() + i;
+    return BASE_DIR + "name" + t.getName() + i + "/";
   }
 
   BackupNode startBackupNode(Configuration conf,

+ 25 - 16
src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java

@@ -19,6 +19,7 @@ package org.apache.hadoop.hdfs.server.namenode;
 
 import junit.framework.TestCase;
 import java.io.*;
+import java.net.URI;
 import java.util.Collection;
 import java.util.List;
 import java.util.Iterator;
@@ -107,7 +108,7 @@ public class TestCheckpoint extends TestCase {
   /*
    * Verify that namenode does not startup if one namedir is bad.
    */
-  private void testNamedirError(Configuration conf, Collection<File> namedirs) 
+  private void testNamedirError(Configuration conf, Collection<URI> namedirs) 
     throws IOException {
     System.out.println("Starting testNamedirError");
     MiniDFSCluster cluster = null;
@@ -119,7 +120,7 @@ public class TestCheckpoint extends TestCase {
     //
     // Remove one namedir & Restart cluster. This should fail.
     //
-    File first = namedirs.iterator().next();
+    File first = new File(namedirs.iterator().next().getPath());
     removeOneNameDir(first);
     try {
       cluster = new MiniDFSCluster(conf, 0, false, null);
@@ -477,12 +478,13 @@ public class TestCheckpoint extends TestCase {
     
     // Remove current image and import a checkpoint.
     System.out.println("Import a checkpoint with existing primary image.");
-    List<File> nameDirs = (List<File>)FSNamesystem.getNamespaceDirs(conf);
-    List<File> nameEditsDirs = (List<File>)FSNamesystem.
+    List<URI> nameDirs = (List<URI>)FSNamesystem.getNamespaceDirs(conf);
+    List<URI> nameEditsDirs = (List<URI>)FSNamesystem.
                                   getNamespaceEditsDirs(conf);
-    long fsimageLength = new File(new File(nameDirs.get(0), "current"), 
+    long fsimageLength = new File(new File(nameDirs.get(0).getPath(), "current"), 
                                         NameNodeFile.IMAGE.getName()).length();
-    for(File dir : nameDirs) {
+    for(URI uri : nameDirs) {
+      File dir = new File(uri.getPath());
       if(dir.exists())
         if(!(FileUtil.fullyDelete(dir)))
           throw new IOException("Cannot remove directory: " + dir);
@@ -490,7 +492,8 @@ public class TestCheckpoint extends TestCase {
         throw new IOException("Cannot create directory " + dir);
     }
 
-    for(File dir : nameEditsDirs) {
+    for(URI uri : nameEditsDirs) {
+      File dir = new File(uri.getPath());
       if(dir.exists())
         if(!(FileUtil.fullyDelete(dir)))
           throw new IOException("Cannot remove directory: " + dir);
@@ -512,26 +515,30 @@ public class TestCheckpoint extends TestCase {
     // recover failed checkpoint
     nn = startNameNode(conf, primaryDirs, primaryEditsDirs,
                         StartupOption.REGULAR);
-    Collection<File> secondaryDirs = FSImage.getCheckpointDirs(conf, null);
-    for(File dir : secondaryDirs) {
+    Collection<URI> secondaryDirs = FSImage.getCheckpointDirs(conf, null);
+    for(URI uri : secondaryDirs) {
+      File dir = new File(uri.getPath());
       Storage.rename(new File(dir, "current"), 
                      new File(dir, "lastcheckpoint.tmp"));
     }
     secondary = startSecondaryNameNode(conf);
     secondary.shutdown();
-    for(File dir : secondaryDirs) {
+    for(URI uri : secondaryDirs) {
+      File dir = new File(uri.getPath());
       assertTrue(new File(dir, "current").exists()); 
       assertFalse(new File(dir, "lastcheckpoint.tmp").exists());
     }
     
     // complete failed checkpoint
-    for(File dir : secondaryDirs) {
+    for(URI uri : secondaryDirs) {
+      File dir = new File(uri.getPath());
       Storage.rename(new File(dir, "previous.checkpoint"), 
                      new File(dir, "lastcheckpoint.tmp"));
     }
     secondary = startSecondaryNameNode(conf);
     secondary.shutdown();
-    for(File dir : secondaryDirs) {
+    for(URI uri : secondaryDirs) {
+      File dir = new File(uri.getPath());
       assertTrue(new File(dir, "current").exists()); 
       assertTrue(new File(dir, "previous.checkpoint").exists()); 
       assertFalse(new File(dir, "lastcheckpoint.tmp").exists());
@@ -574,7 +581,7 @@ public class TestCheckpoint extends TestCase {
   public void testCheckpoint() throws IOException {
     Path file1 = new Path("checkpoint.dat");
     Path file2 = new Path("checkpoint2.dat");
-    Collection<File> namedirs = null;
+    Collection<URI> namedirs = null;
 
     Configuration conf = new Configuration();
     conf.set("dfs.secondary.http.address", "0.0.0.0:0");
@@ -690,8 +697,9 @@ public class TestCheckpoint extends TestCase {
       writeFile(fs, file, replication);
       checkFile(fs, file, replication);
       // verify that the edits file is NOT empty
-      Collection<File> editsDirs = cluster.getNameEditsDirs();
-      for(File ed : editsDirs) {
+      Collection<URI> editsDirs = cluster.getNameEditsDirs();
+      for(URI uri : editsDirs) {
+        File ed = new File(uri.getPath());
         assertTrue(new File(ed, "current/edits").length() > Integer.SIZE/Byte.SIZE);
       }
 
@@ -703,7 +711,8 @@ public class TestCheckpoint extends TestCase {
         throw new IOException(e);
       }
       // verify that the edits file is empty
-      for(File ed : editsDirs) {
+      for(URI uri : editsDirs) {
+        File ed = new File(uri.getPath());
         assertTrue(new File(ed, "current/edits").length() == Integer.SIZE/Byte.SIZE);
       }
 

+ 3 - 2
src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java

@@ -19,6 +19,7 @@ package org.apache.hadoop.hdfs.server.namenode;
 
 import junit.framework.TestCase;
 import java.io.*;
+import java.net.URI;
 import java.util.Iterator;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
@@ -92,8 +93,8 @@ public class TestEditLog extends TestCase {
       fileSys = cluster.getFileSystem();
       final FSNamesystem namesystem = cluster.getNamesystem();
   
-      for (Iterator<File> it = cluster.getNameDirs().iterator(); it.hasNext(); ) {
-        File dir = it.next();
+      for (Iterator<URI> it = cluster.getNameDirs().iterator(); it.hasNext(); ) {
+        File dir = new File(it.next().getPath());
         System.out.println(dir);
       }
   

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

@@ -2,6 +2,7 @@ package org.apache.hadoop.hdfs.server.namenode;
 
 import java.io.File;
 import java.io.IOException;
+import java.net.URI;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Random;
@@ -126,11 +127,11 @@ public class TestStartup extends TestCase {
    */
   private void corruptNameNodeFiles() throws IOException {
     // now corrupt/delete the directrory
-    List<File> nameDirs = (List<File>)FSNamesystem.getNamespaceDirs(config);
-    List<File> nameEditsDirs = (List<File>)FSNamesystem.getNamespaceEditsDirs(config);
+    List<URI> nameDirs = (List<URI>)FSNamesystem.getNamespaceDirs(config);
+    List<URI> nameEditsDirs = (List<URI>)FSNamesystem.getNamespaceEditsDirs(config);
 
     // get name dir and its length, then delete and recreate the directory
-    File dir = nameDirs.get(0); // has only one
+    File dir = new File(nameDirs.get(0).getPath()); // has only one
     this.fsimageLength = new File(new File(dir, "current"), 
         NameNodeFile.IMAGE.getName()).length();
 
@@ -142,7 +143,7 @@ public class TestStartup extends TestCase {
     if (!dir.mkdirs())
       throw new IOException("Cannot create directory " + dir);
 
-    dir = nameEditsDirs.get(0); //has only one
+    dir = new File( nameEditsDirs.get(0).getPath()); //has only one
 
     this.editsLength = new File(new File(dir, "current"), 
         NameNodeFile.EDITS.getName()).length();

+ 3 - 2
src/test/hdfs/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewer.java

@@ -28,6 +28,7 @@ import java.io.FileReader;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
+import java.net.URI;
 import java.util.HashMap;
 import java.util.Set;
 
@@ -124,8 +125,8 @@ public class TestOfflineImageViewer extends TestCase {
       cluster.getNameNode().saveNamespace();
       
       // Determine location of fsimage file
-      File [] files = cluster.getNameDirs().toArray(new File[0]);
-      orig =  new File(files[0], "current/fsimage");
+      URI [] files = cluster.getNameDirs().toArray(new URI[0]);
+      orig =  new File(files[0].getPath(), "current/fsimage");
       
       if(!orig.exists())
         fail("Didn't generate or can't find fsimage.");