Преглед изворни кода

HADOOP-702. Add tools to help automate HDFS upgrades. Contributed by Konstantin.

git-svn-id: https://svn.apache.org/repos/asf/lucene/hadoop/trunk@525290 13f79535-47bb-0310-9956-ffa450edef68
Doug Cutting пре 18 година
родитељ
комит
8355af78e5
34 измењених фајлова са 4116 додато и 988 уклоњено
  1. 3 0
      CHANGES.txt
  2. 25 3
      bin/start-dfs.sh
  3. 46 27
      src/java/org/apache/hadoop/dfs/BlockCommand.java
  4. 11 4
      src/java/org/apache/hadoop/dfs/ClientProtocol.java
  5. 31 0
      src/java/org/apache/hadoop/dfs/DFSAdmin.java
  6. 7 0
      src/java/org/apache/hadoop/dfs/DFSClient.java
  7. 227 174
      src/java/org/apache/hadoop/dfs/DataNode.java
  8. 353 157
      src/java/org/apache/hadoop/dfs/DataStorage.java
  9. 19 10
      src/java/org/apache/hadoop/dfs/DatanodeProtocol.java
  10. 24 12
      src/java/org/apache/hadoop/dfs/DatanodeRegistration.java
  11. 16 0
      src/java/org/apache/hadoop/dfs/DistributedFileSystem.java
  12. 16 4
      src/java/org/apache/hadoop/dfs/FSConstants.java
  13. 9 9
      src/java/org/apache/hadoop/dfs/FSDataset.java
  14. 17 8
      src/java/org/apache/hadoop/dfs/FSDirectory.java
  15. 108 206
      src/java/org/apache/hadoop/dfs/FSEditLog.java
  16. 620 252
      src/java/org/apache/hadoop/dfs/FSImage.java
  17. 30 18
      src/java/org/apache/hadoop/dfs/FSNamesystem.java
  18. 47 0
      src/java/org/apache/hadoop/dfs/InconsistentFSStateException.java
  19. 26 3
      src/java/org/apache/hadoop/dfs/IncorrectVersionException.java
  20. 148 72
      src/java/org/apache/hadoop/dfs/NameNode.java
  21. 75 0
      src/java/org/apache/hadoop/dfs/NamespaceInfo.java
  22. 7 14
      src/java/org/apache/hadoop/dfs/SecondaryNameNode.java
  23. 538 0
      src/java/org/apache/hadoop/dfs/Storage.java
  24. 60 0
      src/java/org/apache/hadoop/fs/FileUtil.java
  25. 2 2
      src/test/org/apache/hadoop/dfs/ClusterTestDFS.java
  26. 1 2
      src/test/org/apache/hadoop/dfs/ClusterTestDFSNamespaceLogging.java
  27. 43 10
      src/test/org/apache/hadoop/dfs/MiniDFSCluster.java
  28. 119 0
      src/test/org/apache/hadoop/dfs/TestDFSFinalize.java
  29. 232 0
      src/test/org/apache/hadoop/dfs/TestDFSRollback.java
  30. 206 0
      src/test/org/apache/hadoop/dfs/TestDFSStartupVersions.java
  31. 245 0
      src/test/org/apache/hadoop/dfs/TestDFSStorageStateRecovery.java
  32. 230 0
      src/test/org/apache/hadoop/dfs/TestDFSUpgrade.java
  33. 1 1
      src/test/org/apache/hadoop/dfs/TestFileCorruption.java
  34. 574 0
      src/test/org/apache/hadoop/dfs/UpgradeUtilities.java

+ 3 - 0
CHANGES.txt

@@ -94,6 +94,9 @@ Trunk (unreleased changes)
 29. HADOOP-1156.  Fix a NullPointerException in MiniDFSCluster.
     (Hairong Kuang via cutting)
 
+30. HADOOP-702.  Add tools to help automate HDFS upgrades.
+    (Konstantin Shvachko via cutting)
+
 
 Release 0.12.3 (not yet released)
 

+ 25 - 3
bin/start-dfs.sh

@@ -1,15 +1,37 @@
 #!/bin/sh
 
-# Start hadoop dfs daemons.  Run this on master node.
+# Start hadoop dfs daemons.
+# Optinally upgrade or rollback dfs state.
+# Run this on master node.
+##
+
+usage="Usage: start-dfs.sh [-upgrade|-rollback]"
 
 bin=`dirname "$0"`
 bin=`cd "$bin"; pwd`
 
 . "$bin"/hadoop-config.sh
 
+# get arguments
+if [ $# -ge 1 ]; then
+	nameStartOpt=$1
+	shift
+	case $nameStartOpt in
+	  (-upgrade)
+	  	;;
+	  (-rollback) 
+	  	dataStartOpt=$nameStartOpt
+	  	;;
+	  (*)
+		  echo $usage
+		  exit 1
+	    ;;
+	esac
+fi
+
 # start dfs daemons
 # start namenode after datanodes, to minimize time namenode is up w/o data
 # note: datanodes will log connection errors until namenode starts
-"$bin"/hadoop-daemon.sh --config $HADOOP_CONF_DIR start namenode
-"$bin"/hadoop-daemons.sh --config $HADOOP_CONF_DIR start datanode
+"$bin"/hadoop-daemon.sh --config $HADOOP_CONF_DIR start namenode $nameStartOpt
+"$bin"/hadoop-daemons.sh --config $HADOOP_CONF_DIR start datanode $dataStartOpt
 "$bin"/hadoop-daemons.sh --config $HADOOP_CONF_DIR --hosts masters start secondarynamenode

+ 46 - 27
src/java/org/apache/hadoop/dfs/BlockCommand.java

@@ -17,9 +17,40 @@
  */
 package org.apache.hadoop.dfs;
 
+import java.io.*;
 import org.apache.hadoop.io.*;
 
-import java.io.*;
+class DatanodeCommand implements Writable {
+  DatanodeProtocol.DataNodeAction action;
+  
+  public DatanodeCommand() {
+    this( DatanodeProtocol.DataNodeAction.DNA_UNKNOWN );
+  }
+  
+  public DatanodeCommand( DatanodeProtocol.DataNodeAction action ) {
+    this.action = action;
+  }
+
+  ///////////////////////////////////////////
+  // Writable
+  ///////////////////////////////////////////
+  static {                                      // register a ctor
+    WritableFactories.setFactory
+      (BlockCommand.class,
+       new WritableFactory() {
+         public Writable newInstance() { return new DatanodeCommand(); }
+       });
+  }
+
+  public void write(DataOutput out) throws IOException {
+    WritableUtils.writeEnum( out, action );
+  }
+  
+  public void readFields(DataInput in) throws IOException {
+    this.action = (DatanodeProtocol.DataNodeAction)
+          WritableUtils.readEnum( in, DatanodeProtocol.DataNodeAction.class );
+  }
+}
 
 /****************************************************
  * A BlockCommand is an instruction to a datanode 
@@ -30,25 +61,11 @@ import java.io.*;
  * 
  * @author Mike Cafarella
  ****************************************************/
-class BlockCommand implements Writable {
-
-    static {                                      // register a ctor
-      WritableFactories.setFactory
-        (BlockCommand.class,
-         new WritableFactory() {
-           public Writable newInstance() { return new BlockCommand(); }
-         });
-    }
-
-    DatanodeProtocol.DataNodeAction action;
+class BlockCommand extends DatanodeCommand {
     Block blocks[];
     DatanodeInfo targets[][];
 
-    public BlockCommand() {
-      this.action = DatanodeProtocol.DataNodeAction.DNA_UNKNOWN;
-      this.blocks = new Block[0];
-      this.targets = new DatanodeInfo[0][];
-    }
+    public BlockCommand() {}
 
     /**
      * Create BlockCommand for transferring blocks to another datanode
@@ -56,7 +73,7 @@ class BlockCommand implements Writable {
      * @param targets   nodes to transfer
      */
     public BlockCommand(Block blocks[], DatanodeInfo targets[][]) {
-      this.action = DatanodeProtocol.DataNodeAction.DNA_TRANSFER;
+      super(  DatanodeProtocol.DataNodeAction.DNA_TRANSFER );
       this.blocks = blocks;
       this.targets = targets;
     }
@@ -66,16 +83,11 @@ class BlockCommand implements Writable {
      * @param blocks  blocks to invalidate
      */
     public BlockCommand(Block blocks[]) {
-      this.action = DatanodeProtocol.DataNodeAction.DNA_INVALIDATE;
+      super( DatanodeProtocol.DataNodeAction.DNA_INVALIDATE );
       this.blocks = blocks;
       this.targets = new DatanodeInfo[0][];
     }
 
-    public BlockCommand( DatanodeProtocol.DataNodeAction action ) {
-      this();
-      this.action = action;
-    }
-
     public Block[] getBlocks() {
         return blocks;
     }
@@ -87,8 +99,16 @@ class BlockCommand implements Writable {
     ///////////////////////////////////////////
     // Writable
     ///////////////////////////////////////////
+    static {                                      // register a ctor
+      WritableFactories.setFactory
+        (BlockCommand.class,
+         new WritableFactory() {
+           public Writable newInstance() { return new BlockCommand(); }
+         });
+    }
+
     public void write(DataOutput out) throws IOException {
-        WritableUtils.writeEnum( out, action );
+        super.write( out );
         out.writeInt(blocks.length);
         for (int i = 0; i < blocks.length; i++) {
             blocks[i].write(out);
@@ -103,8 +123,7 @@ class BlockCommand implements Writable {
     }
 
     public void readFields(DataInput in) throws IOException {
-        this.action = (DatanodeProtocol.DataNodeAction)
-            WritableUtils.readEnum( in, DatanodeProtocol.DataNodeAction.class );
+        super.readFields( in );
         this.blocks = new Block[in.readInt()];
         for (int i = 0; i < blocks.length; i++) {
             blocks[i] = new Block();

+ 11 - 4
src/java/org/apache/hadoop/dfs/ClientProtocol.java

@@ -29,11 +29,10 @@ import org.apache.hadoop.ipc.VersionedProtocol;
  **********************************************************************/
 interface ClientProtocol extends VersionedProtocol {
 
-    /* 7 : periodic checkpoint added.
-     * 8 : refreshNodes added
-     * 9 : clientMachine is removed from open() and create().
+    /*
+     * 10: finalizeUpgrade() added
      */
-    public static final long versionID = 9L;  
+    public static final long versionID = 10L;  
   
     ///////////////////////////////////////
     // File contents
@@ -348,4 +347,12 @@ interface ClientProtocol extends VersionedProtocol {
      */
     public void rollFsImage() throws IOException;
 
+    /**
+     * Finalize previous upgrade.
+     * Remove file system state saved during the upgrade.
+     * The upgrade will become irreversible.
+     * 
+     * @throws IOException
+     */
+    public void finalizeUpgrade() throws IOException;
 }

+ 31 - 0
src/java/org/apache/hadoop/dfs/DFSAdmin.java

@@ -196,6 +196,26 @@ public class DFSAdmin extends FsShell {
     }
 
 
+    /**
+     * Command to ask the namenode to finalize previously performed upgrade.
+     * Usage: java DFSAdmin -finalizeUpgrade
+     * @exception IOException 
+     */
+    public int finalizeUpgrade() throws IOException {
+      int exitCode = -1;
+
+      if (!(fs instanceof DistributedFileSystem)) {
+        System.out.println("FileSystem is " + fs.getUri());
+        return exitCode;
+      }
+
+      DistributedFileSystem dfs = (DistributedFileSystem) fs;
+      dfs.finalizeUpgrade();
+      exitCode = 0;
+   
+      return exitCode;
+    }
+
     /**
      * Displays format of commands.
      * @param cmd The command that is being executed.
@@ -210,11 +230,15 @@ public class DFSAdmin extends FsShell {
           } else if ("-refreshNodes".equals(cmd)) {
             System.err.println("Usage: java DFSAdmin"
                 + " [-refreshNodes]");
+          } else if ("-finalizeUpgrade".equals(cmd)) {
+            System.err.println("Usage: java DFSAdmin"
+                + " [-finalizeUpgrade]");
           } else {
             System.err.println("Usage: java DFSAdmin");
             System.err.println("           [-report]");
             System.err.println("           [-safemode enter | leave | get | wait]");
             System.err.println("           [-refreshNodes]");
+            System.err.println("           [-finalizeUpgrade]");
             System.err.println("           [-help [cmd]]");
           }
     }
@@ -253,6 +277,11 @@ public class DFSAdmin extends FsShell {
                   printUsage(cmd);
                   return exitCode;
                 }
+        } else if ("-finalizeUpgrade".equals(cmd)) {
+                if (argv.length != 1) {
+                  printUsage(cmd);
+                  return exitCode;
+                }
         }
 
 
@@ -276,6 +305,8 @@ public class DFSAdmin extends FsShell {
                 setSafeMode(argv, i);
             } else if ("-refreshNodes".equals(cmd)) {
                 exitCode = refreshNodes();
+            } else if ("-finalizeUpgrade".equals(cmd)) {
+                exitCode = finalizeUpgrade();
             } else if ("-help".equals(cmd)) {
                 if (i < argv.length) {
                     printHelp(argv[i]);

+ 7 - 0
src/java/org/apache/hadoop/dfs/DFSClient.java

@@ -376,6 +376,13 @@ class DFSClient implements FSConstants {
       namenode.refreshNodes();
     }
     
+    /**
+     * @see ClientProtocol#finalizeUpgrade()
+     */
+    public void finalizeUpgrade() throws IOException {
+      namenode.finalizeUpgrade();
+    }
+
     /**
      */
     public boolean mkdirs(UTF8 src) throws IOException {

+ 227 - 174
src/java/org/apache/hadoop/dfs/DataNode.java

@@ -106,18 +106,20 @@ public class DataNode implements FSConstants, Runnable {
         return new InetSocketAddress(hostname, port);
     }
 
-    DatanodeProtocol namenode;
-    FSDataset data;
-    DatanodeRegistration dnRegistration;
+    DatanodeProtocol namenode = null;
+    FSDataset data = null;
+    DatanodeRegistration dnRegistration = null;
     private String networkLoc;
     volatile boolean shouldRun = true;
     Vector receivedBlockList = new Vector();
     int xmitsInProgress = 0;
     Daemon dataXceiveServer = null;
     long blockReportInterval;
+    long lastBlockReport = 0;
+    long lastHeartbeat = 0;
     long heartBeatInterval;
     private DataStorage storage = null;
-    private StatusHttpServer infoServer;
+    private StatusHttpServer infoServer = null;
     private DataNodeMetrics myMetrics = new DataNodeMetrics();
     private static InetSocketAddress nameNodeAddr;
     private static DataNode datanodeObject = null;
@@ -187,73 +189,57 @@ public class DataNode implements FSConstants, Runnable {
     }
 
     /**
-     * Create the DataNode given a configuration and an array of dataDirs.
-     * 'dataDirs' is where the blocks are stored.
+     * @deprecated
+     * TODO: only MiniDFSCluster needs it, should be removed
      */
-    DataNode(Configuration conf, String[] dataDirs) throws IOException {
-      this(conf, NetworkTopology.DEFAULT_RACK, dataDirs );
-    }
-    
-    DataNode(Configuration conf, String networkLoc, String[] dataDirs) throws IOException {
-        this(networkLoc, dataDirs,
-             createSocketAddr(conf.get("fs.default.name", "local")), conf);
-        // register datanode
-        int infoServerPort = conf.getInt("dfs.datanode.info.port", 50075);
-        String infoServerBindAddress = conf.get("dfs.datanode.info.bindAddress", "0.0.0.0");
-        this.infoServer = new StatusHttpServer("datanode", infoServerBindAddress, infoServerPort, true);
-        //create a servlet to serve full-file content
-        this.infoServer.addServlet(null, "/streamFile/*", StreamFile.class);
-        this.infoServer.start();
-        this.dnRegistration.infoPort = this.infoServer.getPort();
-        // register datanode
-        try {
-          register();
-        } catch (IOException ie) {
-          try {
-            infoServer.stop();
-          } catch (Exception e) {
-          }
-          throw ie;
-        }
-        datanodeObject = this;
+    DataNode( Configuration conf, String networkLoc, String[] dataDirs ) throws IOException {
+      // networkLoc is ignored since it is already in the conf
+      this( conf, Storage.makeListOfFiles( dataDirs ) );
     }
     
     /**
-     * A DataNode can also be created with configuration information
-     * explicitly given.
-     * 
-     * @see DataStorage
+     * Create the DataNode given a configuration and an array of dataDirs.
+     * 'dataDirs' is where the blocks are stored.
      */
-    private DataNode(String networkLoc,
-                    String[] dataDirs, 
-                    InetSocketAddress nameNodeAddr, 
-                    Configuration conf ) throws IOException {
-      File[] volumes = new File[dataDirs.length];
-      for (int idx = 0; idx < dataDirs.length; idx++) {
-        volumes[idx] = new File(dataDirs[idx]);
+    DataNode( Configuration conf, 
+              AbstractList<File> dataDirs ) throws IOException {
+      try {
+        startDataNode( conf, dataDirs );
+      } catch (IOException ie) {
+        shutdown();
+        throw ie;
       }
-
+    }
+    
+    void startDataNode( Configuration conf, 
+                        AbstractList<File> dataDirs
+                       ) throws IOException {
       // use configured nameserver & interface to get local hostname
-      machineName =
-        DNS.getDefaultHost
-        (conf.get("dfs.datanode.dns.interface","default"),
-         conf.get("dfs.datanode.dns.nameserver","default"));
- 
-      // get storage info and lock the data dirs
-      storage = new DataStorage( volumes );
-      int numDirs = storage.getNumLocked();
-      if (numDirs == 0) { // all data dirs are in use
-        throw new IOException("Cannot start multiple Datanode instances "
-                              + "sharing the same data directories.\n"
-                              + StringUtils.arrayToString(dataDirs) + " are locked. ");
-      }
-      volumes = storage.getLockedDirs();
+      machineName = DNS.getDefaultHost(
+          conf.get("dfs.datanode.dns.interface","default"),
+          conf.get("dfs.datanode.dns.nameserver","default"));
+      InetSocketAddress nameNodeAddr = createSocketAddr(
+          conf.get("fs.default.name", "local"));
+
       // connect to name node
       this.namenode = (DatanodeProtocol) 
           RPC.waitForProxy(DatanodeProtocol.class,
                            DatanodeProtocol.versionID,
                            nameNodeAddr, 
                            conf);
+      // get version and id info from the name-node
+      NamespaceInfo nsInfo = handshake();
+
+      // read storage info, lock data dirs and transition fs state if necessary
+      StartupOption startOpt = (StartupOption)conf.get( "dfs.datanode.startup", 
+                                                        StartupOption.REGULAR );
+      assert startOpt != null : "Startup option must be set.";
+      storage = new DataStorage();
+      storage.recoverTransitionRead( nsInfo, dataDirs, startOpt );
+      
+      // initialize data node internal structure
+      this.data = new FSDataset( storage, conf );
+      
       // find free port
       ServerSocket ss = null;
       int tmpPort = conf.getInt("dfs.datanode.port", 50010);
@@ -268,15 +254,11 @@ public class DataNode implements FSConstants, Runnable {
         }
       }
       // construct registration
-      this.dnRegistration = new DatanodeRegistration(
-                                        DFS_CURRENT_VERSION, 
-                                        machineName + ":" + tmpPort, 
-                                        storage.getStorageID(),
-                                        -1,
-                                        "" );
-      this.networkLoc = networkLoc;
-      // initialize data node internal structure
-      this.data = new FSDataset(volumes, conf);
+      this.dnRegistration = new DatanodeRegistration( 
+                                    machineName + ":" + tmpPort, 
+                                    -1,   // info port determined later
+                                    storage );
+      
       this.dataXceiveServer = new Daemon(new DataXceiveServer(ss));
 
       long blockReportIntervalBasis =
@@ -284,7 +266,55 @@ public class DataNode implements FSConstants, Runnable {
       this.blockReportInterval =
         blockReportIntervalBasis - new Random().nextInt((int)(blockReportIntervalBasis/10));
       this.heartBeatInterval = conf.getLong("dfs.heartbeat.interval", HEARTBEAT_INTERVAL) * 1000L;
-      this.nameNodeAddr = nameNodeAddr;
+      DataNode.nameNodeAddr = nameNodeAddr;
+
+      //create a servlet to serve full-file content
+      int infoServerPort = conf.getInt("dfs.datanode.info.port", 50075);
+      String infoServerBindAddress = conf.get("dfs.datanode.info.bindAddress", "0.0.0.0");
+      this.infoServer = new StatusHttpServer("datanode", infoServerBindAddress, infoServerPort, true);
+      this.infoServer.addServlet(null, "/streamFile/*", StreamFile.class);
+      this.infoServer.start();
+      this.dnRegistration.infoPort = this.infoServer.getPort();
+      // get network location
+      this.networkLoc = conf.get( "dfs.datanode.rack" );
+      if( networkLoc == null )  // exec network script or set the default rack
+        networkLoc = getNetworkLoc( conf );
+      // register datanode
+      register();
+      datanodeObject = this;
+    }
+
+    private NamespaceInfo handshake() throws IOException {
+      NamespaceInfo nsInfo;
+      while( true ) {
+        try {
+          nsInfo = namenode.versionRequest();
+          break;
+        } catch( SocketTimeoutException e ) {  // namenode is busy
+          LOG.info("Problem connecting to server: " + getNameNodeAddr());
+          try {
+            Thread.sleep(1000);
+          } catch (InterruptedException ie) {}
+        }
+      }
+      String errorMsg = null;
+      // verify build version
+      if( ! nsInfo.getBuildVersion().equals( Storage.getBuildVersion() )) {
+        errorMsg = "Incompatible build versions: namenode BV = " 
+          + nsInfo.getBuildVersion() + "; datanode BV = "
+          + Storage.getBuildVersion();
+        LOG.fatal( errorMsg );
+        try {
+          namenode.errorReport( dnRegistration,
+                                DatanodeProtocol.NOTIFY, errorMsg );
+        } catch( SocketTimeoutException e ) {  // namenode is busy
+          LOG.info("Problem connecting to server: " + getNameNodeAddr());
+        }
+        throw new IOException( errorMsg );
+      }
+      assert FSConstants.LAYOUT_VERSION == nsInfo.getLayoutVersion() :
+        "Data-node and name-node layout versions must be the same.";
+      return nsInfo;
     }
 
     /** Return the DataNode object
@@ -314,7 +344,7 @@ public class DataNode implements FSConstants, Runnable {
      * 2) to receive a registrationID 
      * issued by the namenode to recognize registered datanodes.
      * 
-     * @see FSNamesystem#registerDatanode(DatanodeRegistration)
+     * @see FSNamesystem#registerDatanode(DatanodeRegistration,String)
      * @throws IOException
      */
     private void register() throws IOException {
@@ -355,7 +385,7 @@ public class DataNode implements FSConstants, Runnable {
         }
         if (storage != null) {
           try {
-            this.storage.closeAll();
+            this.storage.unlockAll();
           } catch (IOException ie) {
           }
         }
@@ -388,7 +418,6 @@ public class DataNode implements FSConstants, Runnable {
      */
     public void offerService() throws Exception {
      
-      long lastHeartbeat = 0, lastBlockReport = 0;
       LOG.info("using BLOCKREPORT_INTERVAL of " + blockReportInterval + "msec");
 
       //
@@ -410,47 +439,17 @@ public class DataNode implements FSConstants, Runnable {
             // -- Total capacity
             // -- Bytes remaining
             //
-            BlockCommand cmd = namenode.sendHeartbeat(dnRegistration, 
+            DatanodeCommand cmd = namenode.sendHeartbeat( dnRegistration, 
                                                       data.getCapacity(), 
                                                       data.getRemaining(), 
                                                       xmitsInProgress,
                                                       xceiverCount.getValue());
             //LOG.info("Just sent heartbeat, with name " + localName);
             lastHeartbeat = now;
-
-            if( cmd != null ) {
-              switch( cmd.action ) {
-              case DNA_TRANSFER:
-                //
-                // Send a copy of a block to another datanode
-                //
-                transferBlocks( cmd.getBlocks(), cmd.getTargets() );
-                break;
-              case DNA_INVALIDATE:
-                //
-                // Some local block(s) are obsolete and can be 
-                // safely garbage-collected.
-                //
-                Block toDelete[] = cmd.getBlocks();
-                data.invalidate(toDelete);
-                myMetrics.removedBlocks(toDelete.length);
-                break;
-              case DNA_SHUTDOWN:
-                // shut down the data node
-                this.shutdown();
-                continue;
-              case DNA_REGISTER:
-                // namenode requested a registration
-                register();
-                lastHeartbeat=0;
-                lastBlockReport=0;
-                continue;
-              default:
-                LOG.warn( "Unknown BlockCommand action: " + cmd.action);
-              }
-            }
+            if( ! processCommand( cmd ) )
+              continue;
           }
-            
+
           // send block report
           if (now - lastBlockReport > blockReportInterval) {
             //
@@ -458,9 +457,9 @@ public class DataNode implements FSConstants, Runnable {
             // Get back a list of local block(s) that are obsolete
             // and can be safely GC'ed.
             //
-            Block toDelete[] = namenode.blockReport(dnRegistration,
-                                                    data.getBlockReport());
-            data.invalidate(toDelete);
+            DatanodeCommand cmd = namenode.blockReport( dnRegistration,
+                                                        data.getBlockReport());
+            processCommand( cmd );
             lastBlockReport = now;
             continue;
           }
@@ -516,6 +515,51 @@ public class DataNode implements FSConstants, Runnable {
       } // while (shouldRun)
     } // offerService
 
+    /**
+     * 
+     * @param cmd
+     * @return true if further processing may be required or false otherwise. 
+     * @throws IOException
+     */
+    private boolean processCommand( DatanodeCommand cmd ) throws IOException {
+      if( cmd == null )
+        return true;
+      switch( cmd.action ) {
+      case DNA_TRANSFER:
+        //
+        // Send a copy of a block to another datanode
+        //
+        BlockCommand bcmd = (BlockCommand)cmd;
+        transferBlocks( bcmd.getBlocks(), bcmd.getTargets() );
+        break;
+      case DNA_INVALIDATE:
+        //
+        // Some local block(s) are obsolete and can be 
+        // safely garbage-collected.
+        //
+        Block toDelete[] = ((BlockCommand)cmd).getBlocks();
+        data.invalidate(toDelete);
+        myMetrics.removedBlocks(toDelete.length);
+        break;
+      case DNA_SHUTDOWN:
+        // shut down the data node
+        this.shutdown();
+        return false;
+      case DNA_REGISTER:
+        // namenode requested a registration
+        register();
+        lastHeartbeat=0;
+        lastBlockReport=0;
+        break;
+      case DNA_FINALIZE:
+        storage.finalizeUpgrade();
+        break;
+      default:
+        LOG.warn( "Unknown DatanodeCommand action: " + cmd.action);
+      }
+      return true;
+    }
+    
     private void transferBlocks(  Block blocks[], 
                                   DatanodeInfo xferTargets[][] 
                                 ) throws IOException {
@@ -1074,9 +1118,9 @@ public class DataNode implements FSConstants, Runnable {
     
     /** Start datanode daemon.
      */
-    public static void run(Configuration conf, String networkLoc) throws IOException {
+    public static void run(Configuration conf) throws IOException {
         String[] dataDirs = conf.getStrings("dfs.data.dir");
-        DataNode dn = makeInstance(networkLoc, dataDirs, conf);
+        DataNode dn = makeInstance(dataDirs, conf);
         if (dn != null) {
           dataNodeList.add(dn);
           Thread t = new Thread(dn, "DataNode: [" +
@@ -1087,25 +1131,34 @@ public class DataNode implements FSConstants, Runnable {
         }
     }
     
-    /**
-     * Shut down all datanodes that where started via the 
-     * run(conf,networkLoc) method.
-     * Returns only after shutdown is complete.
-     */
-    public static void shutdownAll(){
-      while (!dataNodeList.isEmpty()) {
-        dataNodeList.remove(0).shutdown();
-        dataNodeThreadList.remove(0).interrupt();
-      }
+  /**
+   * Shut down all datanodes that where started via the 
+   * run(conf,networkLoc) method.
+   * Returns only after shutdown is complete.
+   */
+  public static void shutdownAll(){
+    while (!dataNodeList.isEmpty()) {
+      dataNodeList.remove(0).shutdown();
+      dataNodeThreadList.remove(0).interrupt();
     }
-
+  }
 
   /** Start a single datanode daemon and wait for it to finish.
    *  If this thread is specifically interrupted, it will stop waiting.
    */
-  private static void runAndWait(Configuration conf, String networkLoc)
-    throws IOException {
-    run(conf, networkLoc);
+  static DataNode createDataNode( String args[],
+                                  Configuration conf ) throws IOException {
+    if( conf == null )
+      conf = new Configuration();
+    if( ! parseArguments( args, conf )) {
+      printUsage();
+      return null;
+    }
+    run(conf);
+    return (DataNode)dataNodeList.get(0);
+  }
+
+  void join() {
     if (dataNodeThreadList.size() > 0) {
       Thread t = (Thread) dataNodeThreadList.remove(dataNodeThreadList.size()-1);
       try {
@@ -1125,24 +1178,22 @@ public class DataNode implements FSConstants, Runnable {
    * no directory from this directory list can be created.
    * @throws IOException
    */
-  static DataNode makeInstance( String[] dataDirs, Configuration conf)
-  throws IOException {
-    return makeInstance(NetworkTopology.DEFAULT_RACK, dataDirs, conf );
-  }
-  
-  static DataNode makeInstance(String networkLoc, String[] dataDirs, Configuration conf)
+  static DataNode makeInstance( String[] dataDirs, Configuration conf )
   throws IOException {
-    ArrayList<String> dirs = new ArrayList<String>();
+    ArrayList<File> dirs = new ArrayList<File>();
     for (int i = 0; i < dataDirs.length; i++) {
       File data = new File(dataDirs[i]);
       try {
         DiskChecker.checkDir( data );
-        dirs.add(dataDirs[i]);
+        dirs.add(data);
       } catch( DiskErrorException e ) {
         LOG.warn("Invalid directory in dfs.data.dir: " + e.getMessage() );
       }
     }
-    return ((dirs.size() > 0) ? new DataNode(conf, networkLoc, dirs.toArray(new String[dirs.size()])) : null);
+    if( dirs.size() > 0 ) 
+      return new DataNode(conf, dirs);
+    LOG.error("All directories in dfs.data.dir are invalid." );
+    return null;
   }
 
   public String toString() {
@@ -1154,11 +1205,49 @@ public class DataNode implements FSConstants, Runnable {
         "}";
   }
   
+  private static void printUsage() {
+    System.err.println("Usage: java DataNode");
+    System.err.println("           [-r, --rack <network location>] |");
+    System.err.println("           [-rollback]");
+  }
+
+  /**
+   * Parse and verify command line arguments and set configuration parameters.
+   *
+   * @return false if passed argements are incorrect
+   */
+  private static boolean parseArguments(String args[], 
+                                        Configuration conf ) {
+    int argsLen = (args == null) ? 0 : args.length;
+    StartupOption startOpt = StartupOption.REGULAR;
+    String networkLoc = null;
+    for( int i=0; i < argsLen; i++ ) {
+      String cmd = args[i];
+      if( "-r".equalsIgnoreCase(cmd) || "--rack".equalsIgnoreCase(cmd) ) {
+        if( i==args.length-1 )
+          return false;
+        networkLoc = args[++i];
+        if( networkLoc.startsWith("-") )
+          return false;
+      } else if( "-rollback".equalsIgnoreCase(cmd) ) {
+        startOpt = StartupOption.ROLLBACK;
+      } else if( "-regular".equalsIgnoreCase(cmd) ) {
+        startOpt = StartupOption.REGULAR;
+      } else
+        return false;
+    }
+    if( networkLoc != null )
+      conf.set( "dfs.datanode.rack", NodeBase.normalize( networkLoc ));
+    conf.setObject( "dfs.datanode.startup", startOpt );
+    return true;
+  }
+
     /* Get the network location by running a script configured in conf */
     private static String getNetworkLoc( Configuration conf ) 
                           throws IOException {
         String locScript = conf.get("dfs.network.script" );
-        if( locScript == null ) return null;
+        if( locScript == null ) 
+          return NetworkTopology.DEFAULT_RACK;
 
         LOG.info( "Starting to run script to get datanode network location");
         Process p = Runtime.getRuntime().exec( locScript );
@@ -1222,49 +1311,13 @@ public class DataNode implements FSConstants, Runnable {
         return networkLoc.toString();
     }
 
-
-    /* Get the network location from the command line */
-    private static String getNetworkLoc(String args[]) {
-        for( int i=0; i< args.length; i++ ) { 
-            if ("-r".equals(args[i])||"--rack".equals(args[i]) ) {
-                if( i==args.length-1 ) {
-                    printUsage();
-                } else {
-                    return args[++i];
-                }
-            }
-        }
-        return null;
-    }
-    
-    /* Return the datanode's network location 
-     * either from the command line, from script, or a default value
-     */
-    private static String getNetworkLoc(String args[], Configuration conf)
-                          throws IOException {
-        String networkLoc = getNetworkLoc( args );
-        if( networkLoc == null ) {
-            networkLoc = getNetworkLoc( conf );
-        }
-        if( networkLoc == null ) {
-            return NetworkTopology.DEFAULT_RACK;
-        } else {
-            return NodeBase.normalize( networkLoc );
-        }
-    }
-    
-    private static void printUsage() {
-        System.err.println(
-                "Usage: java DataNode [-r, --rack <network location>]");        
-    }
-
-
     /**
      */
-    public static void main(String args[]) throws IOException {
+    public static void main(String args[]) {
       try {
-        Configuration conf = new Configuration();
-        runAndWait(conf, getNetworkLoc(args, conf));
+        DataNode datanode = createDataNode( args, null );
+        if( datanode != null )
+          datanode.join();
       } catch ( Throwable e ) {
         LOG.error( StringUtils.stringifyException( e ) );
         System.exit(-1);

+ 353 - 157
src/java/org/apache/hadoop/dfs/DataStorage.java

@@ -4,213 +4,409 @@ import java.io.File;
 import java.io.IOException;
 import java.io.RandomAccessFile;
 import java.nio.channels.FileLock;
+import java.util.Collection;
 import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.Properties;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
-import org.apache.hadoop.io.UTF8;
+import org.apache.hadoop.dfs.FSConstants.StartupOption;
+import org.apache.hadoop.dfs.FSConstants.NodeType;
+import org.apache.hadoop.dfs.FSImage.NameNodeFile;
+import org.apache.hadoop.util.Daemon;
+import org.apache.hadoop.fs.FileUtil.HardLink;
 
 /** 
  * Data storage information file.
  * <p>
- * During startup the datanode reads its data storage file.
- * The data storage file is stored in all the dfs.data.dir directories.
- * It contains version and storageID.
- * Datanode holds a lock on all the dataStorage files while it runs so that other 
- * datanodes were not able to start working with the same data storage.
- * The locks are released when the datanode stops (normally or abnormally).
- * 
+ * @see Storage
  * @author Konstantin Shvachko
  */
-class DataStorage {
-  public static final String STORAGE_INFO_FILE_NAME = "storage";
-  public static final Log LOG = LogFactory.getLog("org.apache.hadoop.dfs.DataStorage");
-
-  // persistent fields
-  private int version = 0;  /// stored version
-  private String storageID; /// unique per cluster storageID
+class DataStorage extends Storage {
+  // Constants
+  final static String BLOCK_SUBDIR_PREFIX = "subdir";
+  final static String BLOCK_FILE_PREFIX = "blk_";
   
-  // non persistent fields
-  private ArrayList storageFiles = new ArrayList();
-  private ArrayList storageLocks = new ArrayList();
+  private String storageID;
+
+  DataStorage() {
+    super( NodeType.DATA_NODE );
+    storageID = "";
+  }
   
-  // cache away the names of all passed in dirs
-  private File[] origDirs = null;
+  DataStorage( int nsID, long cT, String strgID ) {
+    super( NodeType.DATA_NODE, nsID, cT );
+    this.storageID = strgID;
+  }
   
-  // cache away the names of locked dirs
-  private File[] dirs = null;
+  DataStorage( StorageInfo storageInfo, String strgID ) {
+    super( NodeType.DATA_NODE, storageInfo );
+    this.storageID = strgID;
+  }
+
+  String getStorageID() {
+    return storageID;
+  }
   
-  private int numLocked = 0;
+  void setStorageID( String newStorageID ) {
+    this.storageID = newStorageID;
+  }
   
   /**
-   * Create DataStorage and verify its version.
+   * Analyze storage directories.
+   * Recover from previous transitions if required. 
+   * Perform fs state transition if necessary depending on the namespace info.
+   * Read storage info. 
    * 
+   * @param nsInfo namespace information
    * @param dataDirs array of data storage directories
+   * @param startOpt startup option
    * @throws IOException
    */
-  public DataStorage( File[] dataDirs ) throws IOException {
-    this( DataNode.DFS_CURRENT_VERSION, dataDirs );
+  void recoverTransitionRead( NamespaceInfo nsInfo,
+                              Collection<File> dataDirs,
+                              StartupOption startOpt
+                            ) throws IOException {
+    assert FSConstants.LAYOUT_VERSION == nsInfo.getLayoutVersion() :
+      "Data-node and name-node layout versions must be the same.";
     
-    if( version < FSConstants.DFS_CURRENT_VERSION ) // future version
-      throw new IncorrectVersionException( version, "data storage" );
-  }
-  
-  /**
-   * Create DataStorage.
-   * 
-   * Read data storage files if they exist or create them if not.
-   * Lock the files.
-   * 
-   * @param curVersion can be used to read file saved with a previous version.
-   * @param dataDirs Array of data storage directories
-   * @throws IOException
-   */
-  public DataStorage( int curVersion, File[] dataDirs ) throws IOException {
-    this.version = curVersion;
-    this.origDirs = dataDirs;
-    for (int idx = 0; idx < dataDirs.length; idx++) {
-      storageFiles.add(idx, new RandomAccessFile( 
-                          new File(dataDirs[idx], STORAGE_INFO_FILE_NAME ), 
-                          "rws" ));
-      lock(idx);
-      boolean needToSave;
+    // 1. For each data directory calculate its state and 
+    // check whether all is consistent before transitioning.
+    // Format and recover.
+    this.storageID = "";
+    this.storageDirs = new ArrayList<StorageDirectory>( dataDirs.size() );
+    ArrayList<StorageState> dataDirStates = new ArrayList<StorageState>( dataDirs.size() );
+    for( Iterator<File> it = dataDirs.iterator(); it.hasNext(); ) {
+      File dataDir = it.next();
+      StorageDirectory sd = new StorageDirectory( dataDir );
+      StorageState curState;
       try {
-        needToSave = read(idx);
-      } catch( java.io.EOFException e ) {
-        storageID = "";
-        needToSave = true;
-      }
-    
-      if( needToSave ) { write(idx); }
-      
-      RandomAccessFile file = (RandomAccessFile) storageFiles.get(idx);
-      if (file != null) { numLocked++; }
-    }
-    if (numLocked > 0) {
-      this.dirs = new File[numLocked];
-      int curidx = 0;
-      for (int idx = 0; idx < dataDirs.length; idx++) {
-        if (storageFiles.get(idx) != null) {
-          dirs[curidx] = dataDirs[idx];
-          curidx++;
+        curState = sd.analyzeStorage( startOpt );
+        // sd is locked but not opened
+        switch( curState ) {
+        case NORMAL:
+          break;
+        case NON_EXISTENT:
+          // ignore this storage
+          LOG.info( "Storage directory " + dataDir + " does not exist." );
+          it.remove();
+          continue;
+        case CONVERT:
+          convertLayout( sd, nsInfo );
+          break;
+        case NOT_FORMATTED: // format
+          LOG.info( "Storage directory " + dataDir + " is not formatted." );
+          LOG.info( "Formatting ..." );
+          format( sd, nsInfo );
+          break;
+        default:  // recovery part is common
+          sd.doRecover( curState );
         }
+      } catch (IOException ioe) {
+        sd.unlock();
+        throw ioe;
       }
+      // add to the storage list
+      addStorageDir( sd );
+      dataDirStates.add( curState );
     }
-  }
-  
-  public int getVersion() {
-    return version;
-  }
 
-  public String getStorageID() {
-    return storageID;
+    if( dataDirs.size() == 0 )  // none of the data dirs exist
+      throw new IOException( 
+          "All specified directories are not accessible or do not exist." );
+
+    // 2. Do transitions
+    // Each storage directory is treated individually.
+    // During sturtup some of them can upgrade or rollback 
+    // while others could be uptodate for the regular startup.
+    for( int idx = 0; idx < getNumStorageDirs(); idx++ ) {
+      doTransition( getStorageDir( idx ), nsInfo, startOpt );
+      assert this.getLayoutVersion() == nsInfo.getLayoutVersion() :
+        "Data-node and name-node layout versions must be the same.";
+      assert this.getCTime() == nsInfo.getCTime() :
+        "Data-node and name-node CTimes must be the same.";
+    }
+    
+    // 3. Update all storages. Some of them might have just been formatted.
+    this.writeAll();
   }
-  
-  public int getNumLocked() {
-    return numLocked;
+
+  void format( StorageDirectory sd, NamespaceInfo nsInfo ) throws IOException {
+    sd.clearDirectory(); // create directory
+    this.layoutVersion = FSConstants.LAYOUT_VERSION;
+    this.namespaceID = nsInfo.getNamespaceID();
+    this.cTime = 0;
+    // store storageID as it currently is
+    sd.write();
   }
-  
-  public File[] getLockedDirs() {
-    return dirs;
+
+  protected void setFields( Properties props, 
+                            StorageDirectory sd 
+                          ) throws IOException {
+    super.setFields( props, sd );
+    props.setProperty( "storageID", storageID );
   }
-  
-  public void setStorageID( String newStorageID ) {
-    this.storageID = newStorageID;
+
+  protected void getFields( Properties props, 
+                            StorageDirectory sd 
+                          ) throws IOException {
+    super.getFields( props, sd );
+    String ssid = props.getProperty( "storageID" );
+    if( ssid == null ||
+        ! ("".equals( storageID ) || "".equals( ssid ) ||
+            storageID.equals( ssid )))
+      throw new InconsistentFSStateException( sd.root,
+                  "has incompatible storage Id." );
+    if( "".equals( storageID ) ) // update id only if it was empty
+      storageID = ssid;
   }
-  
-  public void setVersion( int newVersion ) {
-    this.version = newVersion;
+
+  boolean isConversionNeeded( StorageDirectory sd ) throws IOException {
+    File oldF = new File( sd.root, "storage" );
+    if( ! oldF.exists() )
+      return false;
+    // check consistency of the old storage
+    File oldDataDir = new File( sd.root, "data" );
+    if( ! oldDataDir.exists() ) 
+      throw new InconsistentFSStateException( sd.root,
+          "Old layout block directory " + oldDataDir + " is missing" ); 
+    if( ! oldDataDir.isDirectory() )
+      throw new InconsistentFSStateException( sd.root,
+          oldDataDir + " is not a directory." );
+    if( ! oldDataDir.canWrite() )
+      throw new InconsistentFSStateException( sd.root,
+          oldDataDir + " is not writable." );
+    return true;
   }
   
   /**
-   * Lock datastorage file.
+   * Automatic conversion from the old layout version to the new one.
    * 
+   * @param sd storage directory
+   * @param nsInfo namespace information
    * @throws IOException
    */
-  private void lock(int idx) throws IOException {
-    RandomAccessFile file = (RandomAccessFile) storageFiles.get(idx);
-    FileLock lock = file.getChannel().tryLock();
-    if (lock == null) {
-      // log a warning
-      LOG.warn("Cannot lock storage file in directory "+origDirs[idx].getName());
-      // remove the file from fileList, and close it
-      storageFiles.add(idx, null);
-      file.close();
-    }
-    storageLocks.add(idx, lock);
-  }
+  private void convertLayout( StorageDirectory sd,
+                              NamespaceInfo nsInfo 
+                            ) throws IOException {
+    assert FSConstants.LAYOUT_VERSION < LAST_PRE_UPGRADE_LAYOUT_VERSION :
+      "Bad current layout version: FSConstants.LAYOUT_VERSION should decrease";
+    File oldF = new File( sd.root, "storage" );
+    File oldDataDir = new File( sd.root, "data" );
+    assert oldF.exists() : "Old datanode layout \"storage\" file is missing";
+    assert oldDataDir.exists() : "Old layout block directory \"data\" is missing";
+    LOG.info( "Old layout version file " + oldF
+            + " is found. New layout version is "
+            + FSConstants.LAYOUT_VERSION );
+    LOG.info( "Converting ..." );
+    
+    // Lock and Read old storage file
+    RandomAccessFile oldFile = new RandomAccessFile( oldF, "rws" );
+    if (oldFile == null)
+      throw new IOException( "Cannot read file: " + oldF );
+    FileLock oldLock = oldFile.getChannel().tryLock();
+    if (oldLock == null)
+      throw new IOException( "Cannot lock file: " + oldF );
+    try {
+      oldFile.seek(0);
+      int odlVersion = oldFile.readInt();
+      if( odlVersion < LAST_PRE_UPGRADE_LAYOUT_VERSION )
+        throw new IncorrectVersionException( odlVersion, "file " + oldF,
+                                              LAST_PRE_UPGRADE_LAYOUT_VERSION );
+      String odlStorageID = org.apache.hadoop.io.UTF8.readString( oldFile );
   
-  /**
-   * Unlock datastorage file.
-   * @param idx File index
-   * 
-   * @throws IOException
-   */
-  private void unlock(int idx) throws IOException {
-    FileLock lock = (FileLock) storageLocks.get(idx);
-    if (lock != null) { lock.release(); }
-  }
+      // check new storage
+      File newDataDir = sd.getCurrentDir();
+      File versionF = sd.getVersionFile();
+      if( versionF.exists() )
+        throw new IOException( "Version file already exists: " + versionF );
+      if( newDataDir.exists() ) // somebody created current dir manually
+        deleteDir( newDataDir );
+      // Write new layout
+      rename( oldDataDir, newDataDir );
   
-  /**
-   * Close a datastorage file.
-   * @param idx file index
-   * @throws IOException
-   */
-  private void close(int idx) throws IOException {
-    FileLock lock = (FileLock) storageLocks.get(idx);
-    if (lock == null) { return; }
-    lock.release();
-    RandomAccessFile file = (RandomAccessFile) storageFiles.get(idx);
-    file.close();
+      this.layoutVersion = FSConstants.LAYOUT_VERSION;
+      this.namespaceID = nsInfo.getNamespaceID();
+      this.cTime = 0;
+      this.storageID = odlStorageID;
+      sd.write();
+      // close and unlock old file
+    } finally {
+      oldLock.release();
+      oldFile.close();
+    }
+    // move old storage file into current dir
+    rename( oldF, new File( sd.getCurrentDir(), "storage" ));
+    LOG.info( "Conversion of " + oldF + " is complete." );
   }
-  
+
   /**
-   * Close all datastorage files.
+   * Analize which and whether a transition of the fs state is required
+   * and perform it if necessary.
+   * 
+   * Rollback if previousLV >= LAYOUT_VERSION && prevCTime <= namenode.cTime
+   * Upgrade if this.LV > LAYOUT_VERSION || this.cTime < namenode.cTime
+   * Regular startup if this.LV = LAYOUT_VERSION && this.cTime = namenode.cTime
+   * 
+   * @param sd  storage directory
+   * @param nsInfo  namespace info
+   * @param startOpt  startup option
    * @throws IOException
    */
-  public void closeAll() throws IOException {
-    for (int idx = 0; idx < dirs.length; idx++) {
-      close(idx);
+  private void doTransition(  StorageDirectory sd, 
+                              NamespaceInfo nsInfo, 
+                              StartupOption startOpt
+                            ) throws IOException {
+    if( startOpt == StartupOption.ROLLBACK )
+      doRollback( sd, nsInfo ); // rollback if applicable
+    sd.read();
+    assert this.layoutVersion >= FSConstants.LAYOUT_VERSION :
+      "Future version is not allowed";
+    if( getNamespaceID() != nsInfo.getNamespaceID() )
+      throw new IOException( 
+          "Incompatible namespaceIDs in " + sd.root.getCanonicalPath()
+          + ": namenode namespaceID = " + nsInfo.getNamespaceID() 
+          + "; datanode namespaceID = " + getNamespaceID() );
+    if( this.layoutVersion == FSConstants.LAYOUT_VERSION 
+        && this.cTime == nsInfo.getCTime() )
+      return; // regular startup
+    if( this.layoutVersion > FSConstants.LAYOUT_VERSION
+        || this.cTime < nsInfo.getCTime() ) {
+      doUpgrade( sd, nsInfo );  // upgrade
+      return;
     }
+    // layoutVersion == LAYOUT_VERSION && this.cTime > nsInfo.cTime
+    // must shutdown
+    throw new IOException("Datanode state: LV = " + this.getLayoutVersion() 
+                          + " CTime = " + this.getCTime() 
+                          + " is newer than the namespace state: LV = "
+                          + nsInfo.getLayoutVersion() 
+                          + " CTime = " + nsInfo.getCTime() );
   }
-  
+
   /**
-   * Read data storage file.
-   * @param idx File index
-   * @return whether the data storage file need to be updated.
+   * Move current storage into a backup directory,
+   * and hardlink all its blocks into the new current directory.
+   * 
+   * @param sd  storage directory
    * @throws IOException
    */
-  private boolean read(int idx) throws IOException {
-    RandomAccessFile file = (RandomAccessFile) storageFiles.get(idx);
-    if (file == null) { return false; }
-    file.seek(0);
-    this.version = file.readInt();
-    this.storageID = UTF8.readString( file );
-    return false;
+  void doUpgrade( StorageDirectory sd,
+                  NamespaceInfo nsInfo
+                ) throws IOException {
+    LOG.info( "Upgrading storage directory " + sd.root 
+            + ".\n   old LV = " + this.getLayoutVersion()
+            + "; old CTime = " + this.getCTime()
+            + ".\n   new LV = " + nsInfo.getLayoutVersion()
+            + "; new CTime = " + nsInfo.getCTime() );
+    File curDir = sd.getCurrentDir();
+    File prevDir = sd.getPreviousDir();
+    assert curDir.exists() : "Current directory must exist.";
+    // delete previous dir before upgrading
+    if( prevDir.exists() )
+      deleteDir( prevDir );
+    File tmpDir = sd.getPreviousTmp();
+    assert ! tmpDir.exists() : "previous.tmp directory must not exist.";
+    // rename current to tmp
+    rename( curDir, tmpDir );
+    // hardlink blocks
+    linkBlocks( tmpDir, curDir );
+    // write version file
+    this.layoutVersion = FSConstants.LAYOUT_VERSION;
+    assert this.namespaceID == nsInfo.getNamespaceID() :
+      "Data-node and name-node layout versions must be the same.";
+    this.cTime = nsInfo.getCTime();
+    sd.write();
+    // rename tmp to previous
+    rename( tmpDir, prevDir );
+    LOG.info( "Upgrade of " + sd.root + " is complete." );
   }
 
-  /**
-   * Write data storage file.
-   * @param idx File index
-   * @throws IOException
-   */
-  private void write(int idx) throws IOException {
-    RandomAccessFile file = (RandomAccessFile) storageFiles.get(idx);
-    if (file == null) { return; }
-    file.seek(0);
-    file.writeInt( this.version );
-    UTF8.writeString( file, this.storageID );
+  void doRollback(  StorageDirectory sd,
+                    NamespaceInfo nsInfo
+                  ) throws IOException {
+    File prevDir = sd.getPreviousDir();
+    // regular startup if previous dir does not exist
+    if( ! prevDir.exists() )
+      return;
+    DataStorage prevInfo = new DataStorage();
+    StorageDirectory prevSD = prevInfo.new StorageDirectory( sd.root );
+    prevSD.read( prevSD.getPreviousVersionFile() );
+
+    // We allow rollback to a state, which is either consistent with
+    // the namespace state or can be further upgraded to it.
+    if( ! ( prevInfo.getLayoutVersion() >= FSConstants.LAYOUT_VERSION
+        && prevInfo.getCTime() <= nsInfo.getCTime() ))  // cannot rollback
+      throw new InconsistentFSStateException( prevSD.root,
+          "Cannot rollback to a newer state.\nDatanode previous state: LV = " 
+          + prevInfo.getLayoutVersion() + " CTime = " + prevInfo.getCTime() 
+          + " is newer than the namespace state: LV = "
+          + nsInfo.getLayoutVersion() + " CTime = " + nsInfo.getCTime() );
+    LOG.info( "Rolling back storage directory " + sd.root 
+        + ".\n   target LV = " + nsInfo.getLayoutVersion()
+        + "; target CTime = " + nsInfo.getCTime() );
+    File tmpDir = sd.getRemovedTmp();
+    assert ! tmpDir.exists() : "removed.tmp directory must not exist.";
+    // rename current to tmp
+    File curDir = sd.getCurrentDir();
+    assert curDir.exists() : "Current directory must exist.";
+    rename( curDir, tmpDir );
+    // rename previous to current
+    rename( prevDir, curDir );
+    // delete tmp dir
+    deleteDir( tmpDir );
+    LOG.info( "Rollback of " + sd.root + " is complete." );
+  }
+
+  void doFinalize( StorageDirectory sd ) throws IOException {
+    File prevDir = sd.getPreviousDir();
+    if( ! prevDir.exists() )
+      return; // already discarded
+    final String dataDirPath = sd.root.getCanonicalPath();
+    LOG.info( "Finalizing upgrade for storage directory " 
+            + dataDirPath 
+            + ".\n   cur LV = " + this.getLayoutVersion()
+            + "; cur CTime = " + this.getCTime() );
+    assert sd.getCurrentDir().exists() : "Current directory must exist.";
+    final File tmpDir = sd.getFinalizedTmp();
+    // rename previous to tmp
+    rename( prevDir, tmpDir );
+
+    // delete tmp dir in a separate thread
+    new Daemon( new Runnable() {
+      public void run() {
+        try {
+          deleteDir( tmpDir );
+        } catch( IOException ex ) {
+          LOG.error( "Finalize upgrade for " + dataDirPath + " failed.", ex );
+        }
+        LOG.info( "Finalize upgrade for " + dataDirPath + " is complete." );
+      }
+      public String toString() { return "Finalize " + dataDirPath; }
+    }).start();
   }
   
-  /**
-   * Write all data storage files.
-   * @throws IOException
-   */
-  public void writeAll() throws IOException {
-    for (int idx = 0; idx < dirs.length; idx++) {
-      write(idx);
+  void finalizeUpgrade() throws IOException {
+    for (Iterator<StorageDirectory> it = storageDirs.iterator(); it.hasNext();) {
+      doFinalize( it.next() );
     }
   }
   
+  static void linkBlocks( File from, File to ) throws IOException {
+    if( ! from.isDirectory() ) {
+      HardLink.createHardLink( from, to );
+      return;
+    }
+    // from is a directory
+    if( ! to.mkdir() )
+      throw new IOException("Cannot create directory " + to );
+    String[] blockNames = from.list( new java.io.FilenameFilter() {
+      public boolean accept(File dir, String name) {
+        return name.startsWith( BLOCK_SUBDIR_PREFIX ) 
+            || name.startsWith( BLOCK_FILE_PREFIX );
+      }
+    });
+    
+    for( int i = 0; i < blockNames.length; i++ )
+      linkBlocks( new File(from, blockNames[i]), new File(to, blockNames[i]) );
+  }
 }

+ 19 - 10
src/java/org/apache/hadoop/dfs/DatanodeProtocol.java

@@ -31,9 +31,15 @@ import org.apache.hadoop.ipc.VersionedProtocol;
  * @author Michael Cafarella
  **********************************************************************/
 interface DatanodeProtocol extends VersionedProtocol {
-  public static final long versionID = 5L;  // register takes a new parameter
+  /*
+   * 6: versionRequest() added;
+   * sendHeartbeat() and blockReport() return DatanodeCommand;
+   * DatanodeRegistration contains StorageInfo
+   */
+  public static final long versionID = 6L;
   
   // error code
+  final static int NOTIFY = 0;
   final static int DISK_ERROR = 1;
   final static int INVALID_BLOCK = 2;
 
@@ -45,7 +51,8 @@ interface DatanodeProtocol extends VersionedProtocol {
                               DNA_TRANSFER,   // transfer blocks to another datanode
                               DNA_INVALIDATE, // invalidate blocks
                               DNA_SHUTDOWN,   // shutdown node
-                              DNA_REGISTER; }   // re-register
+                              DNA_REGISTER,   // re-register
+                              DNA_FINALIZE; } // finalize previous upgrade
 
   /** 
    * Register Datanode.
@@ -63,14 +70,14 @@ interface DatanodeProtocol extends VersionedProtocol {
     /**
      * sendHeartbeat() tells the NameNode that the DataNode is still
      * alive and well.  Includes some status info, too. 
-     * It also gives the NameNode a chance to return a "BlockCommand" object.
-     * A BlockCommand tells the DataNode to invalidate local block(s), 
+     * It also gives the NameNode a chance to return a "DatanodeCommand" object.
+     * A DatanodeCommand tells the DataNode to invalidate local block(s), 
      * or to copy them to other DataNodes, etc.
      */
-    public BlockCommand sendHeartbeat(DatanodeRegistration registration,
-                                      long capacity, long remaining,
-                                      int xmitsInProgress,
-                                      int xceiverCount) throws IOException;
+    public DatanodeCommand sendHeartbeat( DatanodeRegistration registration,
+                                          long capacity, long remaining,
+                                          int xmitsInProgress,
+                                          int xceiverCount) throws IOException;
 
     /**
      * blockReport() tells the NameNode about all the locally-stored blocks.
@@ -79,8 +86,8 @@ interface DatanodeProtocol extends VersionedProtocol {
      * the locally-stored blocks.  It's invoked upon startup and then
      * infrequently afterwards.
      */
-    public Block[] blockReport( DatanodeRegistration registration,
-                                Block blocks[]) throws IOException;
+    public DatanodeCommand blockReport( DatanodeRegistration registration,
+                                        Block blocks[]) throws IOException;
     
     /**
      * blockReceived() allows the DataNode to tell the NameNode about
@@ -98,4 +105,6 @@ interface DatanodeProtocol extends VersionedProtocol {
     public void errorReport(DatanodeRegistration registration,
                             int errorCode, 
                             String msg) throws IOException;
+    
+    public NamespaceInfo versionRequest() throws IOException;
 }

+ 24 - 12
src/java/org/apache/hadoop/dfs/DatanodeRegistration.java

@@ -25,40 +25,50 @@ class DatanodeRegistration extends DatanodeID implements Writable {
        });
   }
 
-  int version;            /// current Datanode version
-  String registrationID;  /// a unique per namenode id; indicates   
-                          /// the namenode the datanode is registered with
+  StorageInfo storageInfo;
 
   /**
    * Default constructor.
    */
   public DatanodeRegistration() {
-    this( 0, null, null, -1, null );
+    super( null, null, -1 );
+    this.storageInfo = new StorageInfo();
   }
   
   /**
    * Create DatanodeRegistration
    */
+  public DatanodeRegistration(String nodeName, 
+                              int infoPort,
+                              DataStorage storage ) {
+    super( nodeName, storage.getStorageID(), infoPort );
+    this.storageInfo = new StorageInfo( storage );
+  }
+
+  /**
+   * Create DatanodeRegistration
+   * @deprecated 
+   * use {@link #DatanodeRegistration(String, int, DataStorage)} instead
+   */
   public DatanodeRegistration(int version, 
                               String nodeName, 
                               String storageID,
                               int infoPort,
                               String registrationID ) {
     super( nodeName, storageID, infoPort );
-    this.version = version;
-    this.registrationID = registrationID;
+    this.storageInfo = new StorageInfo();
   }
 
   /**
    */
   public int getVersion() {
-    return version;
+    return storageInfo.getLayoutVersion();
   }
   
   /**
    */
   public String getRegistrationID() {
-    return registrationID;
+    return Storage.getRegistrationID( storageInfo );
   }
 
   /////////////////////////////////////////////////
@@ -67,16 +77,18 @@ class DatanodeRegistration extends DatanodeID implements Writable {
   /**
    */
   public void write(DataOutput out) throws IOException {
-    out.writeInt(this.version);
     super.write( out );
-    UTF8.writeString(out, registrationID);
+    out.writeInt( storageInfo.getLayoutVersion() );
+    out.writeInt( storageInfo.getNamespaceID() );
+    out.writeLong( storageInfo.getCTime() );
   }
 
   /**
    */
   public void readFields(DataInput in) throws IOException {
-    this.version = in.readInt();
     super.readFields(in);
-    this.registrationID = UTF8.readString(in);   
+    storageInfo.layoutVersion = in.readInt();
+    storageInfo.namespaceID = in.readInt();
+    storageInfo.cTime = in.readLong();
   }
 }

+ 16 - 0
src/java/org/apache/hadoop/dfs/DistributedFileSystem.java

@@ -303,6 +303,14 @@ public class DistributedFileSystem extends ChecksumFileSystem {
       dfs.refreshNodes();
     }
 
+    /**
+     * Finalize previously upgraded files system state.
+     * @throws IOException
+     */
+    public void finalizeUpgrade() throws IOException {
+      dfs.finalizeUpgrade();
+    }
+
     /**
      * We need to find the blocks that didn't match.  Likely only one 
      * is corrupt but we will report both to the namenode.  In the future,
@@ -400,6 +408,14 @@ public class DistributedFileSystem extends ChecksumFileSystem {
     public void refreshNodes() throws IOException {
       ((RawDistributedFileSystem)fs).refreshNodes();
     }
+
+    /**
+     * Finalize previously upgraded files system state.
+     */
+    public void finalizeUpgrade() throws IOException {
+      ((RawDistributedFileSystem)fs).finalizeUpgrade();
+    }
+
     /**
      * We need to find the blocks that didn't match.  Likely only one 
      * is corrupt but we will report both to the namenode.  In the future,

+ 16 - 4
src/java/org/apache/hadoop/dfs/FSConstants.java

@@ -122,11 +122,23 @@ public interface FSConstants {
     // SafeMode actions
     public enum SafeModeAction{ SAFEMODE_LEAVE, SAFEMODE_ENTER, SAFEMODE_GET; }
 
+    // Startup options
+    public enum StartupOption{ FORMAT, REGULAR, UPGRADE, ROLLBACK; }
+
+    /**
+     * Type of the node
+     */
+    static public enum NodeType {
+      NAME_NODE,
+      DATA_NODE;
+    }
+
     // Version is reflected in the dfs image and edit log files.
     // Version is reflected in the data storage file.
     // Versions are negative.
-    // Decrement DFS_CURRENT_VERSION to define a new version.
-    public static final int DFS_CURRENT_VERSION = -3;
-    // Current version: New operations OP_DATANODE_REMOVE and OP_DATANODE_ADD
-    // are introduced
+    // Decrement LAYOUT_VERSION to define a new version.
+    public static final int LAYOUT_VERSION = -4;
+    // Current version: 
+    // Top level directory is reorganized to allow file system state 
+    // transitions: upgrade, rollback, and finalize.
 }

+ 9 - 9
src/java/org/apache/hadoop/dfs/FSDataset.java

@@ -118,7 +118,7 @@ class FSDataset implements FSConstants {
             if ( children == null || children.length == 0 ) {
               children = new FSDir[maxBlocksPerDir];
               for (int idx = 0; idx < maxBlocksPerDir; idx++) {
-                children[idx] = new FSDir( new File(dir, "subdir"+idx) );
+                children[idx] = new FSDir( new File(dir, DataStorage.BLOCK_SUBDIR_PREFIX+idx) );
               }
             }
             
@@ -256,19 +256,19 @@ class FSDataset implements FSConstants {
     class FSVolume {
       static final double USABLE_DISK_PCT_DEFAULT = 0.98f; 
 
-      private File dir;
+      private File dir; // TODO this field is redundant equals this.dataDir.dir.getParent()
       private FSDir dataDir;
       private File tmpDir;
       private DF usage;
       private long reserved;
       private double usableDiskPct = USABLE_DISK_PCT_DEFAULT;
     
-      FSVolume(File dir, Configuration conf) throws IOException {
+      FSVolume( File currentDir, Configuration conf) throws IOException {
         this.reserved = conf.getLong("dfs.datanode.du.reserved", 0);
         this.usableDiskPct = conf.getFloat("dfs.datanode.du.pct",
             (float) USABLE_DISK_PCT_DEFAULT);
-        this.dir = dir;
-        this.dataDir = new FSDir(new File(dir, "data"));
+        this.dir = currentDir.getParentFile();
+        this.dataDir = new FSDir( currentDir );
         this.tmpDir = new File(dir, "tmp");
         if (tmpDir.exists()) {
           FileUtil.fullyDelete(tmpDir);
@@ -431,11 +431,11 @@ class FSDataset implements FSConstants {
     /**
      * An FSDataset has a directory where it loads its data files.
      */
-    public FSDataset(File[] dirs, Configuration conf) throws IOException {
+    public FSDataset( DataStorage storage, Configuration conf) throws IOException {
     	this.maxBlocksPerDir = conf.getInt("dfs.datanode.numblocks", 64);
-        FSVolume[] volArray = new FSVolume[dirs.length];
-        for (int idx = 0; idx < dirs.length; idx++) {
-          volArray[idx] = new FSVolume(dirs[idx], conf);
+        FSVolume[] volArray = new FSVolume[storage.getNumStorageDirs()];
+        for (int idx = 0; idx < storage.getNumStorageDirs(); idx++) {
+          volArray[idx] = new FSVolume(storage.getStorageDir(idx).getCurrentDir(), conf);
         }
         volumes = new FSVolumeSet(volArray);
         volumeMap = new HashMap<Block,FSVolume>();

+ 17 - 8
src/java/org/apache/hadoop/dfs/FSDirectory.java

@@ -22,7 +22,6 @@ import org.apache.hadoop.io.*;
 import java.io.*;
 import java.util.*;
 
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.metrics.MetricsRecord;
 import org.apache.hadoop.metrics.MetricsUtil;
@@ -315,13 +314,12 @@ class FSDirectory implements FSConstants {
     TreeMap activeLocks = new TreeMap();
     FSImage fsImage;  
     boolean ready = false;
-    int namespaceID = 0;    // TODO: move to FSImage class, it belongs there
     // Metrics record
     private MetricsRecord directoryMetrics = null;
     
     /** Access an existing dfs name directory. */
-    public FSDirectory(File[] dirs) throws IOException {
-      this.fsImage = new FSImage( dirs );
+    public FSDirectory() throws IOException {
+      this.fsImage = new FSImage();
       initialize();
     }
 
@@ -335,12 +333,23 @@ class FSDirectory implements FSConstants {
       directoryMetrics = MetricsUtil.createRecord(metricsContext, "FSDirectory");
     }
 
-    void loadFSImage( Configuration conf ) throws IOException {
-      fsImage.loadFSImage( conf );
+    void loadFSImage( Collection<File> dataDirs,
+                      StartupOption startOpt ) throws IOException {
+      // format before starting up if requested
+      if( startOpt == StartupOption.FORMAT ) {
+        fsImage.setStorageDirectories( dataDirs );
+        fsImage.format();
+        startOpt = StartupOption.REGULAR;
+      }
+      try {
+        fsImage.recoverTransitionRead( dataDirs, startOpt );
+      } catch( IOException e ) {
+        fsImage.close();
+        throw e;
+      }
       synchronized (this) {
         this.ready = true;
         this.notifyAll();
-        fsImage.getEditLog().create();
       }
     }
 
@@ -353,7 +362,7 @@ class FSDirectory implements FSConstants {
      * Shutdown the filestore
      */
     public void close() throws IOException {
-        fsImage.getEditLog().close();
+        fsImage.close();
     }
 
     /**

+ 108 - 206
src/java/org/apache/hadoop/dfs/FSEditLog.java

@@ -26,10 +26,8 @@ import java.io.FileDescriptor;
 import java.io.FileInputStream;
 import java.io.FileOutputStream;
 import java.io.IOException;
-import java.util.Iterator;
-import java.util.Vector;
+import java.util.ArrayList;
 
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.ArrayWritable;
 import org.apache.hadoop.io.UTF8;
 import org.apache.hadoop.io.Writable;
@@ -48,60 +46,86 @@ class FSEditLog {
   private static final byte OP_DATANODE_ADD = 5;
   private static final byte OP_DATANODE_REMOVE = 6;
 
-  private static final String FS_EDIT = "edits";
-  private static final String FS_EDIT_NEW = "edits.new";
+  private ArrayList<EditLogOutputStream> editStreams = null;
+  private FSImage fsimage = null;
   
-  private File[] editFiles = null;
-  private File[] editFilesNew = null;
+  static class EditLogOutputStream extends DataOutputStream {
+    private FileDescriptor fd;
 
-  DataOutputStream[] editStreams = null;
-  FileDescriptor[] editDescriptors = null;
-  private FSImage fsimage = null;
+    EditLogOutputStream( File name ) throws IOException {
+      super( new FileOutputStream( name, true )); // open for append
+      this.fd = ((FileOutputStream)out).getFD();
+    }
 
-  FSEditLog(File[] fsDirs, FSImage image)  throws IOException {
-    fsimage = image;
-    editFiles = new File[fsDirs.length];
-    editFilesNew = new File[fsDirs.length];
-    for (int idx = 0; idx < fsDirs.length; idx++) {
-       editFiles[idx] = new File(fsDirs[idx], FS_EDIT);
-       editFilesNew[idx] = new File(fsDirs[idx], FS_EDIT_NEW);
-     }
-   }
+    void flushAndSync() throws IOException {
+      this.flush();
+      this.fd.sync();
+    }
+
+    void create() throws IOException {
+      writeInt( FSConstants.LAYOUT_VERSION );
+      flushAndSync();
+    }
+  }
 
-  FSEditLog(File imageDir, FSImage image, String edits)  throws IOException {
+  FSEditLog( FSImage image ) {
     fsimage = image;
-    editFiles = new File[1];
-    editFiles[0] = new File(imageDir, edits);
-   }
+  }
+
+  private File getEditFile( int idx ) {
+    return fsimage.getEditFile( idx );
+  }
+
+  private File getEditNewFile( int idx ) {
+    return fsimage.getEditNewFile( idx );
+  }
+  
+  private int getNumStorageDirs() {
+    return fsimage.getNumStorageDirs();
+  }
+  
+  int getNumEditStreams() {
+    return editStreams == null ? 0 : editStreams.size();
+  }
 
   /**
+   * Create empty edit log files.
    * Initialize the output stream for logging.
    * 
    * @throws IOException
    */
-  void create() throws IOException {
-    editStreams = new DataOutputStream[editFiles.length];
-    editDescriptors = new FileDescriptor[editFiles.length];
-    for (int idx = 0; idx < editStreams.length; idx++) {
-      FileOutputStream stream = new FileOutputStream(editFiles[idx]);
-      editStreams[idx] = new DataOutputStream(stream);
-      editDescriptors[idx] = stream.getFD();
-      editStreams[idx].writeInt( FSConstants.DFS_CURRENT_VERSION );
+  void open() throws IOException {
+    int size = getNumStorageDirs();
+    if( editStreams == null )
+      editStreams = new ArrayList<EditLogOutputStream>( size );
+    for (int idx = 0; idx < size; idx++) {
+      File eFile = getEditFile( idx );
+      try {
+        EditLogOutputStream eStream = new EditLogOutputStream( eFile );
+        editStreams.add( eStream );
+      } catch (IOException e) {
+        FSNamesystem.LOG.warn( "Unable to open edit log file " + eFile );
+        processIOError(idx); 
+        idx--; 
+      }
     }
   }
 
+  void createEditLogFile( File name ) throws IOException {
+    EditLogOutputStream eStream = new EditLogOutputStream( name );
+    eStream.create();
+    eStream.flushAndSync();
+    eStream.close();
+  }
+
   /**
    * Create edits.new if non existant.
    */
   void createNewIfMissing() throws IOException {
-    for (int idx = 0; idx < editFilesNew.length; idx++) {
-      if (!editFilesNew[idx].exists()) {
-        FileOutputStream stream = new FileOutputStream(editFilesNew[idx]);
-        DataOutputStream editStr = new DataOutputStream(stream);
-        editStr.writeInt( FSConstants.DFS_CURRENT_VERSION );
-        editStr.flush();
-        editStr.close();
-      } 
+    for (int idx = 0; idx < getNumStorageDirs(); idx++) {
+      File newFile = getEditNewFile( idx );
+      if( ! newFile.exists() )
+        createEditLogFile( newFile );
     }
   }
   
@@ -112,16 +136,17 @@ class FSEditLog {
     if (editStreams == null) {
       return;
     }
-    for (int idx = 0; idx < editStreams.length; idx++) {
+    for (int idx = 0; idx < editStreams.size(); idx++) {
+      EditLogOutputStream eStream = editStreams.get( idx );
       try {
-        editStreams[idx].flush();
-        editDescriptors[idx].sync();
-        editStreams[idx].close();
+        eStream.flushAndSync();
+        eStream.close();
       } catch (IOException e) {
         processIOError(idx);
         idx--;
       }
     }
+    editStreams.clear();
   }
 
   /**
@@ -131,117 +156,37 @@ class FSEditLog {
    * server to exit
    */
    void processIOError(int index) throws IOException {
-     if (editStreams == null || editStreams.length == 1) {
+     if (editStreams == null || editStreams.size() == 1) {
        throw new IOException("Checkpoint directories inaccessible.");
      }
-     assert(index < editFiles.length);
-     assert(editFiles.length == editFilesNew.length);
-     assert(editFiles.length == editStreams.length);
-     int newsize = editStreams.length - 1;
-     int oldsize = editStreams.length;
-
-     //
-     // save existing values and allocate space for new ones
-     //
-     File[] editFiles1 = editFiles;
-     File[] editFilesNew1 = editFilesNew;
-     DataOutputStream[] editStreams1 = editStreams;
-     FileDescriptor[] editDescriptors1 = editDescriptors;
-     editFiles = new File[newsize];
-     editFilesNew = new File[newsize];
-     editStreams = new DataOutputStream[newsize];
-     editDescriptors = new FileDescriptor[newsize];
+     assert(index < getNumStorageDirs());
+     assert(getNumStorageDirs() == editStreams.size());
 
-     //
-     // copy values from old into new, skip the one with error.
-     //
-     for (int idx = 0; idx < index; idx++) {
-       editFiles[idx] = editFiles1[idx];
-       editFilesNew[idx] = editFilesNew1[idx];
-       editStreams[idx] = editStreams1[idx];
-       editDescriptors[idx] = editDescriptors1[idx];
-     }
-     for (int idx = index; idx < oldsize - 1; idx++) {
-       editFiles[idx] = editFiles1[idx+1];
-       editFilesNew[idx] = editFilesNew1[idx+1];
-       editStreams[idx] = editStreams1[idx+1];
-       editDescriptors[idx] = editDescriptors1[idx+1];
-     }
+     editStreams.remove( index );
      //
      // Invoke the ioerror routine of the fsimage
      //
      fsimage.processIOError(index);
    }
 
-  /**
-   * Delete specified editLog
-   */
-  void delete(int idx) throws IOException {
-    if (editStreams != null) {
-      try {
-        editStreams[idx].close();
-      } catch (IOException e) {
-        processIOError(idx);
-      }
-    }
-    if (!editFiles[idx].delete() || !editFilesNew[idx].delete()) {
-      if (editStreams != null) {
-        processIOError(idx);
-      }
-    }
-  }
-
-  /**
-   * check if ANY edits log exists
-   */
-  boolean exists() throws IOException {
-    for (int idx = 0; idx < editFiles.length; idx++) {
-      if (editFiles[idx].exists()) { 
-        return true;
-      }
-    }
-    return false;
-  }
-
   /**
    * check if ANY edits.new log exists
    */
   boolean existsNew() throws IOException {
-    for (int idx = 0; idx < editFilesNew.length; idx++) {
-      if (editFilesNew[idx].exists()) { 
+    for (int idx = 0; idx < getNumStorageDirs(); idx++) {
+      if (getEditNewFile( idx ).exists()) { 
         return true;
       }
     }
     return false;
   }
 
-  /**
-   * check if a particular edits.new log exists
-   */
-  boolean existsNew(int idx) throws IOException {
-    if (editFilesNew[idx].exists()) { 
-      return true;
-    }
-    return false;
-  }
-
-  
   /**
    * Load an edit log, and apply the changes to the in-memory structure
    * This is where we apply edits that we've been writing to disk all
    * along.
    */
-  int loadFSEdits(Configuration conf, int index) throws IOException {
-    int numEdits = 0;
-    numEdits = loadFSEdits(conf, editFiles[index]);
-    if (editFilesNew[index].exists()) { 
-      numEdits += loadFSEdits(conf, editFilesNew[index]);
-    }
-    return numEdits;
-  }
-
-  int loadFSEdits( Configuration conf, File edits)
-                                                 throws IOException {
+  int loadFSEdits( File edits ) throws IOException {
     FSNamesystem fsNamesys = FSNamesystem.getFSNamesystem();
     FSDirectory fsDir = fsNamesys.dir;
     int numEdits = 0;
@@ -267,15 +212,15 @@ class FSEditLog {
           logVersion = 0;
         else
           logVersion = in.readInt();
-        if( logVersion < FSConstants.DFS_CURRENT_VERSION ) // future version
+        if( logVersion < FSConstants.LAYOUT_VERSION ) // future version
           throw new IOException(
               "Unexpected version of the file system log file: "
               + logVersion
               + ". Current version = " 
-              + FSConstants.DFS_CURRENT_VERSION + "." );
+              + FSConstants.LAYOUT_VERSION + "." );
       }
       
-      short replication = (short)conf.getInt("dfs.replication", 3);
+      short replication = fsNamesys.getDefaultReplication();
       try {
         while (true) {
           byte opcode = -1;
@@ -304,7 +249,7 @@ class FSEditLog {
               name = (UTF8) writables[0];
               replication = Short.parseShort(
                   ((UTF8)writables[1]).toString());
-              replication = adjustReplication( replication, conf );
+              replication = adjustReplication( replication );
             }
             // get blocks
             aw = new ArrayWritable(Block.class);
@@ -321,9 +266,7 @@ class FSEditLog {
             UTF8 repl = new UTF8();
             src.readFields(in);
             repl.readFields(in);
-            replication = adjustReplication(
-                            fromLogReplication(repl),
-                            conf);
+            replication = adjustReplication( fromLogReplication(repl) );
             fsDir.unprotectedSetReplication(src.toString(), 
                 replication,
                 null);
@@ -383,17 +326,18 @@ class FSEditLog {
       }
     }
     
-    if( logVersion != FSConstants.DFS_CURRENT_VERSION ) // other version
+    if( logVersion != FSConstants.LAYOUT_VERSION ) // other version
       numEdits++; // save this image asap
     return numEdits;
   }
   
-  static short adjustReplication( short replication, Configuration conf) {
-    short minReplication = (short)conf.getInt("dfs.replication.min", 1);
+  static short adjustReplication( short replication) {
+    FSNamesystem fsNamesys = FSNamesystem.getFSNamesystem();
+    short minReplication = fsNamesys.getMinReplication();
     if( replication<minReplication ) {
       replication = minReplication;
     }
-    short maxReplication = (short)conf.getInt("dfs.replication.max", 512);
+    short maxReplication = fsNamesys.getMaxReplication();
     if( replication>maxReplication ) {
       replication = maxReplication;
     }
@@ -404,18 +348,19 @@ class FSEditLog {
    * Write an operation to the edit log
    */
   void logEdit(byte op, Writable w1, Writable w2) {
-    for (int idx = 0; idx < editStreams.length; idx++) {
-      synchronized (editStreams[idx]) {
+    assert this.getNumEditStreams() > 0 : "no editlog streams";
+    for (int idx = 0; idx < editStreams.size(); idx++) {
+      EditLogOutputStream eStream;
+      synchronized ( eStream = editStreams.get( idx ) ) {
         try {
-          editStreams[idx].write(op);
+          eStream.write(op);
           if (w1 != null) {
-            w1.write(editStreams[idx]);
+            w1.write( eStream );
           }
           if (w2 != null) {
-            w2.write(editStreams[idx]);
+            w2.write( eStream );
           }
-          editStreams[idx].flush();
-          editDescriptors[idx].sync();
+          eStream.flushAndSync();
         } catch (IOException ie) {
           try {
             processIOError(idx);         
@@ -500,12 +445,12 @@ class FSEditLog {
    * Return the size of the current EditLog
    */
   long getEditLogSize() throws IOException {
-    assert(editFiles.length == editStreams.length);
+    assert(getNumStorageDirs() == editStreams.size());
     long size = 0;
-    for (int idx = 0; idx < editFiles.length; idx++) {
-      synchronized (editStreams[idx]) {
-        assert(size == 0 || size == editFiles[idx].length());
-        size = editFiles[idx].length();
+    for (int idx = 0; idx < getNumStorageDirs(); idx++) {
+      synchronized (editStreams.get( idx )) {
+        assert(size == 0 || size == getEditFile( idx ).length());
+        size = getEditFile( idx ).length();
       }
     }
     return size;
@@ -527,12 +472,11 @@ class FSEditLog {
     //
     // Open edits.new
     //
-    for (int idx = 0; idx < editFiles.length; idx++ ) {
+    for (int idx = 0; idx < getNumStorageDirs(); idx++ ) {
       try {
-        FileOutputStream stream = new FileOutputStream(editFilesNew[idx]);
-        editStreams[idx] = new DataOutputStream(stream);
-        editDescriptors[idx] = stream.getFD();
-        editStreams[idx].writeInt( FSConstants.DFS_CURRENT_VERSION );
+        EditLogOutputStream eStream = new EditLogOutputStream( getEditNewFile( idx ));
+        eStream.create();
+        editStreams.add( eStream );
       } catch (IOException e) {
         processIOError(idx);
         idx--;
@@ -540,42 +484,11 @@ class FSEditLog {
     }
   }
 
-  /**
-   * Closes the current edit log and opens edits.new. 
-   * If edits.new already exists, then ignore it.
-   */
-  void rollEditLogIfNeeded() throws IOException {
-
-    //
-    // Open edits.new
-    //
-    for (int idx = 0; idx < editFiles.length; idx++ ) {
-      if (existsNew(idx)) {
-        continue;
-      }
-      try {
-        FileOutputStream stream = new FileOutputStream(editFilesNew[idx]);
-        editStreams[idx] = new DataOutputStream(stream);
-        editDescriptors[idx] = stream.getFD();
-        editStreams[idx].writeInt( FSConstants.DFS_CURRENT_VERSION );
-      } catch (IOException e) {
-        processIOError(idx);
-        idx--;
-      }
-    }
-  }
   /**
    * Removes the old edit log and renamed edits.new as edits.
    * Reopens the edits file.
    */
   void purgeEditLog() throws IOException {
-    purgeEditLog(true);
-  }
-
-  /**
-   * Removes the old edit log and renamed edits.new as edits.
-   */
-  void purgeEditLog(boolean reopenEdits) throws IOException {
     //
     // If edits.new does not exists, then return error.
     //
@@ -588,14 +501,14 @@ class FSEditLog {
     //
     // Delete edits and rename edits.new to edits.
     //
-    for (int idx = 0; idx < editFiles.length; idx++ ) {
-      if (!editFilesNew[idx].renameTo(editFiles[idx])) {
+    for (int idx = 0; idx < getNumStorageDirs(); idx++ ) {
+      if (!getEditNewFile( idx ).renameTo(getEditFile( idx ))) {
         //
         // renameTo() fails on Windows if the destination
         // file exists.
         //
-        editFiles[idx].delete();
-        if (!editFilesNew[idx].renameTo(editFiles[idx])) {
+        getEditFile( idx ).delete();
+        if (!getEditNewFile( idx ).renameTo(getEditFile( idx ))) {
           processIOError(idx); 
           idx--; 
         }
@@ -604,24 +517,13 @@ class FSEditLog {
     //
     // Reopen all the edits logs.
     //
-    boolean append = true;
-    for (int idx = 0; reopenEdits && idx < editStreams.length; idx++) {
-      try {
-        FileOutputStream stream = new FileOutputStream(editFiles[idx],
-                                                       append);
-        editStreams[idx] = new DataOutputStream(stream);
-        editDescriptors[idx] = stream.getFD();
-      } catch (IOException e) {
-        processIOError(idx); 
-        idx--; 
-      }
-    }
+    open();
   }
 
   /**
    * Return the name of the edit file
    */
   File getFsEditName() throws IOException {
-      return editFiles[0];
+      return getEditFile( 0 );
   }
 }

Разлика између датотеке није приказан због своје велике величине
+ 620 - 252
src/java/org/apache/hadoop/dfs/FSImage.java


+ 30 - 18
src/java/org/apache/hadoop/dfs/FSNamesystem.java

@@ -212,8 +212,7 @@ class FSNamesystem implements FSConstants {
      * dirs is a list oif directories where the filesystem directory state 
      * is stored
      */
-    public FSNamesystem(File[] dirs, 
-                        String hostname,
+    public FSNamesystem(String hostname,
                         int port,
                         NameNode nn, Configuration conf) throws IOException {
         fsNamesystemObject = this;
@@ -246,8 +245,10 @@ class FSNamesystem implements FSConstants {
 
         this.localMachine = hostname;
         this.port = port;
-        this.dir = new FSDirectory(dirs);
-        this.dir.loadFSImage( conf );
+        this.dir = new FSDirectory();
+        StartupOption startOpt = (StartupOption)conf.get( 
+                                "dfs.namenode.startup", StartupOption.REGULAR );
+        this.dir.loadFSImage( getNamespaceDirs(conf), startOpt );
         this.safeMode = new SafeModeInfo( conf );
         setBlockTotal();
         pendingReplications = new PendingReplicationBlocks(LOG);
@@ -281,6 +282,17 @@ class FSNamesystem implements FSConstants {
         LOG.info("Web-server up at: " + conf.get("dfs.info.port"));
     }
 
+    static Collection<File> getNamespaceDirs(Configuration conf) {
+      String[] dirNames = conf.getStrings("dfs.name.dir");
+      if (dirNames == null)
+        dirNames = new String[] {"/tmp/hadoop/dfs/name"};
+      Collection<File> dirs = new ArrayList<File>( dirNames.length );
+      for( int idx = 0; idx < dirNames.length; idx++ ) {
+        dirs.add( new File(dirNames[idx] ));
+      }
+      return dirs;
+    }
+
     /**
      * dirs is a list of directories where the filesystem directory state 
      * is stored
@@ -296,6 +308,11 @@ class FSNamesystem implements FSConstants {
     public static FSNamesystem getFSNamesystem() {
         return fsNamesystemObject;
     } 
+    
+    NamespaceInfo getNamespaceInfo() {
+      return new NamespaceInfo( dir.fsImage.getNamespaceID(),
+                                dir.fsImage.getCTime() );
+    }
 
     /** Close down this filesystem manager.
      * Causes heartbeat and lease daemons to stop; waits briefly for
@@ -1513,7 +1530,6 @@ class FSNamesystem implements FSConstants {
           + "node registration from " + nodeReg.getName()
           + " storage " + nodeReg.getStorageID() );
 
-      nodeReg.registrationID = getRegistrationID();
       DatanodeDescriptor nodeS = datanodeMap.get(nodeReg.getStorageID());
       DatanodeDescriptor nodeN = getDatanodeByName( nodeReg.getName() );
       
@@ -1594,12 +1610,12 @@ class FSNamesystem implements FSConstants {
     /**
      * Get registrationID for datanodes based on the namespaceID.
      * 
-     * @see #registerDatanode(DatanodeRegistration)
+     * @see #registerDatanode(DatanodeRegistration,String)
      * @see FSImage#newNamespaceID()
      * @return registration ID
      */
     public String getRegistrationID() {
-      return "NS" + Integer.toString( dir.namespaceID );
+      return Storage.getRegistrationID( dir.fsImage );
     }
     
     /**
@@ -1622,7 +1638,7 @@ class FSNamesystem implements FSConstants {
     
     private boolean isDatanodeDead(DatanodeDescriptor node) {
       return (node.getLastUpdate() <
-          (System.currentTimeMillis() - heartbeatExpireInterval));
+          (now() - heartbeatExpireInterval));
     }
     
     void setDatanodeDead(DatanodeID nodeID) throws IOException {
@@ -2450,6 +2466,11 @@ class FSNamesystem implements FSConstants {
     public Date getStartTime() {
         return startTime;
     }
+    
+    short getMaxReplication()     { return (short)maxReplication; }
+    short getMinReplication()     { return (short)minReplication; }
+    short getDefaultReplication() { return (short)defaultReplication; }
+    
     /////////////////////////////////////////////////////////
     //
     // These methods are called by the Namenode system, to see
@@ -2476,7 +2497,7 @@ class FSNamesystem implements FSConstants {
         Iterator<Block> it = null;
         int sendNum = invalidateSet.size();
         int origSize = sendNum;
-        ArrayList sendBlock = new ArrayList(sendNum);
+        ArrayList<Block> sendBlock = new ArrayList<Block>(sendNum);
 
         //
         // calculate the number of blocks that we send in one message
@@ -3825,14 +3846,6 @@ class FSNamesystem implements FSConstants {
       dir.fsImage.rollFSImage();
     }
 
-    File getFsImageName() throws IOException {
-      return dir.fsImage.getFsImageName();
-    }
-
-    File[] getFsImageNameCheckpoint() throws IOException {
-      return dir.fsImage.getFsImageNameCheckpoint();
-    }
-
     File getFsEditName() throws IOException {
       return getEditLog().getFsEditName();
     }
@@ -3876,7 +3889,6 @@ class FSNamesystem implements FSConstants {
         try {
           ServletContext context = getServletContext();
           NameNode nn = (NameNode) context.getAttribute("name.node");
-          Configuration conf = (Configuration) context.getAttribute("name.conf");
           TransferFsImage ff = new TransferFsImage(pmap, request, response);
           if (ff.getImage()) {
             // send fsImage to Secondary

+ 47 - 0
src/java/org/apache/hadoop/dfs/InconsistentFSStateException.java

@@ -0,0 +1,47 @@
+/**
+ * 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.dfs;
+
+import java.io.File;
+import java.io.IOException;
+import org.apache.hadoop.util.StringUtils;
+
+/**
+ * The exception is thrown when file system state is inconsistent 
+ * and is not recoverable. 
+ * 
+ * @author Konstantin Shvachko
+ */
+class InconsistentFSStateException extends IOException {
+
+  public InconsistentFSStateException( File dir, String descr ) {
+    super( "Directory " + getFilePath( dir )
+          + " is in an inconsistent state: " + descr );
+  }
+
+  public InconsistentFSStateException( File dir, String descr, Throwable ex ) {
+    this( dir, descr + "\n" + StringUtils.stringifyException(ex) );
+  }
+  
+  private static String getFilePath( File dir ) {
+    try {
+      return dir.getCanonicalPath();
+    } catch( IOException e ) {}
+    return dir.getPath();
+  }
+}

+ 26 - 3
src/java/org/apache/hadoop/dfs/IncorrectVersionException.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.dfs;
 
 import java.io.IOException;
@@ -10,10 +27,16 @@ import java.io.IOException;
  */
 class IncorrectVersionException extends IOException {
 
-  public IncorrectVersionException( int version, String ofWhat ) {
+  public IncorrectVersionException( int versionReported, String ofWhat ) {
+    this( versionReported, ofWhat, FSConstants.LAYOUT_VERSION );
+  }
+  
+  public IncorrectVersionException( int versionReported,
+                                    String ofWhat,
+                                    int versionExpected ) {
     super( "Unexpected version " 
-        + (ofWhat==null ? "" : "of " + ofWhat) + " reported: "
-        + version + ". Expecting = " + FSConstants.DFS_CURRENT_VERSION + "." );
+        + (ofWhat==null ? "" : "of " + ofWhat) + ". Reported: "
+        + versionReported + ". Expecting = " + versionExpected + "." );
   }
 
 }

+ 148 - 72
src/java/org/apache/hadoop/dfs/NameNode.java

@@ -28,7 +28,8 @@ import org.apache.hadoop.util.StringUtils;
 
 import java.io.*;
 import java.net.*;
-import org.apache.hadoop.dfs.DatanodeProtocol.DataNodeAction;
+import java.util.Collection;
+import java.util.Iterator;
 
 import org.apache.hadoop.metrics.MetricsRecord;
 import org.apache.hadoop.metrics.MetricsUtil;
@@ -95,26 +96,7 @@ public class NameNode implements ClientProtocol, DatanodeProtocol, FSConstants {
     /** Format a new filesystem.  Destroys any filesystem that may already
      * exist at this location.  **/
     public static void format(Configuration conf) throws IOException {
-      File[] dirs = getDirs(conf);
-      for (int idx = 0; idx < dirs.length; idx++) {
-        FSImage.format(dirs[idx]);
-      }
-      FSImage fsimage = new FSImage(dirs);
-      FSNamesystem namesystem = new FSNamesystem(fsimage);
-      fsimage.create();
-      fsimage.getEditLog().close();
-    }
-
-    /** Format a new filesystem.  Destroys any filesystem that may already
-     * exist at this location.  **/
-    public static void format(File dir) throws IOException {
-      File dirs[] = new File[1];
-      dirs[0] = dir;
-      FSImage.format(dir);
-      FSImage fsimage = new FSImage(dirs);
-      FSNamesystem namesystem = new FSNamesystem(fsimage);
-      fsimage.create();
-      fsimage.getEditLog().close();
+      format( conf, false );
     }
 
     private class NameNodeMetrics implements Updater {
@@ -170,13 +152,14 @@ public class NameNode implements ClientProtocol, DatanodeProtocol, FSConstants {
     
     /**
      * Initialize the server
-     * @param dirs the list of working directories
+     * 
      * @param hostname which hostname to bind to
      * @param port the port number to bind to
      * @param conf the configuration
      */
-    private void init(File[] dirs, String hostname, int port, 
-                      Configuration conf) throws IOException {
+    private void init(String hostname, int port, 
+                      Configuration conf
+                      ) throws IOException {
       this.handlerCount = conf.getInt("dfs.namenode.handler.count", 10);
       this.server = RPC.getServer(this, hostname, port, handlerCount, 
                                   false, conf);
@@ -187,9 +170,9 @@ public class NameNode implements ClientProtocol, DatanodeProtocol, FSConstants {
       LOG.info("Namenode up at: " + this.nameNodeAddress);
 
       try {
-        this.namesystem = new FSNamesystem(dirs, this.nameNodeAddress.getHostName(), this.nameNodeAddress.getPort(), this, conf);
+        this.namesystem = new FSNamesystem(this.nameNodeAddress.getHostName(), this.nameNodeAddress.getPort(), this, conf);
         this.server.start();  //start RPC server   
-
+  
         this.emptier = new Thread(new Trash(conf).getEmptier(), "Trash Emptier");
         this.emptier.setDaemon(true);
         this.emptier.start();
@@ -201,16 +184,31 @@ public class NameNode implements ClientProtocol, DatanodeProtocol, FSConstants {
     }
     
     /**
-     * Create a NameNode at the default location.
+     * Start NameNode.
+     * <p>
+     * The name-node can be started with one of the following startup options:
+     * <ul> 
+     * <li>{@link FSConstants.StartupOption#REGULAR REGULAR} - normal startup</li>
+     * <li>{@link FSConstants.StartupOption#FORMAT FORMAT} - format name node</li>
+     * <li>{@link FSConstants.StartupOption#UPGRADE UPGRADE} - start the cluster  
+     * upgrade and create a snapshot of the current file system state</li> 
+     * <li>{@link FSConstants.StartupOption#ROLLBACK ROLLBACK} - roll the  
+     *            cluster back to the previous state</li>
+     * </ul>
+     * The option is passed via configuration field: 
+     * <tt>dfs.namenode.startup</tt>
      * 
      * The conf will be modified to reflect the actual ports on which 
      * the NameNode is up and running if the user passes the port as
      * <code>zero</code> in the conf.
+     * 
+     * @param conf  confirguration
+     * @throws IOException
      */
     public NameNode(Configuration conf) throws IOException {
       InetSocketAddress addr = 
         DataNode.createSocketAddr(conf.get("fs.default.name"));
-      init(getDirs(conf), addr.getHostName(), addr.getPort(), conf);
+      init( addr.getHostName(), addr.getPort(), conf );
     }
 
     /**
@@ -220,11 +218,15 @@ public class NameNode implements ClientProtocol, DatanodeProtocol, FSConstants {
      * the NameNode is up and running if the user passes the port as
      * <code>zero</code>.  
      */
-    public NameNode(File[] dirs, String bindAddress, int port, Configuration conf) throws IOException {
-       init(dirs, bindAddress, port, conf);
+    public NameNode(String bindAddress, int port, 
+                    Configuration conf
+                    ) throws IOException {
+      init( bindAddress, port, conf );
     }
 
-    /** Return the configured directories where name data is stored. */
+    /** Return the configured directories where name data is stored. 
+     * @deprecated
+     */
     static File[] getDirs(Configuration conf) {
       String[] dirNames = conf.getStrings("dfs.name.dir");
       if (dirNames == null) { dirNames = new String[] {"/tmp/hadoop/dfs/name"}; }
@@ -578,6 +580,10 @@ public class NameNode implements ClientProtocol, DatanodeProtocol, FSConstants {
     public void rollFsImage() throws IOException {
       namesystem.rollFSImage();
     }
+    
+    public void finalizeUpgrade() throws IOException {
+      getFSImage().finalizeUpgrade();
+    }
 
     ////////////////////////////////////////////////////////////////
     // DatanodeProtocol
@@ -598,11 +604,11 @@ public class NameNode implements ClientProtocol, DatanodeProtocol, FSConstants {
      * Return a block-oriented command for the datanode to execute.
      * This will be either a transfer or a delete operation.
      */
-    public BlockCommand sendHeartbeat(DatanodeRegistration nodeReg,
-                                      long capacity, 
-                                      long remaining,
-                                      int xmitsInProgress,
-                                      int xceiverCount) throws IOException {
+    public DatanodeCommand sendHeartbeat( DatanodeRegistration nodeReg,
+                                          long capacity, 
+                                          long remaining,
+                                          int xmitsInProgress,
+                                          int xceiverCount) throws IOException {
         Object xferResults[] = new Object[2];
         xferResults[0] = xferResults[1] = null;
         Object deleteList[] = new Object[1];
@@ -616,7 +622,7 @@ public class NameNode implements ClientProtocol, DatanodeProtocol, FSConstants {
                                      deleteList)) {
           // request block report from the datanode
           assert(xferResults[0] == null && deleteList[0] == null);
-          return new BlockCommand( DataNodeAction.DNA_REGISTER );
+          return new DatanodeCommand( DataNodeAction.DNA_REGISTER );
         }
         
         //
@@ -639,14 +645,19 @@ public class NameNode implements ClientProtocol, DatanodeProtocol, FSConstants {
         return null;
     }
 
-    public Block[] blockReport( DatanodeRegistration nodeReg,
-                                Block blocks[]) throws IOException {
+    public DatanodeCommand blockReport( DatanodeRegistration nodeReg,
+                                        Block blocks[]) throws IOException {
         verifyRequest( nodeReg );
         stateChangeLog.debug("*BLOCK* NameNode.blockReport: "
                 +"from "+nodeReg.getName()+" "+blocks.length+" blocks" );
 
-        return namesystem.processReport( nodeReg, blocks );
-     }
+        Block blocksToDelete[] = namesystem.processReport( nodeReg, blocks );
+        if( blocksToDelete != null && blocksToDelete.length > 0 )
+            return new BlockCommand( blocksToDelete );
+        if( getFSImage().isUpgradeFinalized() )
+          return new DatanodeCommand( DataNodeAction.DNA_FINALIZE );
+        return null;
+    }
 
     public void blockReceived(DatanodeRegistration nodeReg, 
                               Block blocks[]) throws IOException {
@@ -664,12 +675,19 @@ public class NameNode implements ClientProtocol, DatanodeProtocol, FSConstants {
                             int errorCode, 
                             String msg) throws IOException {
       // Log error message from datanode
+      LOG.info("Report from " + nodeReg.getName() + ": " + msg);
+      if( errorCode == DatanodeProtocol.NOTIFY ) {
+        return;
+      }
       verifyRequest( nodeReg );
-      LOG.warn("Report from " + nodeReg.getName() + ": " + msg);
       if( errorCode == DatanodeProtocol.DISK_ERROR ) {
           namesystem.removeDatanode( nodeReg );            
       }
     }
+    
+    public NamespaceInfo versionRequest() throws IOException {
+      return namesystem.getNamespaceInfo();
+    }
 
     /** 
      * Verify request.
@@ -693,7 +711,7 @@ public class NameNode implements ClientProtocol, DatanodeProtocol, FSConstants {
      * @throws IOException
      */
     public void verifyVersion( int version ) throws IOException {
-      if( version != DFS_CURRENT_VERSION )
+      if( version != LAYOUT_VERSION )
         throw new IncorrectVersionException( version, "data node" );
     }
 
@@ -701,7 +719,11 @@ public class NameNode implements ClientProtocol, DatanodeProtocol, FSConstants {
      * Returns the name of the fsImage file
      */
     public File getFsImageName() throws IOException {
-      return namesystem.getFsImageName();
+      return getFSImage().getFsImageName();
+    }
+    
+    FSImage getFSImage() {
+      return namesystem.dir.fsImage;
     }
 
     /**
@@ -709,7 +731,7 @@ public class NameNode implements ClientProtocol, DatanodeProtocol, FSConstants {
      * checkpointing
      */
     public File[] getFsImageNameCheckpoint() throws IOException {
-      return namesystem.getFsImageNameCheckpoint();
+      return getFSImage().getFsImageNameCheckpoint();
     }
 
     /**
@@ -726,38 +748,92 @@ public class NameNode implements ClientProtocol, DatanodeProtocol, FSConstants {
     public InetSocketAddress getNameNodeAddress() {
       return nameNodeAddress;
     }
+
+    /**
+     * Verify that configured directories exist, then
+     * Interactively confirm that formatting is desired 
+     * for each existing directory and format them.
+     * 
+     * @param conf
+     * @param isConfirmationNeeded
+     * @return true if formatting was aborted, false otherwise
+     * @throws IOException
+     */
+    private static boolean format(Configuration conf,
+                                  boolean isConfirmationNeeded
+                                ) throws IOException {
+      Collection<File> dirsToFormat = FSNamesystem.getNamespaceDirs( conf );
+      for( Iterator<File> it = dirsToFormat.iterator(); it.hasNext(); ) {
+        File curDir = it.next();
+        if( ! curDir.exists() )
+          continue;
+        if( isConfirmationNeeded ) {
+          System.err.print("Re-format filesystem in " + curDir +" ? (Y or N) ");
+          if (!(System.in.read() == 'Y')) {
+            System.err.println("Format aborted in "+ curDir);
+            return true;
+          }
+          while( System.in.read() != '\n' ); // discard the enter-key
+        }
+      }
+
+      FSNamesystem nsys = new FSNamesystem(new FSImage( dirsToFormat ));
+      nsys.dir.fsImage.format();
+      return false;
+    }
+
+    private static void printUsage() {
+      System.err.println(
+      "Usage: java NameNode [-format] | [-upgrade] | [-rollback]");
+    }
+
+    private static StartupOption parseArguments(String args[], 
+                                                Configuration conf ) {
+      int argsLen = (args == null) ? 0 : args.length;
+      StartupOption startOpt = StartupOption.REGULAR;
+      for( int i=0; i < argsLen; i++ ) {
+        String cmd = args[i];
+        if( "-format".equalsIgnoreCase(cmd) ) {
+          startOpt = StartupOption.FORMAT;
+        } else if( "-regular".equalsIgnoreCase(cmd) ) {
+          startOpt = StartupOption.REGULAR;
+        } else if( "-upgrade".equalsIgnoreCase(cmd) ) {
+          startOpt = StartupOption.UPGRADE;
+        } else if( "-rollback".equalsIgnoreCase(cmd) ) {
+          startOpt = StartupOption.ROLLBACK;
+        } else
+          return null;
+      }
+      conf.setObject( "dfs.namenode.startup", startOpt );
+      return startOpt;
+    }
+
+    static NameNode createNameNode( String argv[], 
+                                    Configuration conf ) throws Exception {
+      if( conf == null )
+        conf = new Configuration();
+      StartupOption startOpt = parseArguments( argv, conf );
+      if( startOpt == null ) {
+        printUsage();
+        return null;
+      }
+      
+      if( startOpt == StartupOption.FORMAT ) {
+        boolean aborted = format( conf, true );
+        System.exit(aborted ? 1 : 0);
+      }
+      
+      NameNode namenode = new NameNode(conf);
+      return namenode;
+    }
     
     /**
      */
     public static void main(String argv[]) throws Exception {
       try {
-        Configuration conf = new Configuration();
-
-        if (argv.length == 1 && argv[0].equals("-format")) {
-          boolean aborted = false;
-          File[] dirs = getDirs(conf);
-          for (int idx = 0; idx < dirs.length; idx++) {
-            if (dirs[idx].exists()) {
-              System.err.print("Re-format filesystem in " + dirs[idx] +" ? (Y or N) ");
-              if (!(System.in.read() == 'Y')) {
-                System.err.println("Format aborted in "+ dirs[idx]);
-                aborted = true;
-              } else {
-                format(dirs[idx]);
-                System.err.println("Formatted "+dirs[idx]);
-              }
-              System.in.read(); // discard the enter-key
-            }else{
-              format(dirs[idx]);
-              System.err.println("Formatted "+dirs[idx]);
-            }
-          }
-          System.exit(aborted ? 1 : 0);
-        }
-        
-        NameNode namenode = new NameNode(conf);
-        namenode.join();
-        
+        NameNode namenode = createNameNode( argv, null );
+        if( namenode != null )
+          namenode.join();
       } catch ( Throwable e ) {
         LOG.error( StringUtils.stringifyException( e ) );
         System.exit(-1);

+ 75 - 0
src/java/org/apache/hadoop/dfs/NamespaceInfo.java

@@ -0,0 +1,75 @@
+/**
+ * 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.dfs;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.io.UTF8;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableFactories;
+import org.apache.hadoop.io.WritableFactory;
+
+/**
+ * NamespaceInfo is returned by the name-node in reply 
+ * to a data-node handshake.
+ * 
+ * @author Konstantin Shvachko
+ */
+class NamespaceInfo extends StorageInfo implements Writable {
+  String  buildVersion;
+
+  public NamespaceInfo() {
+    super();
+    buildVersion = null;
+  }
+  
+  public NamespaceInfo( int nsID, long cT ) {
+    super( FSConstants.LAYOUT_VERSION, nsID, cT );
+    buildVersion = Storage.getBuildVersion();
+  }
+  
+  public String getBuildVersion() { return buildVersion; }
+  
+  /////////////////////////////////////////////////
+  // Writable
+  /////////////////////////////////////////////////
+  static {                                      // register a ctor
+    WritableFactories.setFactory
+      (NamespaceInfo.class,
+       new WritableFactory() {
+         public Writable newInstance() { return new NamespaceInfo(); }
+       });
+  }
+
+  public void write(DataOutput out) throws IOException {
+    UTF8.writeString( out, getBuildVersion() );
+    out.writeInt( getLayoutVersion() );
+    out.writeInt( getNamespaceID() );
+    out.writeLong( getCTime() );
+  }
+
+  public void readFields(DataInput in) throws IOException {
+    buildVersion = UTF8.readString( in );
+    layoutVersion = in.readInt();
+    namespaceID = in.readInt();
+    cTime = in.readLong();
+  }
+}

+ 7 - 14
src/java/org/apache/hadoop/dfs/SecondaryNameNode.java

@@ -19,8 +19,6 @@ package org.apache.hadoop.dfs;
 
 import org.apache.commons.logging.*;
 
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.*;
 import org.apache.hadoop.ipc.*;
 import org.apache.hadoop.conf.*;
 import org.apache.hadoop.util.StringUtils;
@@ -60,7 +58,6 @@ public class SecondaryNameNode implements FSConstants, Runnable {
 
     private ClientProtocol namenode;
     private Configuration conf;
-    private String localName;
     private InetSocketAddress nameNodeAddr;
     private boolean shouldRun;
     private StatusHttpServer infoServer;
@@ -95,11 +92,6 @@ public class SecondaryNameNode implements FSConstants, Runnable {
       this.conf = conf;
       this.namenode = (ClientProtocol) RPC.getProxy(ClientProtocol.class,
                        ClientProtocol.versionID, nameNodeAddr, conf);
-      try {
-        this.localName = InetAddress.getLocalHost().getHostName();
-      } catch (UnknownHostException uhe) {
-        this.localName = "";
-      }
 
       //
       // initialize the webserver for uploading files.
@@ -250,7 +242,7 @@ public class SecondaryNameNode implements FSConstants, Runnable {
     void doCheckpoint() throws IOException {
 
       //
-      // Do the rquired initialization of the merge work area.
+      // Do the required initialization of the merge work area.
       //
       doSetup();
 
@@ -297,11 +289,12 @@ public class SecondaryNameNode implements FSConstants, Runnable {
      * DEST_FS_IMAGE
      */
     private void doMerge() throws IOException {
-      FSImage fsImage = new FSImage(checkpointDir, FS_EDITS);
-      FSNamesystem namesystem = new FSNamesystem(fsImage);
-      fsImage.loadFSImage(conf, srcImage);
-      fsImage.getEditLog().loadFSEdits(conf, editFile);
-      fsImage.saveFSImage(DEST_FS_IMAGE);
+      FSNamesystem namesystem = new FSNamesystem(
+                                    new FSImage(checkpointDir));
+      FSImage fsImage = namesystem.dir.fsImage;
+      fsImage.loadFSImage(srcImage);
+      fsImage.getEditLog().loadFSEdits(editFile);
+      fsImage.saveFSImage(destImage);
     }
 
     /**

+ 538 - 0
src/java/org/apache/hadoop/dfs/Storage.java

@@ -0,0 +1,538 @@
+/**
+ * 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.dfs;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.RandomAccessFile;
+import java.nio.channels.FileLock;
+import java.util.AbstractList;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Iterator;
+import java.util.Properties;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.dfs.FSConstants.StartupOption;
+import org.apache.hadoop.dfs.FSConstants.NodeType;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.util.VersionInfo;
+
+/**
+ * Common class for storage information.
+ * 
+ * TODO namespaceID should be long and computed as hash( address + port )
+ * @author Konstantin Shvachko
+ */
+class StorageInfo {
+  int   layoutVersion;  // Version read from the stored file.
+  int   namespaceID;    // namespace id of the storage
+  long  cTime;          // creation timestamp
+  
+  StorageInfo () {
+    this( 0, 0, 0L );
+  }
+  
+  StorageInfo( int layoutV, int nsID, long cT ) {
+    layoutVersion = layoutV;
+    namespaceID = nsID;
+    cTime = cT;
+  }
+  
+  StorageInfo( StorageInfo from ) {
+    layoutVersion = from.layoutVersion;
+    namespaceID = from.namespaceID;
+    cTime = from.cTime;
+  }
+
+  public int    getLayoutVersion(){ return layoutVersion; }
+  public int    getNamespaceID()  { return namespaceID; }
+  public long   getCTime()        { return cTime; }
+}
+
+/**
+ * Storage information file.
+ * <p>
+ * Local storage information is stored in a separate file VERSION.
+ * It contains type of the node, 
+ * the storage layout version, the namespace id, and 
+ * the fs state creation time.
+ * <p>
+ * Local storage can reside in multiple directories. 
+ * Each directory should contain the same VERSION file as the others.
+ * During startup Hadoop servers (name-node and data-nodes) read their local 
+ * storage information from them.
+ * <p>
+ * The servers hold a lock for each storage directory while they run so that 
+ * other nodes were not able to startup sharing the same storage.
+ * The locks are released when the servers stop (normally or abnormally).
+ * 
+ * @author Konstantin Shvachko
+ */
+abstract class Storage extends StorageInfo {
+  public static final Log LOG = LogFactory.getLog("org.apache.hadoop.dfs.Storage");
+
+  // Constants
+  
+  // last layout version that did not suppot upgrades
+  protected static final int LAST_PRE_UPGRADE_LAYOUT_VERSION = -3;
+  
+  private   static final String STORAGE_FILE_LOCK     = "in_use.lock";
+  protected static final String STORAGE_FILE_VERSION  = "VERSION";
+  private   static final String STORAGE_DIR_CURRENT   = "current";
+  private   static final String STORAGE_DIR_PREVIOUS  = "previous";
+  private   static final String STORAGE_TMP_REMOVED   = "removed.tmp";
+  private   static final String STORAGE_TMP_PREVIOUS  = "previous.tmp";
+  private   static final String STORAGE_TMP_FINALIZED = "finalized.tmp";
+  
+  protected enum StorageState {
+    NON_EXISTENT,
+    NOT_FORMATTED,
+    CONVERT,
+    COMPLETE_UPGRADE,
+    RECOVER_UPGRADE,
+    COMPLETE_FINALIZE,
+    COMPLETE_ROLLBACK,
+    RECOVER_ROLLBACK,
+    NORMAL;
+  }
+  
+  private NodeType storageType;    // Type of the node using this storage 
+  protected List<StorageDirectory> storageDirs;
+  
+  /**
+   * One of the storage directories.
+   */
+  class StorageDirectory {
+    File              root; // root directory
+    FileLock          lock; // storage lock
+    
+    StorageDirectory( File dir ) {
+      this.root = dir;
+      this.lock = null;
+    }
+
+    /**
+     * Read version file.
+     * 
+     * @throws IOException if file cannot be read or contains inconsistent data
+     */
+    void read() throws IOException {
+      read( getVersionFile() );
+    }
+    
+    void read( File from ) throws IOException {
+      RandomAccessFile file = new RandomAccessFile( from, "rws" );
+      try {
+        FileInputStream in = new FileInputStream( file.getFD() );
+        file.seek(0);
+        Properties props = new Properties();
+        props.load( in );
+        getFields( props, this );
+      } finally {
+        file.close();
+      }
+    }
+
+    /**
+     * Write version file.
+     * 
+     * @throws IOException
+     */
+    void write() throws IOException {
+      write( getVersionFile() );
+    }
+
+    void write( File to ) throws IOException {
+      Properties props = new Properties();
+      setFields( props, this );
+      RandomAccessFile file = new RandomAccessFile( to, "rws" );
+      try {
+        file.seek(0);
+        FileOutputStream out = new FileOutputStream( file.getFD() );
+        props.store( out, null );
+      } finally {
+        file.close();
+      }
+    }
+
+    /**
+     * Clear and re-create storage directory.
+     * <p>
+     * Removes contents of the current directory and creates an empty directory.
+     * 
+     * This does not fully format storage directory. 
+     * It cannot write the version file since it should be written last after  
+     * all other storage type dependent files are written.
+     * Derived storage is responsible for setting specific storage values and
+     * writing the version file to disk.
+     * 
+     * @throws IOException
+     */
+    void clearDirectory() throws IOException {
+      File curDir = this.getCurrentDir();
+      if( curDir.exists() )
+        if( ! (FileUtil.fullyDelete( curDir )) )
+          throw new IOException("Cannot remove current directory: " + curDir );
+      if( ! curDir.mkdirs() )
+        throw new IOException( "Cannot create directory " + curDir );
+    }
+
+    File getCurrentDir() {
+      return new File( root, STORAGE_DIR_CURRENT );
+    }
+    File getVersionFile() {
+      return new File( new File( root, STORAGE_DIR_CURRENT ), STORAGE_FILE_VERSION );
+    }
+    File getPreviousVersionFile() {
+      return new File( new File( root, STORAGE_DIR_PREVIOUS ), STORAGE_FILE_VERSION );
+    }
+    File getPreviousDir() {
+      return new File( root, STORAGE_DIR_PREVIOUS );
+    }
+    File getPreviousTmp() {
+      return new File( root, STORAGE_TMP_PREVIOUS );
+    }
+    File getRemovedTmp() {
+      return new File( root, STORAGE_TMP_REMOVED );
+    }
+    File getFinalizedTmp() {
+      return new File( root, STORAGE_TMP_FINALIZED );
+    }
+
+    /**
+     * Check consistency of the storage directory
+     * 
+     * @param startOpt a startup option.
+     *  
+     * @return state {@link StorageState} of the storage directory 
+     * @throws {@link InconsistentFSStateException} if directory state is not 
+     * consistent and cannot be recovered 
+     */
+    StorageState analyzeStorage( StartupOption startOpt ) throws IOException {
+      assert root != null : "root is null";
+      String rootPath = root.getCanonicalPath();
+      try { // check that storage exists
+        if( ! root.exists() ) {
+          // storage directory does not exist
+          if( startOpt != StartupOption.FORMAT ) {
+            LOG.info( "Storage directory " + rootPath + " does not exist." );
+            return StorageState.NON_EXISTENT;
+          }
+          LOG.info( rootPath + " does not exist. Creating ..." );
+          if( ! root.mkdirs() )
+            throw new IOException( "Cannot create directory " + rootPath );
+        }
+        // or is inaccessible
+        if( ! root.isDirectory() ) {
+          LOG.info( rootPath + "is not a directory." );
+          return StorageState.NON_EXISTENT;
+        }
+        if( ! root.canWrite() ) {
+          LOG.info( "Cannot access storage directory " + rootPath );
+          return StorageState.NON_EXISTENT;
+        }
+      } catch( SecurityException ex ) {
+        LOG.info( "Cannot access storage directory " + rootPath, ex );
+        return StorageState.NON_EXISTENT;
+      }
+
+      this.lock(); // lock storage if it exists
+
+      if( startOpt == StartupOption.FORMAT )
+        return StorageState.NOT_FORMATTED;
+      // check whether a conversion is required
+      if( isConversionNeeded( this ) )
+        return StorageState.CONVERT;
+      // check whether current directory is valid
+      File versionFile = getVersionFile();
+      boolean hasCurrent = versionFile.exists();
+
+      // check which directories exist
+      boolean hasPrevious = getPreviousDir().exists();
+      boolean hasPreviousTmp = getPreviousTmp().exists();
+      boolean hasRemovedTmp = getRemovedTmp().exists();
+      boolean hasFinalizedTmp = getFinalizedTmp().exists();
+
+      if( !(hasPreviousTmp || hasRemovedTmp || hasFinalizedTmp) ) {
+        // no temp dirs - no recovery
+        if( hasCurrent )
+          return StorageState.NORMAL;
+        if( hasPrevious )
+          throw new InconsistentFSStateException( root,
+                      "version file in current directory it is missing." );
+        return StorageState.NOT_FORMATTED;
+      }
+
+      if( (hasPreviousTmp?1:0)+(hasRemovedTmp?1:0)+(hasFinalizedTmp?1:0) > 1 )
+        // more than one temp dirs
+        throw new InconsistentFSStateException( root,
+                    "too many temporary directories." );
+
+      // # of temp dirs == 1 should either recover or complete a transition
+      if( hasFinalizedTmp ) {
+        if( hasPrevious )
+          throw new InconsistentFSStateException( root,
+              STORAGE_DIR_PREVIOUS + " and " + STORAGE_TMP_FINALIZED
+              + "cannot exist together." );
+        return StorageState.COMPLETE_FINALIZE;
+      }
+
+      if( hasPreviousTmp ) {
+        if( hasPrevious )
+          throw new InconsistentFSStateException( root,
+              STORAGE_DIR_PREVIOUS + " and " + STORAGE_TMP_PREVIOUS
+              + " cannot exist together." );
+        if( hasCurrent )
+          return StorageState.COMPLETE_UPGRADE;
+        return StorageState.RECOVER_UPGRADE;
+      }
+      
+      assert hasRemovedTmp : "hasRemovedTmp must be true";
+      if( !(hasCurrent ^ hasPrevious) )
+        throw new InconsistentFSStateException( root,
+            "one and only one directory " + STORAGE_DIR_CURRENT 
+            + " or " + STORAGE_DIR_PREVIOUS 
+            + " must be present when " + STORAGE_TMP_REMOVED
+            + " exists." );
+      if( hasCurrent )
+        return StorageState.COMPLETE_ROLLBACK;
+      return StorageState.RECOVER_ROLLBACK;
+    }
+
+    /**
+     * Complete or recover storage state from previously failed transition.
+     * 
+     * @param curState specifies what/how the state should be recovered
+     * @throws IOException
+     */
+    void doRecover( StorageState curState ) throws IOException {
+      File curDir = getCurrentDir();
+      String rootPath = root.getCanonicalPath();
+      switch( curState ) {
+        case COMPLETE_UPGRADE:  // mv previous.tmp -> previous
+          LOG.info( "Completing previous upgrade for storage directory " 
+                    + rootPath + "." );
+          rename( getPreviousTmp(), getPreviousDir() );
+          return;
+        case RECOVER_UPGRADE:   // mv previous.tmp -> current
+          LOG.info( "Recovering storage directory " + rootPath
+                    + " from previous upgrade." );
+          if( curDir.exists() )
+            deleteDir( curDir );
+          rename( getPreviousTmp(), curDir );
+          return;
+        case COMPLETE_ROLLBACK: // rm removed.tmp
+          LOG.info( "Completing previous rollback for storage directory "
+                    + rootPath + "." );
+          deleteDir( getRemovedTmp() );
+          return;
+        case RECOVER_ROLLBACK:  // mv removed.tmp -> current
+          LOG.info( "Recovering storage directory " + rootPath
+                    + " from previous rollback." );
+          rename( getRemovedTmp(), curDir );
+          return;
+        case COMPLETE_FINALIZE: // rm finalized.tmp
+          LOG.info( "Completing previous finalize for storage directory "
+                    + rootPath + "." );
+          deleteDir( getFinalizedTmp() );
+          return;
+        default:
+          throw new IOException( "Unexpected FS state: " + curState );
+      }
+    }
+
+    /**
+     * Lock storage.
+     * 
+     * @throws IOException if locking fails
+     */
+    void lock() throws IOException {
+      File lockF = new File( root, STORAGE_FILE_LOCK );
+      lockF.deleteOnExit();
+      RandomAccessFile file = new RandomAccessFile( lockF, "rws" );
+      try {
+        this.lock = file.getChannel().tryLock();
+      } catch( IOException e ) {
+        LOG.info( StringUtils.stringifyException(e) );
+        file.close();
+        throw e;
+      }
+      if( lock == null ) {
+        String msg = "Cannot lock storage " + this.root 
+                      + ". The directory is already locked.";
+        LOG.info( msg );
+        file.close();
+        throw new IOException( msg );
+      }
+    }
+
+    /**
+     * Unlock storage.
+     * 
+     * @throws IOException
+     */
+    void unlock() throws IOException {
+      if( this.lock == null )
+        return;
+      this.lock.release();
+      lock.channel().close();
+    }
+  }
+
+  /**
+   * Create empty storage info of the specified type
+   */
+  Storage( NodeType type ) {
+    super();
+    this.storageType = type;
+  }
+  
+  Storage( NodeType type, int nsID, long cT ) {
+    super( FSConstants.LAYOUT_VERSION, nsID, cT );
+    this.storageType = type;
+  }
+  
+  Storage( NodeType type, StorageInfo storageInfo ) {
+    super( storageInfo );
+    this.storageType = type;
+  }
+  
+  int getNumStorageDirs() {
+    return storageDirs.size();
+  }
+  
+  StorageDirectory getStorageDir( int idx ) {
+    return storageDirs.get( idx );
+  }
+  
+  protected void addStorageDir( StorageDirectory sd ) {
+    storageDirs.add( sd );
+  }
+  
+  abstract boolean isConversionNeeded( StorageDirectory sd ) throws IOException;
+  
+  /**
+   * Get common storage fields.
+   * Should be overloaded if additional fields need to be get.
+   * 
+   * @param props
+   * @throws IOException
+   */
+  protected void getFields( Properties props, 
+                            StorageDirectory sd 
+                          ) throws IOException {
+    String sv, st, sid, sct;
+    sv = props.getProperty( "layoutVersion" );
+    st = props.getProperty( "storageType" );
+    sid = props.getProperty( "namespaceID" );
+    sct = props.getProperty( "cTime" );
+    if( sv == null || st == null || sid == null || sct == null )
+      throw new InconsistentFSStateException( sd.root,
+                    "file " + STORAGE_FILE_VERSION + " is invalid." );
+    int rv = Integer.parseInt( sv );
+    NodeType rt = NodeType.valueOf( st );
+    int rid = Integer.parseInt( sid );
+    long rct = Long.parseLong( sct );
+    if( ! storageType.equals( rt ) ||
+        ! (( namespaceID == 0 ) || ( rid == 0 ) || namespaceID == rid ))
+      throw new InconsistentFSStateException( sd.root,
+                  "is incompatible with others." );
+    if( rv < FSConstants.LAYOUT_VERSION ) // future version
+        throw new IncorrectVersionException(rv, "storage directory " 
+                                            + sd.root.getCanonicalPath() );
+    layoutVersion = rv;
+    storageType = rt;
+    namespaceID = rid;
+    cTime = rct;
+  }
+  
+  /**
+   * Set common storage fields.
+   * Should be overloaded if additional fields need to be set.
+   * 
+   * @param props
+   * @throws IOException
+   */
+  protected void setFields( Properties props, 
+                            StorageDirectory sd 
+                          ) throws IOException {
+    props.setProperty( "layoutVersion", String.valueOf( layoutVersion ));
+    props.setProperty( "storageType", storageType.toString() );
+    props.setProperty( "namespaceID", String.valueOf( namespaceID ));
+    props.setProperty( "cTime", String.valueOf( cTime ));
+  }
+
+  static void rename( File from, File to ) throws IOException {
+    if( ! from.renameTo( to ))
+      throw new IOException( "Failed to rename " 
+          + from.getCanonicalPath() + " to " + to.getCanonicalPath() );
+  }
+
+  static void deleteDir( File dir ) throws IOException {
+    if( ! FileUtil.fullyDelete( dir ) )
+      throw new IOException( "Failed to delete " + dir.getCanonicalPath() );
+  }
+  
+  /**
+   * Write all data storage files.
+   * @throws IOException
+   */
+  public void writeAll() throws IOException {
+    this.layoutVersion = FSConstants.LAYOUT_VERSION;
+    for (Iterator<StorageDirectory> it = storageDirs.iterator(); it.hasNext();) {
+      it.next().write();
+    }
+  }
+
+  /**
+   * Close all the version files.
+   * @throws IOException
+   */
+  public void unlockAll() throws IOException {
+    for (Iterator<StorageDirectory> it = storageDirs.iterator(); it.hasNext();) {
+      it.next().unlock();
+    }
+  }
+
+  public static String getBuildVersion() {
+    return VersionInfo.getRevision();
+  }
+
+  static String getRegistrationID( StorageInfo storage ) {
+    return "NS-" + Integer.toString( storage.getNamespaceID() )
+           + "-" + Integer.toString( storage.getLayoutVersion() )
+           + "-" + Long.toString( storage.getCTime() );
+  }
+  
+  /**
+   * @deprecated
+   * Provides conversion for deprecated DataNode constructor, should be removed
+   */
+  static AbstractList<File> makeListOfFiles( String[] dirs ) {
+    AbstractList<File> list = new ArrayList<File>( dirs.length );
+    for (int idx = 0; idx < dirs.length; idx++) {
+      list.add(new File(dirs[idx]));
+    }
+    return list;
+  }
+}

+ 60 - 0
src/java/org/apache/hadoop/fs/FileUtil.java

@@ -24,6 +24,7 @@ import java.util.zip.ZipEntry;
 import java.util.zip.ZipFile;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.util.StringUtils;
 
 /**
  * A collection of file-processing util methods
@@ -318,6 +319,65 @@ public class FileUtil {
     }
   }
   
+  /**
+   * Class for creating hardlinks.
+   * Supports Unix, Cygwin, WindXP.
+   *  
+   * @author Konstantin Shvachko
+   */
+  public static class HardLink { 
+    enum OSType {
+      OS_TYPE_UNIX, 
+      OS_TYPE_WINXP; 
+    }
+  
+    private static String[] hardLinkCommand;
+    
+    static {
+      switch( getOSType() ) {
+      case OS_TYPE_WINXP:
+        hardLinkCommand = new String[] {"fsutil","hardlink","create",null,null};
+        break;
+      case OS_TYPE_UNIX:
+      default:
+        hardLinkCommand = new String[] {"ln",null,null};
+      }
+    }
+
+    static OSType getOSType() {
+      String osName = System.getProperty("os.name");
+      if( osName.indexOf( "Windows") >= 0 && 
+          (osName.indexOf( "XpP") >= 0 || osName.indexOf( "2003") >= 0 ) )
+        return OSType.OS_TYPE_WINXP;
+      else
+        return OSType.OS_TYPE_UNIX;
+    }
+    
+    public static void createHardLink(File target, 
+                                      File linkName ) throws IOException {
+      int len = hardLinkCommand.length;
+      hardLinkCommand[len-2] = target.getCanonicalPath();
+      hardLinkCommand[len-1] = linkName.getCanonicalPath();
+      // execute shell command
+      Process process = Runtime.getRuntime().exec( hardLinkCommand );
+      try {
+        if (process.waitFor() != 0) {
+          String errMsg = new BufferedReader(new InputStreamReader(
+              process.getInputStream())).readLine();
+          if( errMsg == null )  errMsg = "";
+          String inpMsg = new BufferedReader(new InputStreamReader(
+              process.getErrorStream())).readLine();
+          if( inpMsg == null )  inpMsg = "";
+          throw new IOException( errMsg + inpMsg );
+        }
+      } catch (InterruptedException e) {
+        throw new IOException( StringUtils.stringifyException( e ));
+      } finally {
+        process.destroy();
+      }
+    }
+  }
+
   /**
    * Create a soft link between a src and destination
    * only on a local disk. HDFS does not support this

+ 2 - 2
src/test/org/apache/hadoop/dfs/ClusterTestDFS.java

@@ -219,8 +219,8 @@ public class ClusterTestDFS extends TestCase implements FSConstants {
 
     int nameNodePort = 9000 + testCycleNumber++; // ToDo: settable base port
     String nameNodeSocketAddr = "localhost:" + nameNodePort;
-    NameNode nameNodeDaemon = new NameNode(new File[] { new File(nameFSDir) },
-        "localhost", nameNodePort, conf);
+    conf.set("dfs.name.dir", nameFSDir);
+    NameNode nameNodeDaemon = new NameNode("localhost", nameNodePort, conf);
     DFSClient dfsClient = null;
     try {
       //

+ 1 - 2
src/test/org/apache/hadoop/dfs/ClusterTestDFSNamespaceLogging.java

@@ -352,8 +352,7 @@ public class ClusterTestDFSNamespaceLogging extends TestCase implements FSConsta
 	
     NameNode.format(conf);
     
-    nameNodeDaemon = new NameNode(new File[] { new File(nameFSDir) },
-        "localhost", nameNodePort, conf);
+    nameNodeDaemon = new NameNode("localhost", nameNodePort, conf);
 
      //
       //        start DataNodes

+ 43 - 10
src/test/org/apache/hadoop/dfs/MiniDFSCluster.java

@@ -20,6 +20,7 @@ package org.apache.hadoop.dfs;
 import java.io.*;
 import java.net.*;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.dfs.FSConstants.StartupOption;
 import org.apache.hadoop.fs.*;
 import org.apache.hadoop.net.NetworkTopology;
 
@@ -180,7 +181,7 @@ public class MiniDFSCluster {
           int nDatanodes,
           boolean formatNamenode,
           String[] racks) throws IOException {
-    this(0, conf, nDatanodes, false, formatNamenode, racks);
+    this(0, conf, nDatanodes, formatNamenode, racks);
   }
   
   /**
@@ -254,6 +255,7 @@ public class MiniDFSCluster {
    * @param dataNodeFirst should the datanode be brought up before the namenode?
    * @param formatNamenode should the namenode be formatted before starting up ?
    * @param racks array of strings indicating racks that each datanode is on
+   * @deprecated use {@link #MiniDFSCluster(Configuration, int, String[])}
    */
   public MiniDFSCluster(int namenodePort, 
                         Configuration conf,
@@ -261,7 +263,42 @@ public class MiniDFSCluster {
                         boolean dataNodeFirst,
                         boolean formatNamenode,
                         String[] racks) throws IOException {
+    this(namenodePort, conf, nDatanodes, 
+        ! cannotStartDataNodeFirst(dataNodeFirst) &&  
+        formatNamenode, racks);
+  }
+
+  /**
+   * NameNode should be always started first.
+   * Data-nodes need to handshake with the name-node before they can start.
+   * 
+   * @param dataNodeFirst should the datanode be brought up before the namenode?
+   * @return false if dataNodeFirst is false
+   * @throws IOException if dataNodeFirst is true
+   * 
+   * @deprecated should be removed when dataNodeFirst is gone.
+   */
+  private static boolean cannotStartDataNodeFirst( boolean dataNodeFirst 
+                                                  ) throws IOException {
+    if( dataNodeFirst )
+      throw new IOException( "NameNode should be always started first." );
+    return false;
+  }
 
+  /**
+   * Create the config and start up the servers.  If either the rpc or info port is already 
+   * in use, we will try new ports.
+   * @param namenodePort suggestion for which rpc port to use.  caller should use 
+   *                     getNameNodePort() to get the actual port used.
+   * @param nDatanodes Number of datanodes   
+   * @param formatNamenode should the namenode be formatted before starting up ?
+   * @param racks array of strings indicating racks that each datanode is on
+   */
+  public MiniDFSCluster(int namenodePort, 
+                        Configuration conf,
+                        int nDatanodes,
+                        boolean formatNamenode,
+                        String[] racks) throws IOException {
     this.conf = conf;
     
     this.nDatanodes = nDatanodes;
@@ -279,18 +316,16 @@ public class MiniDFSCluster {
     this.conf.setInt("dfs.safemode.extension", 0);
 
     // Create the NameNode
-    if (formatNamenode) { NameNode.format(conf); }
+    StartupOption startOpt = 
+      formatNamenode ? StartupOption.FORMAT : StartupOption.REGULAR;
+    conf.setObject( "dfs.namenode.startup", startOpt );
+    conf.setObject( "dfs.datanode.startup", startOpt );
     nameNode = new NameNodeRunner();
     nameNodeThread = new Thread(nameNode);
 
     //
     // Start the MiniDFSCluster
     //
-    
-    if (dataNodeFirst) {
-      startDataNodes(conf, racks, data_dir);
-    }
-    
     // Start the namenode and wait for it to be initialized
     nameNodeThread.start();
     while (!nameNode.isCrashed() && !nameNode.isInitialized()) {
@@ -310,9 +345,7 @@ public class MiniDFSCluster {
     this.conf.set("fs.default.name", nnAddr.getHostName()+ ":" + Integer.toString(nameNodePort));
     
     // Start the datanodes
-    if (!dataNodeFirst) {
-      startDataNodes(conf, racks, data_dir);
-    }
+    startDataNodes(conf, racks, data_dir);
     
     while (!nameNode.isCrashed() && !nameNode.isUp()) {
       try {                                     // let daemons get started

+ 119 - 0
src/test/org/apache/hadoop/dfs/TestDFSFinalize.java

@@ -0,0 +1,119 @@
+/**
+* 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.dfs;
+
+import java.io.File;
+import java.io.IOException;
+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.dfs.FSConstants.NodeType;
+import static org.apache.hadoop.dfs.FSConstants.NodeType.NAME_NODE;
+import static org.apache.hadoop.dfs.FSConstants.NodeType.DATA_NODE;
+import org.apache.hadoop.dfs.FSConstants.StartupOption;
+import org.apache.hadoop.fs.Path;
+
+/**
+ * This test ensures the appropriate response from the system when 
+ * the system is finalized.
+ *
+ * @author Nigel Daley
+ */
+public class TestDFSFinalize extends TestCase {
+ 
+  private static final Log LOG = LogFactory.getLog(
+    "org.apache.hadoop.dfs.TestDFSFinalize");
+  Configuration conf;
+  private int testCounter = 0;
+  
+  /**
+   * Writes an INFO log message containing the parameters.
+   */
+  void log(String label, int numDirs) {
+    LOG.info("============================================================");
+    LOG.info("***TEST " + (testCounter++) + "*** " 
+      + label + ":"
+      + " numDirs="+numDirs);
+  }
+  
+  /**
+   * Verify that the current directory exists and that the previous directory
+   * does not exist.  Verify that current hasn't been modified by comparing 
+   * the checksum of all it's containing files with their original checksum.
+   * Note that we do not check that previous is removed on the DataNode
+   * because its removal is asynchronous therefore we have no reliable
+   * way to know when it will happen.  
+   */
+  void checkResult(String[] nameNodeDirs, String[] dataNodeDirs) throws IOException {
+    for (int i = 0; i < nameNodeDirs.length; i++) {
+      assertTrue(new File(nameNodeDirs[i],"current").isDirectory());
+      assertTrue(new File(nameNodeDirs[i],"current/VERSION").isFile());
+      assertTrue(new File(nameNodeDirs[i],"current/edits").isFile());
+      assertTrue(new File(nameNodeDirs[i],"current/fsimage").isFile());
+      assertTrue(new File(nameNodeDirs[i],"current/fstime").isFile());
+    }
+    for (int i = 0; i < dataNodeDirs.length; i++) {
+      assertEquals(
+        UpgradeUtilities.checksumContents(
+          DATA_NODE, new File(dataNodeDirs[i],"current")),
+        UpgradeUtilities.checksumMasterContents(DATA_NODE));
+    }
+    for (int i = 0; i < nameNodeDirs.length; i++) {
+      assertFalse(new File(nameNodeDirs[i],"previous").isDirectory());
+    }
+  }
+ 
+  /**
+   * This test attempts to finalize the NameNode and DataNode.
+   */
+  public void testFinalize() throws Exception {
+    File[] baseDirs;
+    UpgradeUtilities.initialize();
+    
+    for (int numDirs = 1; numDirs <= 2; numDirs++) {
+      conf = UpgradeUtilities.initializeStorageStateConf(numDirs);
+      String[] nameNodeDirs = conf.getStrings("dfs.name.dir");
+      String[] dataNodeDirs = conf.getStrings("dfs.data.dir");
+      
+      log("Finalize with existing previous dir",numDirs);
+      UpgradeUtilities.createStorageDirs(NAME_NODE, nameNodeDirs, "current");
+      UpgradeUtilities.createStorageDirs(NAME_NODE, nameNodeDirs, "previous");
+      UpgradeUtilities.startCluster(NAME_NODE,StartupOption.REGULAR,conf);
+      UpgradeUtilities.createStorageDirs(DATA_NODE, dataNodeDirs, "current");
+      UpgradeUtilities.createStorageDirs(DATA_NODE, dataNodeDirs, "previous");
+      UpgradeUtilities.startCluster(DATA_NODE,StartupOption.REGULAR,conf);
+      UpgradeUtilities.finalizeCluster(conf);
+      checkResult(nameNodeDirs, dataNodeDirs);
+      
+      log("Finalize without existing previous dir",numDirs);
+      UpgradeUtilities.finalizeCluster(conf);
+      checkResult(nameNodeDirs, dataNodeDirs);
+      UpgradeUtilities.stopCluster(null);
+      UpgradeUtilities.createEmptyDirs(nameNodeDirs);
+      UpgradeUtilities.createEmptyDirs(dataNodeDirs);
+    } // end numDir loop
+  }
+ 
+  public static void main(String[] args) throws Exception {
+    new TestDFSFinalize().testFinalize();
+  }
+  
+}
+
+

+ 232 - 0
src/test/org/apache/hadoop/dfs/TestDFSRollback.java

@@ -0,0 +1,232 @@
+/**
+* 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.dfs;
+
+import java.io.File;
+import java.io.IOException;
+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.dfs.FSConstants.NodeType;
+import static org.apache.hadoop.dfs.FSConstants.NodeType.NAME_NODE;
+import static org.apache.hadoop.dfs.FSConstants.NodeType.DATA_NODE;
+import org.apache.hadoop.dfs.FSConstants.StartupOption;
+import org.apache.hadoop.fs.Path;
+
+/**
+* This test ensures the appropriate response (successful or failure) from
+* the system when the system is rolled back under various storage state and
+* version conditions.
+*
+* @author Nigel Daley
+*/
+public class TestDFSRollback extends TestCase {
+ 
+  private static final Log LOG = LogFactory.getLog(
+    "org.apache.hadoop.dfs.TestDFSRollback");
+  Configuration conf;
+  private int testCounter = 0;
+  
+  /**
+   * Writes an INFO log message containing the parameters.
+   */
+  void log(String label, int numDirs) {
+    LOG.info("============================================================");
+    LOG.info("***TEST " + (testCounter++) + "*** " 
+      + label + ":"
+      + " numDirs="+numDirs);
+  }
+  
+  /**
+   * Verify that the new current directory is the old previous.  
+   * It is assumed that the server has recovered and rolled back.
+   */
+  void checkResult(NodeType nodeType, String[] baseDirs) throws IOException {
+    switch (nodeType) {
+      case NAME_NODE:
+        for (int i = 0; i < baseDirs.length; i++) {
+          assertTrue(new File(baseDirs[i],"current").isDirectory());
+          assertTrue(new File(baseDirs[i],"current/VERSION").isFile());
+          assertTrue(new File(baseDirs[i],"current/edits").isFile());
+          assertTrue(new File(baseDirs[i],"current/fsimage").isFile());
+          assertTrue(new File(baseDirs[i],"current/fstime").isFile());
+        }
+        break;
+      case DATA_NODE:
+        for (int i = 0; i < baseDirs.length; i++) {
+          assertEquals(
+            UpgradeUtilities.checksumContents(
+              nodeType, new File(baseDirs[i],"current")),
+            UpgradeUtilities.checksumMasterContents(nodeType));
+        }
+        break;
+    }
+    for (int i = 0; i < baseDirs.length; i++) {
+      assertFalse(new File(baseDirs[i],"previous").isDirectory());
+    }
+  }
+ 
+  /**
+   * Starts the given nodeType with the given operation.  The remaining 
+   * parameters are used to verify the expected result.
+   * 
+   * @param nodeType must not be null
+   */
+  void runTest(NodeType nodeType, StartupOption operation, boolean shouldStart) 
+    throws Exception 
+  {
+    if (shouldStart) {
+      UpgradeUtilities.startCluster(nodeType, operation, conf);
+      UpgradeUtilities.stopCluster(nodeType);
+    } else {
+      try {
+        UpgradeUtilities.startCluster(nodeType, operation, conf); // should fail
+        throw new AssertionError("Cluster should have failed to start");
+      } catch (Exception expected) {
+        // expected
+        //expected.printStackTrace();
+        assertFalse(UpgradeUtilities.isNodeRunning(nodeType));
+      } finally {
+        UpgradeUtilities.stopCluster(nodeType);
+      }
+    }
+  }
+ 
+  /**
+   * This test attempts to rollback the NameNode and DataNode under
+   * a number of valid and invalid conditions.
+   */
+  public void testRollback() throws Exception {
+    File[] baseDirs;
+    UpgradeUtilities.initialize();
+    
+    for (int numDirs = 1; numDirs <= 2; numDirs++) {
+      conf = UpgradeUtilities.initializeStorageStateConf(numDirs);
+      String[] nameNodeDirs = conf.getStrings("dfs.name.dir");
+      String[] dataNodeDirs = conf.getStrings("dfs.data.dir");
+      
+      log("Normal NameNode rollback",numDirs);
+      UpgradeUtilities.createStorageDirs(NAME_NODE, nameNodeDirs, "current");
+      UpgradeUtilities.createStorageDirs(NAME_NODE, nameNodeDirs, "previous");
+      runTest(NAME_NODE, StartupOption.ROLLBACK, true);
+      checkResult(NAME_NODE, nameNodeDirs);
+      UpgradeUtilities.createEmptyDirs(nameNodeDirs);
+      
+      log("Normal DataNode rollback",numDirs);
+      UpgradeUtilities.createStorageDirs(NAME_NODE, nameNodeDirs, "current");
+      UpgradeUtilities.createStorageDirs(NAME_NODE, nameNodeDirs, "previous");
+      UpgradeUtilities.startCluster(NAME_NODE,StartupOption.ROLLBACK,conf);
+      UpgradeUtilities.createStorageDirs(DATA_NODE, dataNodeDirs, "current");
+      UpgradeUtilities.createStorageDirs(DATA_NODE, dataNodeDirs, "previous");
+      runTest(DATA_NODE, StartupOption.ROLLBACK, true);
+      checkResult(DATA_NODE, dataNodeDirs);
+      UpgradeUtilities.stopCluster(null);
+      UpgradeUtilities.createEmptyDirs(nameNodeDirs);
+      UpgradeUtilities.createEmptyDirs(dataNodeDirs);
+
+      log("NameNode rollback without existing previous dir",numDirs);
+      UpgradeUtilities.createStorageDirs(NAME_NODE, nameNodeDirs, "current");
+      runTest(NAME_NODE, StartupOption.ROLLBACK, false);
+      UpgradeUtilities.createEmptyDirs(nameNodeDirs);
+      
+      log("DataNode rollback without existing previous dir",numDirs);
+      UpgradeUtilities.createStorageDirs(NAME_NODE, nameNodeDirs, "current");
+      UpgradeUtilities.startCluster(NAME_NODE,StartupOption.UPGRADE,conf);
+      UpgradeUtilities.createStorageDirs(DATA_NODE, dataNodeDirs, "current");
+      runTest(DATA_NODE, StartupOption.ROLLBACK, true);
+      UpgradeUtilities.stopCluster(null);
+      UpgradeUtilities.createEmptyDirs(nameNodeDirs);
+      UpgradeUtilities.createEmptyDirs(dataNodeDirs);
+
+      log("DataNode rollback with future stored layout version in previous",numDirs);
+      UpgradeUtilities.createStorageDirs(NAME_NODE, nameNodeDirs, "current");
+      UpgradeUtilities.createStorageDirs(NAME_NODE, nameNodeDirs, "previous");
+      UpgradeUtilities.startCluster(NAME_NODE,StartupOption.ROLLBACK,conf);
+      UpgradeUtilities.createStorageDirs(DATA_NODE, dataNodeDirs, "current");
+      baseDirs = UpgradeUtilities.createStorageDirs(DATA_NODE, dataNodeDirs, "previous");
+      UpgradeUtilities.createVersionFile(DATA_NODE,baseDirs,
+        new StorageInfo(Integer.MIN_VALUE,
+                        UpgradeUtilities.getCurrentNamespaceID(),
+                        UpgradeUtilities.getCurrentFsscTime()));
+      runTest(DATA_NODE, StartupOption.ROLLBACK, false);
+      UpgradeUtilities.stopCluster(null);
+      UpgradeUtilities.createEmptyDirs(nameNodeDirs);
+      UpgradeUtilities.createEmptyDirs(dataNodeDirs);
+      
+      log("DataNode rollback with newer fsscTime in previous",numDirs);
+      UpgradeUtilities.createStorageDirs(NAME_NODE, nameNodeDirs, "current");
+      UpgradeUtilities.createStorageDirs(NAME_NODE, nameNodeDirs, "previous");
+      UpgradeUtilities.startCluster(NAME_NODE,StartupOption.ROLLBACK,conf);
+      UpgradeUtilities.createStorageDirs(DATA_NODE, dataNodeDirs, "current");
+      baseDirs = UpgradeUtilities.createStorageDirs(DATA_NODE, dataNodeDirs, "previous");
+      UpgradeUtilities.createVersionFile(DATA_NODE,baseDirs,
+        new StorageInfo(UpgradeUtilities.getCurrentLayoutVersion(),
+                        UpgradeUtilities.getCurrentNamespaceID(),
+                        Long.MAX_VALUE));
+      runTest(DATA_NODE, StartupOption.ROLLBACK, false);
+      UpgradeUtilities.stopCluster(null);
+      UpgradeUtilities.createEmptyDirs(nameNodeDirs);
+      UpgradeUtilities.createEmptyDirs(dataNodeDirs);
+
+      log("NameNode rollback with no edits file",numDirs);
+      UpgradeUtilities.createStorageDirs(NAME_NODE, nameNodeDirs, "current");
+      baseDirs = UpgradeUtilities.createStorageDirs(NAME_NODE, nameNodeDirs, "previous");
+      for (File f : baseDirs) { 
+        UpgradeUtilities.remove(new File(f,"edits"));
+      }
+      runTest(NAME_NODE, StartupOption.ROLLBACK, false);
+      UpgradeUtilities.createEmptyDirs(nameNodeDirs);
+      
+      log("NameNode rollback with no image file",numDirs);
+      UpgradeUtilities.createStorageDirs(NAME_NODE, nameNodeDirs, "current");
+      baseDirs = UpgradeUtilities.createStorageDirs(NAME_NODE, nameNodeDirs, "previous");
+      for (File f : baseDirs) { 
+        UpgradeUtilities.remove(new File(f,"fsimage")); 
+      }
+      runTest(NAME_NODE, StartupOption.ROLLBACK, false);
+      UpgradeUtilities.createEmptyDirs(nameNodeDirs);
+      
+      log("NameNode rollback with corrupt version file",numDirs);
+      UpgradeUtilities.createStorageDirs(NAME_NODE, nameNodeDirs, "current");
+      baseDirs = UpgradeUtilities.createStorageDirs(NAME_NODE, nameNodeDirs, "previous");
+      for (File f : baseDirs) { 
+        UpgradeUtilities.corruptFile(new File(f,"VERSION")); 
+      }
+      runTest(NAME_NODE, StartupOption.ROLLBACK, false);
+      UpgradeUtilities.createEmptyDirs(nameNodeDirs);
+      
+      log("NameNode rollback with old layout version in previous",numDirs);
+      UpgradeUtilities.createStorageDirs(NAME_NODE, nameNodeDirs, "current");
+      baseDirs = UpgradeUtilities.createStorageDirs(NAME_NODE, nameNodeDirs, "previous");
+      UpgradeUtilities.createVersionFile(NAME_NODE,baseDirs,
+        new StorageInfo(1,
+                        UpgradeUtilities.getCurrentNamespaceID(),
+                        UpgradeUtilities.getCurrentFsscTime()));
+      runTest(NAME_NODE, StartupOption.UPGRADE, false);
+      UpgradeUtilities.createEmptyDirs(nameNodeDirs);
+    } // end numDir loop
+  }
+ 
+  public static void main(String[] args) throws Exception {
+    new TestDFSRollback().testRollback();
+  }
+  
+}
+
+

+ 206 - 0
src/test/org/apache/hadoop/dfs/TestDFSStartupVersions.java

@@ -0,0 +1,206 @@
+/**
+ * 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.dfs;
+
+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.dfs.FSConstants.NodeType;
+import static org.apache.hadoop.dfs.FSConstants.NodeType.NAME_NODE;
+import static org.apache.hadoop.dfs.FSConstants.NodeType.DATA_NODE;
+import org.apache.hadoop.dfs.FSConstants.StartupOption;
+import org.apache.hadoop.fs.Path;
+
+/**
+ * This test ensures the appropriate response (successful or failure) from 
+ * a Datanode when the system is started with differing version combinations. 
+ * 
+ * @author Nigel Daley
+ */
+public class TestDFSStartupVersions extends TestCase {
+  
+  private static final Log LOG = LogFactory.getLog(
+    "org.apache.hadoop.dfs.TestDFSStartupVersions");
+  private static Path TEST_ROOT_DIR = new Path(
+    System.getProperty("test.build.data","/tmp").toString().replace(' ', '+'));
+  
+  /**
+   * Writes an INFO log message containing the parameters.
+   */
+  void log(String label, NodeType nodeType, Integer testCase, StorageInfo version) {
+    String testCaseLine = "";
+    if (testCase != null) {
+      testCaseLine = " testCase="+testCase;
+    }
+    LOG.info("============================================================");
+    LOG.info("***TEST*** " + label + ":"
+      + testCaseLine
+      + " nodeType="+nodeType
+      + " layoutVersion="+version.getLayoutVersion()
+      + " namespaceID="+version.getNamespaceID()
+      + " fsscTime="+version.getCTime());
+  }
+  
+  /**
+   * Initialize the versions array.  This array stores all combinations 
+   * of cross product:
+   *  {oldLayoutVersion,currentLayoutVersion,futureLayoutVersion} X
+   *    {currentNamespaceId,incorrectNamespaceId} X
+   *      {pastFsscTime,currentFsscTime,futureFsscTime}
+   */
+  private StorageInfo[] initializeVersions() throws Exception {
+    int layoutVersionOld = -3;
+    int layoutVersionCur = UpgradeUtilities.getCurrentLayoutVersion();
+    int layoutVersionNew = Integer.MIN_VALUE;
+    int namespaceIdCur = UpgradeUtilities.getCurrentNamespaceID();
+    int namespaceIdOld = Integer.MIN_VALUE;
+    long fsscTimeOld = Long.MIN_VALUE;
+    long fsscTimeCur = UpgradeUtilities.getCurrentFsscTime();
+    long fsscTimeNew = Long.MAX_VALUE;
+    
+    return new StorageInfo[] {
+      new StorageInfo(layoutVersionOld, namespaceIdCur, fsscTimeOld), // 0
+      new StorageInfo(layoutVersionOld, namespaceIdCur, fsscTimeCur), // 1
+      new StorageInfo(layoutVersionOld, namespaceIdCur, fsscTimeNew), // 2
+      new StorageInfo(layoutVersionOld, namespaceIdOld, fsscTimeOld), // 3
+      new StorageInfo(layoutVersionOld, namespaceIdOld, fsscTimeCur), // 4
+      new StorageInfo(layoutVersionOld, namespaceIdOld, fsscTimeNew), // 5
+      new StorageInfo(layoutVersionCur, namespaceIdCur, fsscTimeOld), // 6
+      new StorageInfo(layoutVersionCur, namespaceIdCur, fsscTimeCur), // 7
+      new StorageInfo(layoutVersionCur, namespaceIdCur, fsscTimeNew), // 8
+      new StorageInfo(layoutVersionCur, namespaceIdOld, fsscTimeOld), // 9
+      new StorageInfo(layoutVersionCur, namespaceIdOld, fsscTimeCur), // 10
+      new StorageInfo(layoutVersionCur, namespaceIdOld, fsscTimeNew), // 11
+      new StorageInfo(layoutVersionNew, namespaceIdCur, fsscTimeOld), // 12
+      new StorageInfo(layoutVersionNew, namespaceIdCur, fsscTimeCur), // 13
+      new StorageInfo(layoutVersionNew, namespaceIdCur, fsscTimeNew), // 14
+      new StorageInfo(layoutVersionNew, namespaceIdOld, fsscTimeOld), // 15
+      new StorageInfo(layoutVersionNew, namespaceIdOld, fsscTimeCur), // 16
+      new StorageInfo(layoutVersionNew, namespaceIdOld, fsscTimeNew), // 17
+    };
+  }
+  
+  /**
+   * Determines if the given Namenode version and Datanode version
+   * are compatible with each other. Compatibility in this case mean
+   * that the Namenode and Datanode will successfully start up and
+   * will work together. The rules for compatibility,
+   * taken from the DFS Upgrade Design, are as follows:
+   * <pre>
+   * 1. The data-node does regular startup (no matter which options 
+   *    it is started with) if
+   *       softwareLV == storedLV AND 
+   *       DataNode.FSSCTime == NameNode.FSSCTime
+   * 2. The data-node performs an upgrade if it is started without any 
+   *    options and
+   *       |softwareLV| > |storedLV| OR 
+   *       (softwareLV == storedLV AND
+   *        DataNode.FSSCTime < NameNode.FSSCTime)
+   * 3. NOT TESTED: The data-node rolls back if it is started with
+   *    the -rollback option and
+   *       |softwareLV| >= |previous.storedLV| AND 
+   *       DataNode.previous.FSSCTime <= NameNode.FSSCTime
+   * 4. In all other cases the startup fails.
+   * </pre>
+   */
+  boolean isVersionCompatible(StorageInfo namenodeVer, StorageInfo datanodeVer) {
+    // check #0
+    if (namenodeVer.getNamespaceID() != datanodeVer.getNamespaceID()) {
+      LOG.info("namespaceIDs are not equal: isVersionCompatible=false");
+      return false;
+    }
+    // check #1
+    int softwareLV = FSConstants.LAYOUT_VERSION;  // will also be Namenode's LV
+    int storedLV = datanodeVer.getLayoutVersion();
+    if (softwareLV == storedLV &&  
+        datanodeVer.getCTime() == namenodeVer.getCTime()) 
+    {
+      LOG.info("layoutVersions and cTimes are equal: isVersionCompatible=true");
+      return true;
+    }
+    // check #2
+    long absSoftwareLV = Math.abs((long)softwareLV);
+    long absStoredLV = Math.abs((long)storedLV);
+    if (absSoftwareLV > absStoredLV ||
+        (softwareLV == storedLV &&
+         datanodeVer.getCTime() < namenodeVer.getCTime())) 
+    {
+      LOG.info("softwareLayoutVersion is newer OR namenode cTime is newer: isVersionCompatible=true");
+      return true;
+    }
+    // check #4
+    LOG.info("default case: isVersionCompatible=false");
+    return false;
+  }
+  
+  /**
+   * This test ensures the appropriate response (successful or failure) from 
+   * a Datanode when the system is started with differing version combinations. 
+   * <pre>
+   * For each 3-tuple in the cross product
+   *   ({oldLayoutVersion,currentLayoutVersion,futureLayoutVersion},
+   *    {currentNamespaceId,incorrectNamespaceId},
+   *    {pastFsscTime,currentFsscTime,futureFsscTime})
+   *      1. Startup Namenode with version file containing 
+   *         (currentLayoutVersion,currentNamespaceId,currentFsscTime)
+   *      2. Attempt to startup Datanode with version file containing 
+   *         this iterations version 3-tuple
+   * </pre>
+   */
+  public void testVersions() throws Exception {
+    UpgradeUtilities.initialize();
+    Configuration conf = UpgradeUtilities.initializeStorageStateConf(1);
+    StorageInfo[] versions = initializeVersions();
+    UpgradeUtilities.createStorageDirs(
+      NAME_NODE, conf.getStrings("dfs.name.dir"), "current");
+    UpgradeUtilities.startCluster(NAME_NODE,StartupOption.REGULAR,conf);
+    StorageInfo nameNodeVersion = new StorageInfo(
+      UpgradeUtilities.getCurrentLayoutVersion(),
+      UpgradeUtilities.getCurrentNamespaceID(),
+      UpgradeUtilities.getCurrentFsscTime());
+    log("NameNode version info",NAME_NODE,null,nameNodeVersion);
+    try {
+      for (int i = 0; i < versions.length; i++) {
+        File[] storage = UpgradeUtilities.createStorageDirs(
+          DATA_NODE, conf.getStrings("dfs.data.dir"), "current");
+        log("DataNode version info",DATA_NODE,i,versions[i]);
+        UpgradeUtilities.createVersionFile(DATA_NODE, storage, versions[i]);
+        try {
+          UpgradeUtilities.startCluster(DATA_NODE,StartupOption.REGULAR,conf);
+        } catch (Exception ignore) {
+          // Ignore.  The asserts below will check for problems.
+          // ignore.printStackTrace();
+        }
+        assertTrue(UpgradeUtilities.isNodeRunning(NAME_NODE));
+        assertEquals(isVersionCompatible(nameNodeVersion, versions[i]),
+          UpgradeUtilities.isNodeRunning(DATA_NODE));
+        UpgradeUtilities.stopCluster(DATA_NODE);
+      }
+    } finally {
+      UpgradeUtilities.stopCluster(null);
+    }
+  }
+
+  public static void main(String[] args) throws Exception {
+    new TestDFSStartupVersions().testVersions();
+  }
+  
+}
+

+ 245 - 0
src/test/org/apache/hadoop/dfs/TestDFSStorageStateRecovery.java

@@ -0,0 +1,245 @@
+/**
+* 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.dfs;
+
+import java.io.File;
+import java.io.IOException;
+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.dfs.FSConstants.NodeType;
+import static org.apache.hadoop.dfs.FSConstants.NodeType.NAME_NODE;
+import static org.apache.hadoop.dfs.FSConstants.NodeType.DATA_NODE;
+import org.apache.hadoop.dfs.FSConstants.StartupOption;
+import org.apache.hadoop.fs.Path;
+
+/**
+* This test ensures the appropriate response (successful or failure) from
+* the system when the system is started under various storage state and
+* version conditions.
+*
+* @author Nigel Daley
+*/
+public class TestDFSStorageStateRecovery extends TestCase {
+ 
+  private static final Log LOG = LogFactory.getLog(
+    "org.apache.hadoop.dfs.TestDFSStorageStateRecovery");
+  Configuration conf;
+  private int testCounter = 0;
+  
+  /**
+   * The test case table.  Each row represents a test case.  This table is
+   * taken from the table in Apendix A of the HDFS Upgrade Test Plan
+   * (TestPlan-HdfsUpgrade.html) attached to
+   * http://issues.apache.org/jira/browse/HADOOP-702
+   * The column meanings are:
+   *  0) current directory exists
+   *  1) previous directory exists
+   *  2) previous.tmp directory exists
+   *  3) removed.tmp directory exists
+   *  4) node should recover and startup
+   *  5) current directory should exist after recovery but before startup
+   *  6) previous directory should exist after recovery but before startup
+   */
+  static boolean[][] testCases = new boolean[][] {
+    new boolean[] {true,  false, false, false, true,  true,  false}, // 1
+    new boolean[] {true,  true,  false, false, true,  true,  true }, // 2
+    new boolean[] {true,  false, true,  false, true,  true,  true }, // 3
+    new boolean[] {true,  true,  true,  true,  false, false, false }, // 4
+    new boolean[] {true,  true,  true,  false, false, false, false }, // 4
+    new boolean[] {false, true,  true,  true,  false, false, false }, // 4
+    new boolean[] {false, true,  true,  false, false, false, false }, // 4
+    new boolean[] {false, false, false, false, false, false, false }, // 5
+    new boolean[] {false, true,  false, false, false, false, false }, // 6
+    new boolean[] {false, false, true,  false, true,  true,  false}, // 7
+    new boolean[] {true,  false, false, true,  true,  true,  false}, // 8
+    new boolean[] {true,  true,  false, true,  false, false, false }, // 9
+    new boolean[] {true,  true,  true,  true,  false, false, false }, // 10
+    new boolean[] {true,  false, true,  true,  false, false, false }, // 10
+    new boolean[] {false, true,  true,  true,  false, false, false }, // 10
+    new boolean[] {false, false, true,  true,  false, false, false }, // 10
+    new boolean[] {false, false, false, true,  false, false, false }, // 11
+    new boolean[] {false, true,  false, true,  true,  true,  true }, // 12
+  };
+  
+  /**
+   * Writes an INFO log message containing the parameters. Only
+   * the first 4 elements of the state array are included in the message.
+   */
+  void log(String label, int numDirs, int testCaseNum, boolean[] state) {
+    LOG.info("============================================================");
+    LOG.info("***TEST " + (testCounter++) + "*** " 
+      + label + ":"
+      + " numDirs="+numDirs
+      + " testCase="+testCaseNum
+      + " current="+state[0]
+      + " previous="+state[1]
+      + " previous.tmp="+state[2]
+      + " removed.tmp="+state[3]);
+  }
+  
+  /**
+   * Sets up the storage directories for the given node type, either
+   * dfs.name.dir or dfs.data.dir. For each element in dfs.name.dir or
+   * dfs.data.dir, the subdirectories represented by the first four elements 
+   * of the <code>state</code> array will be created and populated.
+   * See UpgradeUtilities.createStorageDirs().
+   * 
+   * @param nodeType
+   *   the type of node that storage should be created for. Based on this
+   *   parameter either dfs.name.dir or dfs.data.dir is used from the global conf.
+   * @param state
+   *   a row from the testCases table which indicates which directories
+   *   to setup for the node
+   * @return file paths representing either dfs.name.dir or dfs.data.dir
+   *   directories
+   */
+  String[] createStorageState(NodeType nodeType, boolean[] state) throws Exception {
+    String[] baseDirs = (nodeType == NAME_NODE ?
+      conf.getStrings("dfs.name.dir") :
+      conf.getStrings("dfs.data.dir"));
+    UpgradeUtilities.createEmptyDirs(baseDirs);
+    if (state[0])  // current
+      UpgradeUtilities.createStorageDirs(nodeType, baseDirs, "current");
+    if (state[1])  // previous
+      UpgradeUtilities.createStorageDirs(nodeType, baseDirs, "previous");
+    if (state[2])  // previous.tmp
+      UpgradeUtilities.createStorageDirs(nodeType, baseDirs, "previous.tmp");
+    if (state[3])  // removed.tmp
+      UpgradeUtilities.createStorageDirs(nodeType, baseDirs, "removed.tmp");
+    return baseDirs;
+  }
+ 
+  /**
+   * Verify that the current and/or previous exist as indicated by 
+   * the method parameters.  If previous exists, verify that
+   * it hasn't been modified by comparing the checksum of all it's
+   * containing files with their original checksum.  It is assumed that
+   * the server has recovered.
+   */
+  void checkResult(NodeType nodeType, String[] baseDirs, 
+                   boolean currentShouldExist, boolean previousShouldExist) 
+                     throws IOException
+  {
+    switch (nodeType) {
+      case NAME_NODE:
+        if (currentShouldExist) {
+          for (int i = 0; i < baseDirs.length; i++) {
+            assertTrue(new File(baseDirs[i],"current").isDirectory());
+            assertTrue(new File(baseDirs[i],"current/VERSION").isFile());
+            assertTrue(new File(baseDirs[i],"current/edits").isFile());
+            assertTrue(new File(baseDirs[i],"current/fsimage").isFile());
+            assertTrue(new File(baseDirs[i],"current/fstime").isFile());
+          }
+        }
+        break;
+      case DATA_NODE:
+        if (currentShouldExist) {
+          for (int i = 0; i < baseDirs.length; i++) {
+            assertEquals(
+              UpgradeUtilities.checksumContents(
+                nodeType, new File(baseDirs[i],"current")),
+              UpgradeUtilities.checksumMasterContents(nodeType));
+          }
+        }
+        break;
+    }
+    if (previousShouldExist) {
+      for (int i = 0; i < baseDirs.length; i++) {
+        assertTrue(new File(baseDirs[i],"previous").isDirectory());
+        assertEquals(
+          UpgradeUtilities.checksumContents(
+            nodeType, new File(baseDirs[i],"previous")),
+          UpgradeUtilities.checksumMasterContents(nodeType));
+      }
+    }
+  }
+ 
+  /**
+   * Does a regular start of the given nodeType.
+   * 
+   * @param nodeType must not be null
+   * @param indicates whether or not the node should start
+   */
+  void runTest(NodeType nodeType, boolean shouldStart) throws Exception {
+    if (shouldStart) {
+      UpgradeUtilities.startCluster(nodeType, StartupOption.REGULAR, conf);
+      UpgradeUtilities.stopCluster(nodeType);
+    } else {
+      try {
+        UpgradeUtilities.startCluster(nodeType, StartupOption.REGULAR, conf); // should fail
+        throw new AssertionError("Cluster should have failed to start");
+      } catch (Exception expected) {
+        // expected
+        //expected.printStackTrace();
+        assertFalse(UpgradeUtilities.isNodeRunning(nodeType));
+      } finally {
+        UpgradeUtilities.stopCluster(nodeType);
+      }
+    }
+  }
+ 
+  /**
+   * This test iterates over the testCases table and attempts
+   * to startup the NameNode and DataNode normally.
+   */
+  public void testStorageStates() throws Exception {
+    String[] baseDirs;
+    UpgradeUtilities.initialize();
+
+    for (int numDirs = 1; numDirs <= 2; numDirs++) {
+      conf = UpgradeUtilities.initializeStorageStateConf(numDirs);
+      for (int i = 0; i < testCases.length; i++) {
+        boolean[] testCase = testCases[i];
+        boolean shouldRecover = testCase[4];
+        boolean curAfterRecover = testCase[5];
+        boolean prevAfterRecover = testCase[6];
+
+        log("NAME_NODE recovery",numDirs,i,testCase);
+        baseDirs = createStorageState(NAME_NODE, testCase);
+        runTest(NAME_NODE, shouldRecover);
+        if (shouldRecover) {
+          checkResult(NAME_NODE, baseDirs, curAfterRecover, prevAfterRecover);
+        }
+        
+        log("DATA_NODE recovery",numDirs,i,testCase);
+        createStorageState(NAME_NODE, new boolean[] {true,true,false,false});
+        UpgradeUtilities.startCluster(NAME_NODE,StartupOption.REGULAR,conf);
+        baseDirs = createStorageState(DATA_NODE, testCase);
+        if (!testCase[0] && !testCase[1] && !testCase[2] && !testCase[3]) {
+          // DataNode will create and format current if no directories exist
+          runTest(DATA_NODE, true);
+        } else {
+          runTest(DATA_NODE, shouldRecover);
+          if (shouldRecover) {
+            checkResult(DATA_NODE, baseDirs, curAfterRecover, prevAfterRecover);
+          }
+        }
+        UpgradeUtilities.stopCluster(null);
+      } // end testCases loop
+    } // end numDirs loop
+  }
+ 
+  public static void main(String[] args) throws Exception {
+    new TestDFSStorageStateRecovery().testStorageStates();
+  }
+  
+}
+
+

+ 230 - 0
src/test/org/apache/hadoop/dfs/TestDFSUpgrade.java

@@ -0,0 +1,230 @@
+/**
+* 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.dfs;
+
+import java.io.File;
+import java.io.IOException;
+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.dfs.FSConstants.NodeType;
+import static org.apache.hadoop.dfs.FSConstants.NodeType.NAME_NODE;
+import static org.apache.hadoop.dfs.FSConstants.NodeType.DATA_NODE;
+import org.apache.hadoop.dfs.FSConstants.StartupOption;
+import org.apache.hadoop.fs.Path;
+
+/**
+* This test ensures the appropriate response (successful or failure) from
+* the system when the system is upgraded under various storage state and
+* version conditions.
+*
+* @author Nigel Daley
+*/
+public class TestDFSUpgrade extends TestCase {
+ 
+  private static final Log LOG = LogFactory.getLog(
+    "org.apache.hadoop.dfs.TestDFSUpgrade");
+  Configuration conf;
+  private int testCounter = 0;
+  
+  /**
+   * Writes an INFO log message containing the parameters.
+   */
+  void log(String label, int numDirs) {
+    LOG.info("============================================================");
+    LOG.info("***TEST " + (testCounter++) + "*** " 
+      + label + ":"
+      + " numDirs="+numDirs);
+  }
+  
+  /**
+   * Verify that the current and previous directories exist.  Verify that 
+   * previous hasn't been modified by comparing the checksum of all it's
+   * containing files with their original checksum.  It is assumed that
+   * the server has recovered and upgraded.
+   */
+  void checkResult(NodeType nodeType, String[] baseDirs) throws IOException {
+    switch (nodeType) {
+      case NAME_NODE:
+        for (int i = 0; i < baseDirs.length; i++) {
+          assertTrue(new File(baseDirs[i],"current").isDirectory());
+          assertTrue(new File(baseDirs[i],"current/VERSION").isFile());
+          assertTrue(new File(baseDirs[i],"current/edits").isFile());
+          assertTrue(new File(baseDirs[i],"current/fsimage").isFile());
+          assertTrue(new File(baseDirs[i],"current/fstime").isFile());
+        }
+        break;
+      case DATA_NODE:
+        for (int i = 0; i < baseDirs.length; i++) {
+          assertEquals(
+            UpgradeUtilities.checksumContents(
+              nodeType, new File(baseDirs[i],"current")),
+            UpgradeUtilities.checksumMasterContents(nodeType));
+        }
+        break;
+    }
+    for (int i = 0; i < baseDirs.length; i++) {
+      assertTrue(new File(baseDirs[i],"previous").isDirectory());
+      assertEquals(
+        UpgradeUtilities.checksumContents(
+          nodeType, new File(baseDirs[i],"previous")),
+        UpgradeUtilities.checksumMasterContents(nodeType));
+    }
+  }
+ 
+  /**
+   * Starts the given nodeType with the given operation.  The remaining 
+   * parameters are used to verify the expected result.
+   * 
+   * @param nodeType must not be null
+   */
+  void runTest(NodeType nodeType, StartupOption operation, boolean shouldStart) 
+    throws Exception 
+  {
+    if (shouldStart) {
+      UpgradeUtilities.startCluster(nodeType, operation, conf);
+      UpgradeUtilities.stopCluster(nodeType);
+    } else {
+      try {
+        UpgradeUtilities.startCluster(nodeType, operation, conf); // should fail
+        throw new AssertionError("Cluster should have failed to start");
+      } catch (Exception expected) {
+        // expected
+        //expected.printStackTrace();
+        assertFalse(UpgradeUtilities.isNodeRunning(nodeType));
+      } finally {
+        UpgradeUtilities.stopCluster(nodeType);
+      }
+    }
+  }
+ 
+  /**
+   * This test attempts to upgrade the NameNode and DataNode under
+   * a number of valid and invalid conditions.
+   */
+  public void testUpgrade() throws Exception {
+    File[] baseDirs;
+    UpgradeUtilities.initialize();
+    
+    for (int numDirs = 1; numDirs <= 2; numDirs++) {
+      conf = UpgradeUtilities.initializeStorageStateConf(numDirs);
+      String[] nameNodeDirs = conf.getStrings("dfs.name.dir");
+      String[] dataNodeDirs = conf.getStrings("dfs.data.dir");
+      
+      log("Normal NameNode upgrade",numDirs);
+      UpgradeUtilities.createStorageDirs(NAME_NODE, nameNodeDirs, "current");
+      runTest(NAME_NODE, StartupOption.UPGRADE, true);
+      checkResult(NAME_NODE, nameNodeDirs);
+      UpgradeUtilities.createEmptyDirs(nameNodeDirs);
+      
+      log("Normal DataNode upgrade",numDirs);
+      UpgradeUtilities.createStorageDirs(NAME_NODE, nameNodeDirs, "current");
+      UpgradeUtilities.startCluster(NAME_NODE,StartupOption.UPGRADE,conf);
+      UpgradeUtilities.createStorageDirs(DATA_NODE, dataNodeDirs, "current");
+      runTest(DATA_NODE, StartupOption.REGULAR, true);
+      checkResult(DATA_NODE, dataNodeDirs);
+      UpgradeUtilities.stopCluster(null);
+      UpgradeUtilities.createEmptyDirs(nameNodeDirs);
+      UpgradeUtilities.createEmptyDirs(dataNodeDirs);
+      
+      log("NameNode upgrade with existing previous dir",numDirs);
+      UpgradeUtilities.createStorageDirs(NAME_NODE, nameNodeDirs, "current");
+      UpgradeUtilities.createStorageDirs(NAME_NODE, nameNodeDirs, "previous");
+      runTest(NAME_NODE, StartupOption.UPGRADE, false);
+      UpgradeUtilities.createEmptyDirs(nameNodeDirs);
+      
+      log("DataNode upgrade with existing previous dir",numDirs);
+      UpgradeUtilities.createStorageDirs(NAME_NODE, nameNodeDirs, "current");
+      UpgradeUtilities.startCluster(NAME_NODE,StartupOption.UPGRADE,conf);
+      UpgradeUtilities.createStorageDirs(DATA_NODE, dataNodeDirs, "current");
+      UpgradeUtilities.createStorageDirs(DATA_NODE, dataNodeDirs, "previous");
+      runTest(DATA_NODE, StartupOption.REGULAR, true);
+      checkResult(DATA_NODE, dataNodeDirs);
+      UpgradeUtilities.stopCluster(null);
+      UpgradeUtilities.createEmptyDirs(nameNodeDirs);
+      UpgradeUtilities.createEmptyDirs(dataNodeDirs);
+
+      log("DataNode upgrade with future stored layout version in current",numDirs);
+      UpgradeUtilities.createStorageDirs(NAME_NODE, nameNodeDirs, "current");
+      UpgradeUtilities.startCluster(NAME_NODE,StartupOption.UPGRADE,conf);
+      baseDirs = UpgradeUtilities.createStorageDirs(DATA_NODE, dataNodeDirs, "current");
+      UpgradeUtilities.createVersionFile(DATA_NODE,baseDirs,
+        new StorageInfo(Integer.MIN_VALUE,
+                        UpgradeUtilities.getCurrentNamespaceID(),
+                        UpgradeUtilities.getCurrentFsscTime()));
+      runTest(DATA_NODE, StartupOption.REGULAR, false);
+      UpgradeUtilities.stopCluster(null);
+      UpgradeUtilities.createEmptyDirs(nameNodeDirs);
+      UpgradeUtilities.createEmptyDirs(dataNodeDirs);
+      
+      log("DataNode upgrade with newer fsscTime in current",numDirs);
+      UpgradeUtilities.createStorageDirs(NAME_NODE, nameNodeDirs, "current");
+      UpgradeUtilities.startCluster(NAME_NODE,StartupOption.UPGRADE,conf);
+      baseDirs = UpgradeUtilities.createStorageDirs(DATA_NODE, dataNodeDirs, "current");
+      UpgradeUtilities.createVersionFile(DATA_NODE,baseDirs,
+        new StorageInfo(UpgradeUtilities.getCurrentLayoutVersion(),
+                        UpgradeUtilities.getCurrentNamespaceID(),
+                        Long.MAX_VALUE));
+      runTest(DATA_NODE, StartupOption.REGULAR, false);
+      UpgradeUtilities.stopCluster(null);
+      UpgradeUtilities.createEmptyDirs(nameNodeDirs);
+      UpgradeUtilities.createEmptyDirs(dataNodeDirs);
+
+      log("NameNode upgrade with no edits file",numDirs);
+      baseDirs = UpgradeUtilities.createStorageDirs(NAME_NODE, nameNodeDirs, "current");
+      for (File f : baseDirs) { 
+        UpgradeUtilities.remove(new File(f,"edits"));
+      }
+      runTest(NAME_NODE, StartupOption.UPGRADE, false);
+      UpgradeUtilities.createEmptyDirs(nameNodeDirs);
+      
+      log("NameNode upgrade with no image file",numDirs);
+      baseDirs = UpgradeUtilities.createStorageDirs(NAME_NODE, nameNodeDirs, "current");
+      for (File f : baseDirs) { 
+        UpgradeUtilities.remove(new File(f,"fsimage")); 
+      }
+      runTest(NAME_NODE, StartupOption.UPGRADE, false);
+      UpgradeUtilities.createEmptyDirs(nameNodeDirs);
+      
+      log("NameNode upgrade with corrupt version file",numDirs);
+      baseDirs = UpgradeUtilities.createStorageDirs(NAME_NODE, nameNodeDirs, "current");
+      for (File f : baseDirs) { 
+        UpgradeUtilities.corruptFile(new File(f,"VERSION")); 
+      }
+      runTest(NAME_NODE, StartupOption.UPGRADE, false);
+      UpgradeUtilities.createEmptyDirs(nameNodeDirs);
+      
+      log("NameNode upgrade with future layout version in current",numDirs);
+      baseDirs = UpgradeUtilities.createStorageDirs(NAME_NODE, nameNodeDirs, "current");
+      UpgradeUtilities.createVersionFile(NAME_NODE,baseDirs,
+        new StorageInfo(Integer.MIN_VALUE,
+                        UpgradeUtilities.getCurrentNamespaceID(),
+                        UpgradeUtilities.getCurrentFsscTime()));
+      runTest(NAME_NODE, StartupOption.UPGRADE, false);
+      UpgradeUtilities.createEmptyDirs(nameNodeDirs);
+    } // end numDir loop
+  }
+ 
+  public static void main(String[] args) throws Exception {
+    new TestDFSUpgrade().testUpgrade();
+  }
+  
+}
+
+

+ 1 - 1
src/test/org/apache/hadoop/dfs/TestFileCorruption.java

@@ -52,7 +52,7 @@ public class TestFileCorruption extends TestCase {
       util.createFiles(fs, "/srcdat");
       // Now deliberately remove the blocks
       File data_dir = new File(System.getProperty("test.build.data"),
-          "dfs/data/data5/data");
+          "dfs/data/data5/current");
       assertTrue("data directory does not exist", data_dir.exists());
       File[] blocks = data_dir.listFiles();
       assertTrue("Blocks do not exist in data-dir", (blocks != null) && (blocks.length > 0));

+ 574 - 0
src/test/org/apache/hadoop/dfs/UpgradeUtilities.java

@@ -0,0 +1,574 @@
+/*
+ * UpgradeUtilities.java
+ *
+ * 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.dfs;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.io.RandomAccessFile;
+import java.net.InetSocketAddress;
+import java.util.Random;
+import java.util.zip.CRC32;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.fs.LocalFileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.dfs.FSConstants.NodeType;
+import static org.apache.hadoop.dfs.FSConstants.NodeType.NAME_NODE;
+import static org.apache.hadoop.dfs.FSConstants.NodeType.DATA_NODE;
+import org.apache.hadoop.dfs.FSConstants.StartupOption;
+import org.apache.hadoop.dfs.Storage.StorageDirectory;
+
+/**
+ * This class defines a number of static helper methods used by the
+ * DFS Upgrade unit tests.  By default, a singleton master populated storage
+ * directory is created for a Namenode (contains edits, fsimage,
+ * version, and time files) and a Datanode (contains version and
+ * block files).  The master directories are lazily created.  They are then
+ * copied by the createStorageDirs() method to create new storage
+ * directories of the appropriate type (Namenode or Datanode).
+ * 
+ * @author Nigel Daley
+ */
+public class UpgradeUtilities {
+
+  // The fs.default.name configuration host:port value for the Namenode
+  private static final String NAMENODE_HOST = "localhost:0";
+  // Root scratch directory on local filesystem 
+  private static File TEST_ROOT_DIR = new File(
+    System.getProperty("test.build.data","/tmp").toString().replace(' ', '+'));
+  // The singleton master storage directory for Namenode
+  private static File namenodeStorage = new File(TEST_ROOT_DIR, "namenodeMaster");
+  // A checksum of the contents in namenodeStorage directory
+  private static long namenodeStorageChecksum;
+  // The namespaceId of the namenodeStorage directory
+  private static int namenodeStorageNamespaceID;
+  // The fsscTime of the namenodeStorage directory
+  private static long namenodeStorageFsscTime;
+  // The singleton master storage directory for Datanode
+  private static File datanodeStorage = new File(TEST_ROOT_DIR, "datanodeMaster");
+  // A checksum of the contents in datanodeStorage directory
+  private static long datanodeStorageChecksum;
+  // The NameNode started by this Utility class
+  private static NameNode namenode = null;
+  // The DataNode started by this Utility class
+  private static DataNode datanode = null;
+  
+  /**
+   * Initialize the data structures used by this class.  
+   * IMPORTANT NOTE: This method must be called once before calling 
+   *                 any other public method on this class.  
+   */
+  public static void initialize() throws Exception {
+    createEmptyDirs(new String[] {TEST_ROOT_DIR.toString()});
+    initializeStorage();
+  }
+  
+  /**
+   * Initialize dfs.name.dir and dfs.data.dir with the specified number of
+   * directory entries. Also initialize fs.default.name and 
+   * dfs.blockreport.intervalMsec.
+   */
+  public static Configuration initializeStorageStateConf(int numDirs) {
+    StringBuffer nameNodeDirs =
+      new StringBuffer(new File(TEST_ROOT_DIR, "name1").toString());
+    StringBuffer dataNodeDirs =
+      new StringBuffer(new File(TEST_ROOT_DIR, "data1").toString());
+    for (int i = 2; i <= numDirs; i++) {
+      nameNodeDirs.append("," + new File(TEST_ROOT_DIR, "name"+i));
+      dataNodeDirs.append("," + new File(TEST_ROOT_DIR, "data"+i));
+    }
+    Configuration conf = new Configuration();
+    conf.set("dfs.name.dir", nameNodeDirs.toString());
+    conf.set("dfs.data.dir", dataNodeDirs.toString());
+    conf.set("dfs.blockreport.intervalMsec", 10000);
+    return conf;
+  }
+  
+  /**
+   * Starts the given type of node or all nodes.
+   *
+   * The UpgradeUtilities.initialize() method must be called once before
+   * calling this method.
+   *
+   * @param nodeType
+   *    The type of node to start.  If NAME_NODE, then one
+   *    Namenode is started.  If DATA_NODE, then one Datanode
+   *    is started.
+   * @param operation
+   *    The operation with which to startup the given type
+   *    node. FORMAT and null are treated as a REGULAR startup. If nodeType
+   *    if DATA_NODE, then UPGRADE is also treated as REGULAR.
+   * @param conf
+   *    The configuration to be used in starting the node.
+   *
+   * @throw IllegalStateException
+   *    If this method is called to start a
+   *    node that is already running.
+   */
+  public static void startCluster(NodeType nodeType, StartupOption operation, Configuration conf) throws Exception {
+    if (isNodeRunning(nodeType)) {
+      throw new IllegalStateException("Attempting to start "
+        + nodeType + " but it is already running");
+    }
+    if (nodeType == DATA_NODE && operation == StartupOption.UPGRADE) {
+      operation = StartupOption.REGULAR;
+    }
+    String[] args = (operation == null ||
+      operation == StartupOption.FORMAT ||
+      operation == StartupOption.REGULAR) ?
+        new String[] {} : new String[] {"-"+operation.toString()};
+    switch (nodeType) {
+      case NAME_NODE:
+        // Set up the right ports for the datanodes
+        conf.set("fs.default.name",NAMENODE_HOST);
+        namenode = NameNode.createNameNode(args, conf);
+        break;
+      case DATA_NODE:
+        if (namenode == null) {
+          throw new IllegalStateException("Attempting to start DATA_NODE "
+            + "but NAME_NODE is not running");
+        }
+        // Set up the right ports for the datanodes
+        InetSocketAddress nnAddr = namenode.getNameNodeAddress(); 
+        conf.set("fs.default.name", nnAddr.getHostName()+ ":" + nnAddr.getPort());
+        conf.setInt("dfs.info.port", 0);
+        conf.setInt("dfs.datanode.info.port", 0);
+        datanode = DataNode.createDataNode(args, conf);
+        break;
+    }
+  }
+  
+  /**
+   * Stops the given type of node or all nodes.
+   *
+   * The UpgradeUtilities.initialize() method must be called once before
+   * calling this method.
+   *
+   * @param nodeType
+   *    The type of node to stop if it is running. If null, then both
+   *    Namenode and Datanodes are stopped if they are running.
+   */
+  public static void stopCluster(NodeType nodeType) {
+    if (nodeType == NAME_NODE || nodeType == null) {
+      if (namenode != null) {
+        namenode.stop();
+      }
+      namenode = null;
+    }
+    if (nodeType == DATA_NODE || nodeType == null) {
+      if (datanode != null) {
+        datanode.shutdown(); 
+      }
+      DataNode.shutdownAll();
+      datanode = null;
+    }
+  }
+  
+  /**
+   * If the Namenode is running, attempt to finalize a previous upgrade.
+   * When this method return, the NameNode should be finalized, but
+   * DataNodes may not be since that occurs asynchronously.
+   *
+   * @throw IllegalStateException if the Namenode is not running.
+   */
+  public static void finalizeCluster(Configuration conf) throws Exception {
+    if (! isNodeRunning(NAME_NODE)) {
+      throw new IllegalStateException("Attempting to finalize "
+        + "Namenode but it is not running");
+    }
+    new DFSAdmin().doMain(conf, new String[] {"-finalizeUpgrade"});
+  }
+  
+  /**
+   * Determines if the given node type is currently running.
+   * If the node type is DATA_NODE, then all started Datanodes
+   * must be running in-order for this method to return
+   * <code>true</code>.
+   *
+   * The UpgradeUtilities.initialize() method must be called once before
+   * calling this method.
+   */
+  public static boolean isNodeRunning(NodeType nodeType) {
+    switch( nodeType ) {
+      case NAME_NODE:
+        return namenode != null;
+      case DATA_NODE:
+        return datanode != null;
+      default:
+        assert false : "Invalid node type: " + nodeType;
+    }
+    return false;
+  }
+  
+  /**
+   * Format the given directories.  This is equivalent to the Namenode
+   * formatting the given directories.  If a given directory already exists,
+   * it is first deleted; otherwise if it does not exist, it is first created.
+   *
+   * @throw IOException if unable to format one of the given dirs
+   */
+  public static void format(File... dirs) throws IOException {
+    String imageDirs = "";
+    for (int i = 0; i < dirs.length; i++) {
+      if( i == 0 )
+        imageDirs = dirs[i].getCanonicalPath();
+      else
+        imageDirs += "," + dirs[i].getCanonicalPath();
+    }
+    Configuration conf = new Configuration();
+    conf.set("dfs.name.dir", imageDirs);
+    NameNode.format(conf);
+  }
+  
+  /**
+   * Create empty directories.  If a specified directory already exists
+   * then it is first removed.
+   */
+  public static void createEmptyDirs(String[] dirs) {
+    for (String d : dirs) {
+      File dir = new File(d);
+      if (dir.exists()) {
+        remove(dir);
+      }
+      dir.mkdirs();
+    }
+  }
+  
+  /**
+   * Return the checksum for the singleton master storage directory
+   * of the given node type.
+   */
+  public static long checksumMasterContents(NodeType nodeType) throws IOException {
+    if (nodeType == NAME_NODE) {
+      return namenodeStorageChecksum;
+    } else {
+      return datanodeStorageChecksum;
+    }
+  }
+  
+  /**
+   * Compute the checksum of all the files in the specified directory.
+   * The contents of subdirectories are not included. This method provides
+   * an easy way to ensure equality between the contents of two directories.
+   *
+   * @param nodeType if DATA_NODE then any file named "VERSION" is ignored.
+   *    This is because this file file is changed every time
+   *    the Datanode is started.
+   * @param dir must be a directory. Subdirectories are ignored.
+   *
+   * @throw IllegalArgumentException if specified directory is not a directory
+   * @throw IOException if an IOException occurs while reading the files
+   * @return the computed checksum value
+   */
+  public static long checksumContents(NodeType nodeType, File dir) throws IOException {
+    if (!dir.isDirectory()) {
+      throw new IllegalArgumentException(
+        "Given argument is not a directory:" + dir);
+    }
+    File[] list = dir.listFiles();
+    CRC32 checksum = new CRC32();
+    for (int i = 0; i < list.length; i++) {
+      if (list[i].isFile()) {
+        // skip VERSION file for DataNodes
+        if (nodeType == DATA_NODE &&
+          list[i].getName().equals("VERSION")) 
+        {
+          continue; 
+        }
+        FileInputStream fis = new FileInputStream(list[i]);
+        byte[] buffer = new byte[1024];
+        int bytesRead;
+        while ((bytesRead = fis.read(buffer)) != -1) {
+          checksum.update(buffer,0,bytesRead);
+        }
+        fis.close();
+      }
+    }
+    return checksum.getValue();
+  }
+  
+  /**
+   * Simulate the <code>dfs.name.dir</code> or <code>dfs.data.dir</code>
+   * of a populated DFS filesystem.
+   *
+   * This method creates and populates the directory specified by
+   *  <code>parent/dirName</code>, for each parent directory.
+   * The contents of the new directories will be
+   * appropriate for the given node type.  If the directory does not
+   * exist, it will be created.  If the directory already exists, it
+   * will first be deleted.
+   *
+   * By default, a singleton master populated storage
+   * directory is created for a Namenode (contains edits, fsimage,
+   * version, and time files) and a Datanode (contains version and
+   * block files).  These directories are then
+   * copied by this method to create new storage
+   * directories of the appropriate type (Namenode or Datanode).
+   *
+   * @return the array of created directories
+   */
+  public static File[] createStorageDirs(NodeType nodeType, String[] parents, String dirName) throws Exception {
+    File[] retVal = new File[parents.length];
+    for (int i = 0; i < parents.length; i++) {
+      File newDir = new File(parents[i], dirName);
+      createEmptyDirs(new String[] {newDir.toString()});
+      populateDir(nodeType, newDir);
+      retVal[i] = newDir;
+    }
+    return retVal;
+  }
+  
+  /**
+   * Create a <code>version</code> file inside the specified parent
+   * directory.  If such a file already exists, it will be overwritten.
+   * The given version string will be written to the file as the layout
+   * version. If null, then the current layout version will be used.
+   * The parent and nodeType parameters must not be null.
+   *
+   * @param version
+   *
+   * @return the created version file
+   */
+  public static File[] createVersionFile(NodeType nodeType, File[] parent,
+    StorageInfo version) throws IOException 
+  {
+    if (version == null)
+      version = getCurrentNamespaceInfo();
+    Storage storage = null;
+    File[] versionFiles = new File[parent.length];
+    for (int i = 0; i < parent.length; i++) {
+      File versionFile = new File(parent[i], "VERSION");
+      remove(versionFile);
+      switch (nodeType) {
+        case NAME_NODE:
+          System.out.println("HERE");
+          storage = new FSImage( version );
+          break;
+        case DATA_NODE:
+                  System.out.println("HERE2");
+          storage = new DataStorage( version, "doNotCare" );
+          break;
+      }
+      StorageDirectory sd = storage.new StorageDirectory(parent[i].getParentFile());
+      sd.write(versionFile);
+      versionFiles[i] = versionFile;
+    }
+    return versionFiles;
+  }
+  
+  /**
+   * Remove the specified file.  If the given file is a directory,
+   * then the directory and all its contents will be removed.
+   */
+  public static boolean remove(File file) {
+    try {
+      boolean retVal = FileUtil.fullyDelete(file);
+      return retVal;
+    } catch (IOException ioe) {
+      // this should never happen
+      throw new IllegalStateException(
+        "WHAT? FileUtil.fullyDelete threw and IOException?",ioe);
+    }
+  }
+  
+  /**
+   * Corrupt the specified file.  Some random bytes within the file
+   * will be changed to some random values.
+   *
+   * @throw IllegalArgumentException if the given file is not a file
+   * @throw IOException if an IOException occurs while reading or writing the file
+   */
+  public static void corruptFile(File file) throws IOException {
+    if (!file.isFile()) {
+      throw new IllegalArgumentException(
+        "Given argument is not a file:" + file);
+    }
+    RandomAccessFile raf = new RandomAccessFile(file,"rws");
+    Random random = new Random();
+    for (long i = 0; i < raf.length(); i++) {
+      raf.seek(i);
+      if (random.nextBoolean()) {
+        raf.writeByte(random.nextInt());
+      }
+    }
+    raf.close();
+  }
+  
+  /**
+   * Retrieve the current NamespaceInfo object from a running Namenode.
+   */
+  public static NamespaceInfo getCurrentNamespaceInfo() throws IOException {
+    if (isNodeRunning(NAME_NODE))
+      return namenode.versionRequest();
+    return null;
+  }
+  
+  /**
+   * Return the layout version inherent in the current version
+   * of the Namenode, whether it is running or not.
+   */
+  public static int getCurrentLayoutVersion() {
+    return FSConstants.LAYOUT_VERSION;
+  }
+  
+  /**
+   * Return the namespace ID inherent in the currently running
+   * Namenode.  If no Namenode is running, return the namespace ID of
+   * the master Namenode storage directory.
+   *
+   * The UpgradeUtilities.initialize() method must be called once before
+   * calling this method.
+   */
+  public static int getCurrentNamespaceID() throws IOException {
+    if (isNodeRunning(NAME_NODE)) {
+      return namenode.versionRequest().getNamespaceID();
+    }
+    return namenodeStorageNamespaceID;
+  }
+  
+  /**
+   * Return the File System State Creation Timestamp (FSSCTime) inherent
+   * in the currently running Namenode.  If no Namenode is running,
+   * return the FSSCTime of the master Namenode storage directory.
+   *
+   * The UpgradeUtilities.initialize() method must be called once before
+   * calling this method.
+   */
+  public static long getCurrentFsscTime() throws IOException {
+    if (isNodeRunning(NAME_NODE)) {
+      return namenode.versionRequest().getCTime();
+    }
+    return namenodeStorageFsscTime;
+  }
+  
+  /**********************************************************************
+   ********************* PRIVATE METHODS ********************************
+   *********************************************************************/
+  
+  /**
+   * Populates the given directory with valid version, edits, and fsimage
+   * files.  The version file will contain the current layout version.
+   *
+   * The UpgradeUtilities.initialize() method must be called once before
+   * calling this method.
+   *
+   * @throw IllegalArgumentException if dir does not already exist
+   */
+  private static void populateDir(NodeType nodeType, File dir) throws Exception {
+    if (!dir.exists()) {
+      throw new IllegalArgumentException(
+        "Given argument is not an existing directory:" + dir);
+    }
+    LocalFileSystem localFS = FileSystem.getLocal(new Configuration());
+    switch (nodeType) {
+      case NAME_NODE:
+        localFS.copyToLocalFile(
+          new Path(namenodeStorage.toString(), "current"),
+          new Path(dir.toString()),
+          false);
+        break;
+      case DATA_NODE:
+        localFS.copyToLocalFile(
+          new Path(datanodeStorage.toString(), "current"),
+          new Path(dir.toString()),
+          false);
+        break;
+    }
+  }
+  
+  static void writeFile(FileSystem fs,
+    Path path,
+    byte[] buffer,
+    int bufferSize ) throws IOException {
+    OutputStream out;
+    out = fs.create(path, true, bufferSize, (short) 1, 1024);
+    out.write( buffer, 0, bufferSize );
+    out.close();
+  }
+  
+  /**
+   * Creates a singleton master populated storage
+   * directory for a Namenode (contains edits, fsimage,
+   * version, and time files) and a Datanode (contains version and
+   * block files).  This can be a lengthy operation.
+   *
+   * @param conf must not be null.  These properties will be set:
+   *    fs.default.name
+   *    dfs.name.dir
+   *    dfs.data.dir
+   */
+  private static void initializeStorage() throws Exception {
+    Configuration config = new Configuration();
+    config.set("fs.default.name",NAMENODE_HOST);
+    config.set("dfs.name.dir", namenodeStorage.toString());
+    config.set("dfs.data.dir", datanodeStorage.toString());
+
+    try {
+      // format data-node
+      createEmptyDirs(new String[] {datanodeStorage.toString()});
+
+      // format name-node
+      NameNode.format(config);
+      
+      // start name-node
+      startCluster(NAME_NODE, null, config);
+      namenodeStorageNamespaceID = namenode.versionRequest().getNamespaceID();
+      namenodeStorageFsscTime = namenode.versionRequest().getCTime();
+      
+      // start data-node
+      startCluster(DATA_NODE, null, config);
+      
+      FileSystem fs = FileSystem.get(config);
+      Path baseDir = new Path("/TestUpgrade");
+      fs.mkdirs( baseDir );
+      
+      // write some files
+      int bufferSize = 4096;
+      byte[] buffer = new byte[bufferSize];
+      for( int i=0; i < bufferSize; i++ )
+        buffer[i] = (byte)('0' + i % 50);
+      writeFile(fs, new Path(baseDir, "file1"), buffer, bufferSize);
+      writeFile(fs, new Path(baseDir, "file2"), buffer, bufferSize);
+      
+      // save image
+      namenode.getFSImage().saveFSImage();
+      namenode.getFSImage().getEditLog().open();
+      
+      // write more files
+      writeFile(fs, new Path(baseDir, "file3"), buffer, bufferSize);
+      writeFile(fs, new Path(baseDir, "file4"), buffer, bufferSize);
+    } finally {
+      // shutdown
+      stopCluster(null);
+      remove(new File(namenodeStorage,"in_use.lock"));
+      remove(new File(datanodeStorage,"in_use.lock"));
+    }
+    namenodeStorageChecksum = checksumContents(
+      NAME_NODE, new File(namenodeStorage,"current"));
+    datanodeStorageChecksum = checksumContents(
+      DATA_NODE, new File(datanodeStorage,"current"));
+  }
+
+}
+

Неке датотеке нису приказане због велике количине промена