You are viewing a plain text version of this content. The canonical link for it is here.
Posted to hdfs-commits@hadoop.apache.org by st...@apache.org on 2009/08/20 19:13:43 UTC

svn commit: r806261 - in /hadoop/hdfs/branches/HDFS-326: ./ src/java/org/apache/hadoop/hdfs/ src/java/org/apache/hadoop/hdfs/server/datanode/ src/java/org/apache/hadoop/hdfs/server/namenode/ src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/

Author: stevel
Date: Thu Aug 20 17:13:41 2009
New Revision: 806261

URL: http://svn.apache.org/viewvc?rev=806261&view=rev
Log:
HDFS-326 Service Lifecycle

Added:
    hadoop/hdfs/branches/HDFS-326/ivybuild.xml
Modified:
    hadoop/hdfs/branches/HDFS-326/build.xml
    hadoop/hdfs/branches/HDFS-326/src/java/org/apache/hadoop/hdfs/DFSClient.java
    hadoop/hdfs/branches/HDFS-326/src/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
    hadoop/hdfs/branches/HDFS-326/src/java/org/apache/hadoop/hdfs/server/datanode/FSDataset.java
    hadoop/hdfs/branches/HDFS-326/src/java/org/apache/hadoop/hdfs/server/namenode/BackupNode.java
    hadoop/hdfs/branches/HDFS-326/src/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
    hadoop/hdfs/branches/HDFS-326/src/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
    hadoop/hdfs/branches/HDFS-326/src/java/org/apache/hadoop/hdfs/server/namenode/PendingReplicationBlocks.java
    hadoop/hdfs/branches/HDFS-326/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestReplicationPolicy.java

Modified: hadoop/hdfs/branches/HDFS-326/build.xml
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-326/build.xml?rev=806261&r1=806260&r2=806261&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-326/build.xml (original)
+++ hadoop/hdfs/branches/HDFS-326/build.xml Thu Aug 20 17:13:41 2009
@@ -17,7 +17,7 @@
    limitations under the License.
 -->
 
-<project name="Hadoop-Hdfs" default="compile" 
+<project name="hadoop-hdfs" default="compile" 
    xmlns:ivy="antlib:org.apache.ivy.ant"> 
 
   <!-- Load all the default properties, and any the user wants    -->
@@ -28,8 +28,8 @@
   <property name="Name" value="Hadoop-Hdfs"/>
   <property name="name" value="hadoop-hdfs"/>
   <property name="version" value="0.21.0-dev"/>
-  <property name="hadoop-core.version" value="0.21.0-dev"/>
-  <property name="hadoop-mr.version" value="0.21.0-dev"/>
+  <property name="hadoop-core.version" value="${version}"/>
+  <property name="hadoop-mr.version" value="${version}"/>
   <property name="final.name" value="${name}-${version}"/>
   <property name="test.hdfs.final.name" value="${name}-test-${version}"/>
   <property name="test.hdfswithmr.final.name" value="${name}-hdfswithmr-test-${version}"/>

Added: hadoop/hdfs/branches/HDFS-326/ivybuild.xml
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-326/ivybuild.xml?rev=806261&view=auto
==============================================================================
--- hadoop/hdfs/branches/HDFS-326/ivybuild.xml (added)
+++ hadoop/hdfs/branches/HDFS-326/ivybuild.xml Thu Aug 20 17:13:41 2009
@@ -0,0 +1,353 @@
+<?xml version="1.0"?>
+<project name="hadoop-hdfs" default="published"
+  xmlns:ivy="antlib:org.apache.ivy.ant">
+<!--
+   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.
+-->
+
+  <description>
+    This is a build file to publish Hadoop hdfs as ivy and maven artifacts.
+    It currently works alongside the original build.xml file, and exists
+    purely to hook up hadoop into the SmartFrog test/release process.
+  </description>
+
+  <!--Override point: allow for overridden in properties to be loaded-->
+  <property file="build.properties" />
+  <property file="../build.properties" />
+
+
+  <target name="ivy-init-properties" >
+    <property name="ivy.dir" location="ivy" />
+    <loadproperties srcfile="${ivy.dir}/libraries.properties" />
+    <property name="ivysettings.xml" location="${ivy.dir}/ivysettings.xml" />
+    <property name="ivy.jar" location="${ivy.dir}/ivy-${ivy.version}.jar"/>
+    <property name="ivy.org" value="org.apache.hadoop"/>
+
+    <property name="build.dir" location="build" />
+    <property name="build.ivy.dir" location="${build.dir}/ivy" />
+    <property name="build.ivy.lib.dir" location="${build.ivy.dir}/lib" />
+    <property name="build.ivy.report.dir" location="${build.ivy.dir}/report" />
+    <property name="build.ivy.maven.dir" location="${build.ivy.dir}/maven" />
+    <property name="module" value="hdfs" />
+    <property name="build.ivy.maven.pom" 
+      location="${build.ivy.maven.dir}/hadoop-${module}-${hadoop.version}.pom" />
+    <property name="build.ivy.maven.jar" 
+      location="${build.ivy.maven.dir}/hadoop-${module}-${hadoop.version}.jar" />
+
+    <!--this is the naming policy for artifacts we want pulled down-->
+    <property name="ivy.artifact.retrieve.pattern"
+      value="[conf]/[artifact]-[revision].[ext]"/>
+    <!--this is how artifacts that get built are named-->
+    <property name="ivy.publish.pattern"
+      value="hadoop-[revision]-core.[ext]"/>
+    <property name="hadoop.jar"
+      location="${build.dir}/hadoop-${hadoop.version}-${module}.jar" />
+
+    <!--preset to build down; puts us in control of version naming-->
+    <presetdef name="delegate">
+      <ant antfile="build.xml" inheritall="false" inheritrefs="false" >
+        <property name="version" value="${hadoop.version}"/>
+      </ant>
+    </presetdef>
+    <!--preset to build down; puts us in control of version naming-->
+    <presetdef name="delegate2">
+      <subant antfile="build.xml" buildpath="." inheritall="false" inheritrefs="false" >
+        <property name="version" value="${hadoop.version}"/>
+      </subant>
+    </presetdef>
+
+    <!--preset to copy with ant property expansion (and always overwrite)-->
+    <presetdef name="expandingcopy" >
+    <copy overwrite="true">
+      <filterchain>
+        <expandproperties/>
+      </filterchain>
+    </copy>
+  </presetdef>
+  </target>
+
+
+  <target name="ivy-init-dirs" depends="ivy-init-properties" >
+    <mkdir dir="${build.ivy.dir}" />
+    <mkdir dir="${build.ivy.lib.dir}" />
+    <mkdir dir="${build.ivy.report.dir}" />
+    <mkdir dir="${build.ivy.maven.dir}" />
+  </target>
+
+
+  <target name="clean"  depends="ivy-init-properties"
+    description="Clean the output directories" >
+    <delegate target="clean" />
+  </target>
+
+
+  <target name="jar"  depends="ivy-init-dirs"
+    description="build the JAR">
+    <delegate target="jar" />
+  </target>
+
+  <!--
+    This looks for Ivy on the classpath, and is used to skip reloading it if found.
+    It looks for an ivy-2.0 file.
+  -->
+  <target name="ivy-probe-antlib" >
+    <condition property="ivy.found">
+      <typefound uri="antlib:org.apache.ivy.ant" name="cleancache"/>
+    </condition>
+  </target>
+
+
+  <!--
+  To avoid Ivy leaking things across big projects, always load Ivy in the same classloader.
+  Also note how we skip loading Ivy if it is already there, just to make sure all is well.
+  -->
+  <target name="ivy-init-antlib" depends="ivy-init-properties,ivy-init-dirs,ivy-probe-antlib" unless="ivy.found">
+
+    <typedef uri="antlib:org.apache.ivy.ant" onerror="fail"
+      loaderRef="ivyLoader">
+      <classpath>
+        <pathelement location="${ivy.jar}"/>
+      </classpath>
+    </typedef>
+    <fail >
+      <condition >
+        <not>
+          <typefound uri="antlib:org.apache.ivy.ant" name="cleancache"/>
+        </not>
+      </condition>
+      You need Apache Ivy 2.0 or later from http://ant.apache.org/
+      It could not be loaded from ${ivy.jar}
+    </fail>
+  </target>
+
+
+  <target name="ivy-init" depends="ivy-init-antlib" >
+
+    <!--Configure Ivy by reading in the settings file
+        If anyone has already read in a settings file into this settings ID, it gets priority
+    -->
+    <ivy:configure settingsId="hadoop.ivy.settings" file="${ivysettings.xml}" override="false"/>
+
+  </target>
+
+  <target name="ivy-resolve" depends="ivy-init">
+    <ivy:resolve settingsRef="hadoop.ivy.settings"/>
+  </target>
+
+  <target name="ivy-retrieve" depends="ivy-resolve"
+    description="Retrieve all Ivy-managed artifacts for the different configurations">
+    <ivy:retrieve settingsRef="hadoop.ivy.settings"
+      pattern="${build.ivy.lib.dir}/${ivy.artifact.retrieve.pattern}" sync="true" />
+  </target>
+
+  <target name="ivy-report" depends="ivy-resolve"
+    description="Generate">
+    <ivy:report todir="${build.ivy.report.dir}" settingsRef="hadoop.ivy.settings"/>
+    <echo>
+      Reports generated:
+${build.ivy.report.dir}
+    </echo>
+  </target>
+
+  <target name="assert-hadoop-jar-exists" depends="ivy-init">
+    <fail>
+      <condition >
+        <not>
+          <available file="${hadoop.jar}" />
+        </not>
+      </condition>
+      Not found: ${hadoop.jar}
+      Please run the target "jar" in the main build file
+    </fail>
+
+  </target>
+
+  <target name="ready-to-publish" depends="jar,assert-hadoop-jar-exists,ivy-resolve"/>
+
+  <target name="ivy-publish-local" depends="ready-to-publish">
+    <ivy:publish
+      settingsRef="hadoop.ivy.settings"
+      resolver="local"
+      pubrevision="${hadoop.version}"
+      overwrite="true"
+      artifactspattern="${build.dir}/${ivy.publish.pattern}" />
+  </target>
+
+
+  <!-- this is here for curiosity, to see how well the makepom task works
+  Answer: it depends whether you want transitive dependencies excluded or not
+  -->
+  <target name="makepom" depends="ivy-resolve">
+    <ivy:makepom settingsRef="hadoop.ivy.settings"
+      ivyfile="ivy.xml"
+      pomfile="${build.ivy.maven.dir}/generated.pom">
+      <ivy:mapping conf="default" scope="default"/>
+      <ivy:mapping conf="master"  scope="master"/>
+      <ivy:mapping conf="runtime" scope="runtime"/>
+    </ivy:makepom>
+  </target>
+
+
+  <target name="copy-jar-to-maven" depends="ready-to-publish">
+    <copy file="${hadoop.jar}"
+      tofile="${build.ivy.maven.jar}"/>
+    <checksum file="${build.ivy.maven.jar}" algorithm="md5"/>
+  </target>
+
+  <target name="copypom" depends="ivy-init-dirs">
+    <expandingcopy file="ivy/hadoop-core.pom"
+      tofile="${build.ivy.maven.pom}"/>
+    <checksum file="${build.ivy.maven.pom}" algorithm="md5"/>
+  </target>
+
+  <target name="maven-artifacts" depends="copy-jar-to-maven,copypom" />
+
+  <target name="published" depends="ivy-publish-local,maven-artifacts">
+
+  </target>
+
+  <target name="ready-to-test" depends="ivy-init-dirs">
+    <property name="test.data.dir" location="${build.dir}/test/data" />
+    <property name="test.reports.dir" location="${build.dir}/test/reports" />
+    <mkdir dir="${test.data.dir}" />
+    <mkdir dir="${test.reports.dir}" />
+  </target>
+
+  <target name="testjob.jar"  depends="ready-to-test">
+    <delegate2 target="jar-test"
+        failonerror="true">
+    </delegate2>
+  </target>
+
+
+  <target name="junit"  depends="ready-to-test,testjob.jar"
+      description="run the junit tests and generate an XML report">
+    <delegate2 target="test-core"
+        failonerror="false">
+      <property name="test.junit.output.format" value="xml" />
+      <property name="test.build.dir" value="${test.data.dir}"/>
+    </delegate2>
+  </target>
+
+  <!-- generate a junit report. 
+  tip: you can run this while junit is still going on-->
+  <target name="junitreport"  depends="ready-to-test">
+    <junitreport todir="${test.reports.dir}">
+      <fileset dir="${test.data.dir}">
+        <include name="TEST-*.xml"/>
+      </fileset>
+      <report format="frames" todir="${test.reports.dir}"/>
+    </junitreport>
+    <echo>reports in ${test.reports.dir}/index.html</echo>
+  </target>
+
+  <target name="tested" depends="junit,junitreport" />
+
+  <target name="svn-init">
+    <presetdef name="svn">
+      <exec executable="svn" failonerror="true">
+      </exec>
+    </presetdef>
+    <property name="issue" value="HDFS-326"/>
+    <property name="hadoop-svn"
+      value="https://svn.apache.org/repos/asf/hadoop/common"/>
+    <property name="trunk"
+      value="${hadoop-svn}/trunk"/>
+    <property name="branch"
+      value="${hadoop-svn}/branches/${issue}"/>
+    <property name="patches.dir" location="../outgoing"/>
+    <mkdir dir="${patches.dir}" />
+    <property name="patch.version" value="1" />
+    <property name="patch.file"
+      location="${patches.dir}/${issue}-${patch.version}.patch" />
+  </target>  
+
+  <target name="svn-merge" depends="svn-init"
+    description="merge in the trunk"  >
+    <svn>
+      <arg value="merge"/>
+      <arg value="${trunk}"/>
+      <arg value="--accept"/>
+      <arg value="postpone"/>
+    </svn>
+  </target>
+  
+  <target name="svn-diff" depends="svn-init"
+    description="diff the local code against the branch"  >
+    <svn>
+      <arg value="diff"/>
+    </svn>
+  </target>
+
+  <target name="svn-resolved" depends="svn-init"
+    description="mark the tree as resolved"  >
+    <svn>
+      <arg value="resolve"/>
+    </svn>
+  </target>
+
+  <!--
+  svn diff \
+  https://svn.apache.org/repos/asf/hadoop/core/trunk \
+  https://svn.apache.org/repos/asf/hadoop/core/branches/HADOOP-3628-2
+  -->
+  <target name="svn-diff-trunk" depends="svn-init"
+      description="diff against trunk"  >
+    <svn>
+      <arg value="diff" />
+      <arg value="${trunk}"/>
+      <arg value="${branch}"/>
+    </svn>
+  </target>
+
+
+  <target name="svn-create-changelist" depends="svn-init"
+      description="Create a changelist of everything we want in the big patch"  >
+    <property name="hdfs/server"
+        value="src/java/org/apache/hadoop/hdfs/server" />
+    <property name="test/hdfs"
+        value="src/test/hdfs/org/apache/hadoop/hdfs" />
+    <svn>
+      <arg value="changelist"/>
+      <arg value="${issue}"/>
+
+      <arg value="${hdfs/server}/datanode/DataNode.java" />
+      <arg value="${hdfs/server}/datanode/FSDataset.java" />
+      <arg value="${hdfs/server}/namenode/BackupNode.java" />
+      <arg value="${hdfs/server}/namenode/FSNamesystem.java" />
+      <arg value="${hdfs/server}/namenode/NameNode.java" />
+      <arg value="${hdfs/server}/namenode/PendingReplicationBlocks.java" />
+      <arg value="${test/hdfs}/server/namenode/TestReplicationPolicy.java" />
+    </svn>
+  </target>
+
+
+  <!--
+  
+  -->
+  <target name="svn-diff-src" depends="svn-init"
+      description="diff against trunk"  >
+    <echo> Writing to ${patch.file}</echo>
+    <svn output="${patch.file}" >
+      <arg value="diff" />
+      <arg value="${trunk}/src" />
+      <arg value="${branch}/src" />
+      <arg value="--changelist" />
+      <arg value="${issue}"/>
+    </svn>
+  </target>
+
+</project>
\ No newline at end of file

Modified: hadoop/hdfs/branches/HDFS-326/src/java/org/apache/hadoop/hdfs/DFSClient.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-326/src/java/org/apache/hadoop/hdfs/DFSClient.java?rev=806261&r1=806260&r2=806261&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-326/src/java/org/apache/hadoop/hdfs/DFSClient.java (original)
+++ hadoop/hdfs/branches/HDFS-326/src/java/org/apache/hadoop/hdfs/DFSClient.java Thu Aug 20 17:13:41 2009
@@ -144,6 +144,7 @@
   private final FileSystem.Statistics stats;
   private int maxBlockAcquireFailures;
   private final int hdfsTimeout;    // timeout value for a DFS operation.
+  private IOException closedLocation; // where this filesystem was closed
 
   /**
    * The locking hierarchy is to first acquire lock on DFSClient object, followed by 
@@ -288,6 +289,10 @@
   private void checkOpen() throws IOException {
     if (!clientRunning) {
       IOException result = new IOException("Filesystem closed");
+      if (closedLocation != null) {
+        //report where the client was closed
+        result.initCause(closedLocation);
+      }
       throw result;
     }
   }
@@ -307,6 +312,9 @@
   
       // close connections to the namenode
       RPC.stopProxy(rpcNamenode);
+      //note where the location was closed
+      closedLocation = new IOException("Filesystem closed");
+      LOG.debug("Closing filesystem", closedLocation);
     }
   }
 

Modified: hadoop/hdfs/branches/HDFS-326/src/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-326/src/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java?rev=806261&r1=806260&r2=806261&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-326/src/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java (original)
+++ hadoop/hdfs/branches/HDFS-326/src/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java Thu Aug 20 17:13:41 2009
@@ -98,6 +98,7 @@
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.DiskChecker.DiskErrorException;
 import org.apache.hadoop.util.DiskChecker.DiskOutOfSpaceException;
+import org.apache.hadoop.util.Service;
 
 /**********************************************************
  * DataNode is a class (and program) that stores a set of
@@ -130,7 +131,7 @@
  * information to clients or other DataNodes that might be interested.
  *
  **********************************************************/
-public class DataNode extends Configured 
+public class DataNode extends Service
     implements InterDatanodeProtocol, ClientDatanodeProtocol, FSConstants, Runnable {
   public static final Log LOG = LogFactory.getLog(DataNode.class);
   
@@ -204,7 +205,8 @@
   
   /** Activated plug-ins. */
   private List<ServicePlugin> plugins;
-  
+  /** data directories */
+  private AbstractList<File> dataDirs;
   private static final Random R = new Random();
   
   // For InterDataNodeProtocol
@@ -221,19 +223,55 @@
   /**
    * Create the DataNode given a configuration and an array of dataDirs.
    * 'dataDirs' is where the blocks are stored.
+   * <p>?
+   * Important: this constructor does not start
+   * the node, merely initializes it
+   *
+   * @param conf     configuration to use
+   * @param dataDirs list of directories that may be used for data
+   * @throws IOException for historical reasons
    */
-  DataNode(Configuration conf, 
+  DataNode(Configuration conf,
            AbstractList<File> dataDirs) throws IOException {
     super(conf);
     DataNode.setDataNode(this);
-    try {
-      startDataNode(conf, dataDirs);
-    } catch (IOException ie) {
-      shutdown();
-      throw ie;
-    }
+    this.dataDirs = dataDirs;
+  }
+
+/////////////////////////////////////////////////////                                           
+// Lifecycle                                                                                    
+/////////////////////////////////////////////////////                                           
+
+  /**
+   * Start any work (in separate threads)
+   *
+   * @throws IOException for any startup failure
+   */
+  @Override
+  public void innerStart() throws IOException {
+    startDataNode(getConf(), dataDirs);
   }
     
+  /**
+   * {@inheritDoc}.
+   *
+   * This implementation checks for the IPC server running and the
+   * DataNode being registered to a namenode.
+   *
+   * @param status the initial status
+   * @throws IOException       for any ping failure
+   * @throws LivenessException if the IPC server is not defined 
+   */
+  @Override
+  public void innerPing(ServiceStatus status) throws IOException {
+    if (ipcServer == null) {
+      status.addThrowable(new LivenessException("No IPC Server running"));
+    }
+    if (dnRegistration == null) {
+      status.addThrowable(
+              new LivenessException("Not registered to a namenode"));
+    }
+  }
   
   /**
    * This method starts the data node with the specified conf.
@@ -362,6 +400,9 @@
     int tmpInfoPort = infoSocAddr.getPort();
     this.infoServer = new HttpServer("datanode", infoHost, tmpInfoPort,
         tmpInfoPort == 0, conf);
+    if(LOG.isDebugEnabled()) {
+      LOG.debug("Datanode listening on " + infoHost + ":" + tmpInfoPort);
+    }
     if (conf.getBoolean("dfs.https.enable", false)) {
       boolean needClientAuth = conf.getBoolean("dfs.https.need.client.auth", false);
       InetSocketAddress secInfoSocAddr = NetUtils.createSocketAddr(conf.get(
@@ -369,6 +410,9 @@
       Configuration sslConf = new Configuration(false);
       sslConf.addResource(conf.get("dfs.https.server.keystore.resource",
           "ssl-server.xml"));
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Datanode listening for SSL on " + secInfoSocAddr);
+      }
       this.infoServer.addSslListener(secInfoSocAddr, sslConf, needClientAuth);
     }
     this.infoServer.addInternalServlet(null, "/streamFile/*", StreamFile.class);
@@ -435,6 +479,10 @@
         } catch (InterruptedException ie) {}
       }
     }
+    if(!shouldRun) {
+      throw new IOException("Datanode shut down during handshake with NameNode "
+               + getNameNodeAddr());
+    }
     String errorMsg = null;
     // verify build version
     if( ! nsInfo.getBuildVersion().equals( Storage.getBuildVersion() )) {
@@ -544,10 +592,14 @@
    * @see FSNamesystem#registerDatanode(DatanodeRegistration)
    * @throws IOException
    */
-  private void register() throws IOException {
+  protected void register() throws IOException {
     if (dnRegistration.getStorageID().equals("")) {
       setNewStorageID(dnRegistration);
     }
+    //if we are LIVE, move into the STARTED state, as registration implies that
+    //the node is no longer LIVE
+    enterState(ServiceState.LIVE, ServiceState.STARTED);
+    //spin until the server is up.
     while(shouldRun) {
       try {
         // reset name to machineName. Mainly for web interface.
@@ -598,39 +650,55 @@
       dnRegistration.exportedKeys = ExportedAccessKeys.DUMMY_KEYS;
     }
 
+    //at this point the DataNode now considers itself live.  
+    enterLiveState();
     // random short delay - helps scatter the BR from all DNs
     scheduleBlockReport(initialBlockReportDelay);
   }
 
+
+  /**
+   * Shut down this instance of the datanode. Returns only after shutdown is
+   * complete.
+   */
+  public void shutdown() {
+    closeQuietly();
+  }
+
   /**
    * Shut down this instance of the datanode.
    * Returns only after shutdown is complete.
    * This method can only be called by the offerService thread.
    * Otherwise, deadlock might occur.
    */
-  public void shutdown() {
-    if (plugins != null) {
-      for (ServicePlugin p : plugins) {
+  @Override
+  protected void innerClose() throws IOException {
+    synchronized (this) {
+      //disable the should run flag first, so that everything out there starts
+      //to shut down
+      shouldRun = false;
+      if (plugins != null) {
+        for (ServicePlugin p : plugins) {
+          try {
+            p.stop();
+            LOG.info("Stopped plug-in " + p);
+          } catch (Throwable t) {
+            LOG.warn("ServicePlugin " + p + " could not be stopped", t);
+          }
+        }
+      }
+
+      if (infoServer != null) {
         try {
-          p.stop();
-          LOG.info("Stopped plug-in " + p);
-        } catch (Throwable t) {
-          LOG.warn("ServicePlugin " + p + " could not be stopped", t);
+          infoServer.stop();
+        } catch (Exception e) {
+          LOG.warn("Exception shutting down DataNode", e);
         }
       }
-    }
-    
-    if (infoServer != null) {
-      try {
-        infoServer.stop();
-      } catch (Exception e) {
-        LOG.warn("Exception shutting down DataNode", e);
+      if (ipcServer != null) {
+        ipcServer.stop();
       }
     }
-    if (ipcServer != null) {
-      ipcServer.stop();
-    }
-    this.shouldRun = false;
     if (dataXceiverServer != null) {
       ((DataXceiverServer) this.dataXceiverServer.getRunnable()).kill();
       this.dataXceiverServer.interrupt();
@@ -676,6 +744,8 @@
       try {
         this.storage.unlockAll();
       } catch (IOException ie) {
+        LOG.warn("Ignoring exception when unlocking storage: " + ie,
+                ie);
       }
     }
     if (dataNodeThread != null) {
@@ -1265,7 +1335,9 @@
         startDistributedUpgradeIfNeeded();
         offerService();
       } catch (Exception ex) {
-        LOG.error("Exception: " + StringUtils.stringifyException(ex));
+        LOG.error("Exception while in state " + getServiceState()
+                + " and shouldRun=" + shouldRun + ": " + ex,
+                ex);
         if (shouldRun) {
           try {
             Thread.sleep(5000);
@@ -1357,28 +1429,46 @@
   public static DataNode makeInstance(String[] dataDirs, Configuration conf)
     throws IOException {
     ArrayList<File> dirs = new ArrayList<File>();
-    for (int i = 0; i < dataDirs.length; i++) {
-      File data = new File(dataDirs[i]);
+    StringBuffer invalid = new StringBuffer();
+    for (String dataDir : dataDirs) {
+      File data = new File(dataDir);
       try {
         DiskChecker.checkDir(data);
         dirs.add(data);
       } catch(DiskErrorException e) {
-        LOG.warn("Invalid directory in dfs.data.dir: " + e.getMessage());
+        LOG.warn("Invalid directory in dfs.data.dir: " + e, e);
+        invalid.append(dataDir);
+        invalid.append(" ");
       }
     }
-    if (dirs.size() > 0) 
-      return new DataNode(conf, dirs);
-    LOG.error("All directories in dfs.data.dir are invalid.");
-    return null;
+    if (dirs.size() > 0) {
+      DataNode dataNode = new DataNode(conf, dirs);
+      Service.startService(dataNode);
+      return dataNode;
+    } else {
+      LOG.error("All directories in dfs.data.dir are invalid: " + invalid);
+      return null;
+    }
   }
 
+  /**
+   * {@inheritDoc}
+   *
+   * @return the name of this service
+   */
+  @Override
+  public String getServiceName() {
+    return "DataNode";
+  }
+  
   @Override
   public String toString() {
-    return "DataNode{" +
+    return getServiceName() + " {" +
       "data=" + data +
       ", localName='" + dnRegistration.getName() + "'" +
       ", storageID='" + dnRegistration.getStorageID() + "'" +
       ", xmitsInProgress=" + xmitsInProgress.get() +
+      ", state=" + getServiceState() +
       "}";
   }
   

Modified: hadoop/hdfs/branches/HDFS-326/src/java/org/apache/hadoop/hdfs/server/datanode/FSDataset.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-326/src/java/org/apache/hadoop/hdfs/server/datanode/FSDataset.java?rev=806261&r1=806260&r2=806261&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-326/src/java/org/apache/hadoop/hdfs/server/datanode/FSDataset.java (original)
+++ hadoop/hdfs/branches/HDFS-326/src/java/org/apache/hadoop/hdfs/server/datanode/FSDataset.java Thu Aug 20 17:13:41 2009
@@ -179,6 +179,9 @@
       }
 
       File blockFiles[] = dir.listFiles();
+      if (blockFiles == null) {
+        throw new IllegalStateException("Not a valid directory: " + dir);
+      }
       for (int i = 0; i < blockFiles.length; i++) {
         if (Block.isBlockFilename(blockFiles[i])) {
           long genStamp = getGenerationStampFromFile(blockFiles, blockFiles[i]);

Modified: hadoop/hdfs/branches/HDFS-326/src/java/org/apache/hadoop/hdfs/server/namenode/BackupNode.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-326/src/java/org/apache/hadoop/hdfs/server/namenode/BackupNode.java?rev=806261&r1=806260&r2=806261&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-326/src/java/org/apache/hadoop/hdfs/server/namenode/BackupNode.java (original)
+++ hadoop/hdfs/branches/HDFS-326/src/java/org/apache/hadoop/hdfs/server/namenode/BackupNode.java Thu Aug 20 17:13:41 2009
@@ -130,8 +130,17 @@
     runCheckpointDaemon(conf);
   }
 
+  /**
+   * {@inheritDoc} 
+   * <p/> 
+   * When shutting down, this service shuts down the checkpoint manager.
+   * If registered to a namenode, it reports that it is shutting down
+   * via {@link NameNode#errorReport(NamenodeRegistration, int, String)} 
+   *
+   * @throws IOException for any IO problem
+   */
   @Override // NameNode
-  public void stop() {
+  protected void innerClose() throws IOException {
     if(checkpointManager != null) checkpointManager.shouldRun = false;
     if(cpDaemon != null) cpDaemon.interrupt();
     if(namenode != null && getRegistration() != null) {
@@ -143,7 +152,17 @@
       }
     }
     RPC.stopProxy(namenode); // stop the RPC threads
-    super.stop();
+    super.innerClose();
+  }
+
+  /**
+   * {@inheritDoc}
+   *
+   * @return "BackupNode"
+   */
+  @Override
+  public String getServiceName() {
+    return "BackupNode";
   }
 
   /////////////////////////////////////////////////////

Modified: hadoop/hdfs/branches/HDFS-326/src/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-326/src/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java?rev=806261&r1=806260&r2=806261&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-326/src/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java (original)
+++ hadoop/hdfs/branches/HDFS-326/src/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java Thu Aug 20 17:13:41 2009
@@ -106,7 +106,7 @@
       }
   };
 
-  private static final void logAuditEvent(UserGroupInformation ugi,
+  private static void logAuditEvent(UserGroupInformation ugi,
       InetAddress addr, String cmd, String src, String dst,
       FileStatus stat) {
     final Formatter fmt = auditFormatter.get();
@@ -447,6 +447,46 @@
   }
 
   /**
+   * Test for a thread ref not being null or pointing to a dead thread
+   * @param thread the thread to check
+   * @return true if the thread is considered dead
+   */
+  private boolean isDead(Thread thread) {
+      return thread == null || !thread.isAlive();
+  }
+
+  /**
+   * Perform a cursory health check of the namesystem, particulary that it has
+   * not been closed and that all threads are running.
+   * @throws IOException for any health check
+   */
+  void ping() throws IOException {
+    if (!fsRunning) {
+      throw new IOException("Namesystem is not running");
+    }
+    StringBuilder sb = new StringBuilder();
+    if (isDead(hbthread)) {
+      sb.append("[Heartbeat thread is dead] ");
+    }
+    if (isDead(replthread)) {
+      sb.append("[Replication thread is dead] ");
+    }
+    // this thread's liveness is only relevant in safe mode.
+    if (safeMode != null && isDead(smmthread)) {
+      sb.append("[SafeModeMonitor thread is dead while the name system"
+              + " is in safe mode] ");
+    }
+    if (isDead(dnthread)) {
+      sb.append("[DecommissionedMonitor thread is dead] ");
+    }
+    if (isDead(lmthread)) {
+      sb.append("[Lease monitor thread is dead]");
+    }
+    if (sb.length() > 0) {
+      throw new IOException(sb.toString());
+    }
+  }
+  /**
    * Close down this file system manager.
    * Causes heartbeat and lease daemons to stop; waits briefly for
    * them to finish, but a short timeout returns control back to caller.
@@ -468,7 +508,10 @@
           lmthread.interrupt();
           lmthread.join(3000);
         }
-        dir.close();
+        if(dir != null) {
+         dir.close();
+         dir =  null;
+        }
       } catch (InterruptedException ie) {
       } catch (IOException ie) {
         LOG.error("Error closing FSDirectory", ie);
@@ -1134,7 +1177,10 @@
     if (targets.length < blockManager.minReplication) {
       throw new IOException("File " + src + " could only be replicated to " +
                             targets.length + " nodes, instead of " +
-                            blockManager.minReplication);
+                            blockManager.minReplication
+                            + ". ( there are currently " 
+                            + heartbeats.size()
+                            +" live data nodes in the cluster)");
     }
 
     // Allocate a new block and record it in the INode. 

Modified: hadoop/hdfs/branches/HDFS-326/src/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-326/src/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java?rev=806261&r1=806260&r2=806261&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-326/src/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java (original)
+++ hadoop/hdfs/branches/HDFS-326/src/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java Thu Aug 20 17:13:41 2009
@@ -78,6 +78,7 @@
 import org.apache.hadoop.security.authorize.RefreshAuthorizationPolicyProtocol;
 import org.apache.hadoop.security.authorize.ServiceAuthorizationManager;
 import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.hadoop.util.Service;
 import org.apache.hadoop.util.ServicePlugin;
 import org.apache.hadoop.util.StringUtils;
 
@@ -115,7 +116,7 @@
  * secondary namenodes or rebalancing processes to get partial namenode's
  * state, for example partial blocksMap etc.
  **********************************************************/
-public class NameNode implements ClientProtocol, DatanodeProtocol,
+public class NameNode extends Service implements ClientProtocol, DatanodeProtocol,
                                  NamenodeProtocol, FSConstants,
                                  RefreshAuthorizationPolicyProtocol {
   static{
@@ -369,7 +370,7 @@
   }
 
   /**
-   * Start NameNode.
+   * Create a NameNode.
    * <p>
    * The name-node can be started with one of the following startup options:
    * <ul> 
@@ -400,12 +401,90 @@
   }
 
   protected NameNode(Configuration conf, NamenodeRole role) throws IOException {
+    super(conf);
     this.role = role;
-    try {
-      initialize(conf);
-    } catch (IOException e) {
-      this.stop();
-      throw e;
+  }
+
+  /**
+   * The toString operator returns the super class name/id, and the state. This
+   * gives all services a slightly useful message in a debugger or test report
+   *
+   * @return a string representation of the object.
+   */
+  @Override
+  public String toString() {
+    return super.toString() 
+            + (httpAddress != null ? (" at " + httpAddress + " , ") : "")
+            + (server != null ? (" IPC " + server.getListenerAddress()) : "");
+  }
+
+  /////////////////////////////////////////////////////
+  // Service Lifecycle and other methods
+  /////////////////////////////////////////////////////
+  
+  /**
+   * {@inheritDoc}
+   *
+   * @return "NameNode"
+   */
+  @Override
+  public String getServiceName() {
+    return "NameNode";
+  }
+  
+  /**
+   * This method does all the startup. It is invoked from {@link #start()} when
+   * needed.
+   *
+   * This implementation delegates all the work to the (overridable)
+   * {@link #initialize(Configuration)} method, then calls
+   * {@link #setServiceState(ServiceState)} to mark the service as live.
+   * Any subclasses that do not consider themsevles to be live once 
+   * any subclassed initialize method has returned should override the method
+   * {@link #goLiveAtTheEndOfStart()} to change that behavior.
+   * @throws IOException for any problem.
+   */
+  @Override
+  protected void innerStart() throws IOException {
+    initialize(getConf());
+    if(goLiveAtTheEndOfStart()) {
+      setServiceState(ServiceState.LIVE);
+    }
+  }
+
+  /**
+   * Override point: should the NameNode enter the live state at the end of
+   * the {@link #innerStart()} operation?
+   * @return true if the service should enter the live state at this point,
+   * false to leave the service in its current state.
+   */
+  protected boolean goLiveAtTheEndOfStart() {
+    return true;
+  }
+
+  /**
+   * {@inheritDoc}.
+   *
+   * This implementation checks for the name system being non-null and live
+   *
+   * @param status status response to build up
+   * @throws IOException       for IO failure; this will be caught and included
+   * in the status message
+   */
+  @Override
+  public void innerPing(ServiceStatus status) throws IOException {
+    if (namesystem == null) {
+      status.addThrowable(new LivenessException("No name system"));
+    } else {
+      try {
+        namesystem.ping();
+      } catch (IOException e) {
+        status.addThrowable(e);
+      }
+    }
+    if (httpServer == null || !httpServer.isAlive()) {
+      status.addThrowable(
+              new LivenessException("NameNode HttpServer is not running"));
     }
   }
 
@@ -415,15 +494,33 @@
    */
   public void join() {
     try {
-      this.server.join();
+      if (server != null) {
+        server.join();
+      }
     } catch (InterruptedException ie) {
     }
   }
 
   /**
    * Stop all NameNode threads and wait for all to finish.
+   * <p/>
+   * Retained for backwards compatibility.
+   */
+  public final void stop() {
+    closeQuietly();
+  }
+
+  /**
+   * {@inheritDoc} 
+   * <p/>
+   * To shut down, this service stops all NameNode threads and
+   * waits for them to finish. It also stops the metrics.
+   * @throws IOException for any IO problem
    */
-  public void stop() {
+  @Override
+  protected synchronized void innerClose() throws IOException {
+    LOG.info("Closing " + getServiceName());
+
     if (stopRequested)
       return;
     stopRequested = true;
@@ -441,14 +538,23 @@
     } catch (Exception e) {
       LOG.error(StringUtils.stringifyException(e));
     }
-    if(namesystem != null) namesystem.close();
-    if(emptier != null) emptier.interrupt();
-    if(server != null) server.stop();
+    if(namesystem != null) {
+      namesystem.close();
+    }
+    if(emptier != null) {
+      emptier.interrupt();
+      emptier = null;
+    }
+    if(server != null) {
+      server.stop();
+      server = null;
+    }
     if (myMetrics != null) {
       myMetrics.shutdown();
     }
     if (namesystem != null) {
       namesystem.shutdown();
+      namesystem = null;
     }
   }
   
@@ -1160,9 +1266,13 @@
         return null; // avoid javac warning
       case BACKUP:
       case CHECKPOINT:
-        return new BackupNode(conf, startOpt.toNodeRole());
+        BackupNode backupNode = new BackupNode(conf, startOpt.toNodeRole());
+        startService(backupNode);
+        return backupNode;
       default:
-        return new NameNode(conf);
+        NameNode nameNode = new NameNode(conf);
+        startService(nameNode);
+        return nameNode;
     }
   }
     

Modified: hadoop/hdfs/branches/HDFS-326/src/java/org/apache/hadoop/hdfs/server/namenode/PendingReplicationBlocks.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-326/src/java/org/apache/hadoop/hdfs/server/namenode/PendingReplicationBlocks.java?rev=806261&r1=806260&r2=806261&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-326/src/java/org/apache/hadoop/hdfs/server/namenode/PendingReplicationBlocks.java (original)
+++ hadoop/hdfs/branches/HDFS-326/src/java/org/apache/hadoop/hdfs/server/namenode/PendingReplicationBlocks.java Thu Aug 20 17:13:41 2009
@@ -131,6 +131,15 @@
   }
 
   /**
+   * Test for the replicator being alive.
+   * @return true if the thread is running.
+   */
+  boolean isAlive() {
+    Daemon daemon = timerThread;
+    return daemon != null && daemon.isAlive();
+  }
+
+  /**
    * An object that contains information about a block that 
    * is being replicated. It records the timestamp when the 
    * system started replicating the most recent copy of this

Modified: hadoop/hdfs/branches/HDFS-326/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestReplicationPolicy.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/branches/HDFS-326/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestReplicationPolicy.java?rev=806261&r1=806260&r2=806261&view=diff
==============================================================================
--- hadoop/hdfs/branches/HDFS-326/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestReplicationPolicy.java (original)
+++ hadoop/hdfs/branches/HDFS-326/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestReplicationPolicy.java Thu Aug 20 17:13:41 2009
@@ -58,6 +58,7 @@
       CONF.set("dfs.http.address", "0.0.0.0:0");
       NameNode.format(CONF);
       namenode = new NameNode(CONF);
+      NameNode.startService(namenode);
     } catch (IOException e) {
       e.printStackTrace();
       throw (RuntimeException)new RuntimeException().initCause(e);