You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by st...@apache.org on 2009/08/17 18:24:09 UTC

svn commit: r805030 - in /hadoop/common/branches/HADOOP-6194: ./ src/java/org/apache/hadoop/http/ src/java/org/apache/hadoop/io/ src/java/org/apache/hadoop/security/authorize/ src/java/org/apache/hadoop/util/ src/test/core/org/apache/hadoop/io/ src/tes...

Author: stevel
Date: Mon Aug 17 16:24:09 2009
New Revision: 805030

URL: http://svn.apache.org/viewvc?rev=805030&view=rev
Log:
HADOOP-6194:  Service lifecycle

Added:
    hadoop/common/branches/HADOOP-6194/ivybuild.xml
    hadoop/common/branches/HADOOP-6194/src/java/org/apache/hadoop/io/ThrowableWritable.java
    hadoop/common/branches/HADOOP-6194/src/java/org/apache/hadoop/util/MockService.java
    hadoop/common/branches/HADOOP-6194/src/java/org/apache/hadoop/util/Service.java
    hadoop/common/branches/HADOOP-6194/src/test/core/org/apache/hadoop/io/TestThrowableWritable.java
    hadoop/common/branches/HADOOP-6194/src/test/core/org/apache/hadoop/util/TestServiceLifecycle.java
Modified:
    hadoop/common/branches/HADOOP-6194/src/java/org/apache/hadoop/http/HttpServer.java
    hadoop/common/branches/HADOOP-6194/src/java/org/apache/hadoop/security/authorize/ConfiguredPolicy.java

Added: hadoop/common/branches/HADOOP-6194/ivybuild.xml
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HADOOP-6194/ivybuild.xml?rev=805030&view=auto
==============================================================================
--- hadoop/common/branches/HADOOP-6194/ivybuild.xml (added)
+++ hadoop/common/branches/HADOOP-6194/ivybuild.xml Mon Aug 17 16:24:09 2009
@@ -0,0 +1,367 @@
+<?xml version="1.0"?>
+<project name="hadoop-core" 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 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" />
+
+
+  <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="build.ivy.maven.pom" location="${build.ivy.maven.dir}/hadoop-core-${hadoop.version}.pom" />
+    <property name="build.ivy.maven.jar" location="${build.ivy.maven.dir}/hadoop-core-${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}-core.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="HADOOP-6194"/>
+    <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 commit all changes to the branch
+  -->
+  <target name="svn-commit" depends="svn-init"
+      description="commit to the branch"  >
+    <property name="message" value="Service lifecycle" />
+    <property name="full.message" value="${issue}:  ${message}" />
+    <svn>
+      <arg value="commit" />
+      <arg value="-m"/>
+      <arg value="${full.message}"/>
+    </svn>
+  </target>
+
+
+  <!--
+  svn diff \
+  https://svn.apache.org/repos/asf/hadoop/common/trunk \
+  https://svn.apache.org/repos/asf/hadoop/common/branches/HADOOP-6194
+  -->
+  <target name="svn-diff-trunk" depends="svn-init"
+      description="diff against trunk"  >
+    <svn>
+      <arg value="diff" />
+      <arg value="${trunk}"/>
+      <arg value="${branch}"/>
+    </svn>
+  </target>
+
+  <!--
+  create a changelist; this is what you need for patches
+  -->
+  <target name="svn-create-changelist" depends="svn-init"
+      description="Create a changelist of everything we want in the big patch"  >
+    <property name="core/hadoop" value="src/java/org/apache/hadoop" />
+    <property name="test/core"
+        value="src/test/core/org/apache/hadoop" />
+    <svn>
+      <arg value="changelist"/>
+      <arg value="${issue}"/>
+      <!-- core -->
+      <!-- question this one -->
+      <!--
+      <arg value="${core/hadoop}/security/authorize/ConfiguredPolicy.java" />
+      -->
+      <arg value="${core/hadoop}/http/HttpServer.java" />
+      <arg value="${core/hadoop}/io/ThrowableWritable.java" />
+      <arg value="${core/hadoop}/util/Service.java" />
+      <arg value="${core/hadoop}/util/MockService.java" />
+      <arg value="${test/core}/io/TestThrowableWritable.java" />
+      <arg value="${test/core}/util/TestServiceLifecycle.java" />
+    </svn>
+  </target>
+
+
+  <!--
+  
+  -->
+  <target name="svn-diff-changelist" 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/common/branches/HADOOP-6194/src/java/org/apache/hadoop/http/HttpServer.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HADOOP-6194/src/java/org/apache/hadoop/http/HttpServer.java?rev=805030&r1=805029&r2=805030&view=diff
==============================================================================
--- hadoop/common/branches/HADOOP-6194/src/java/org/apache/hadoop/http/HttpServer.java (original)
+++ hadoop/common/branches/HADOOP-6194/src/java/org/apache/hadoop/http/HttpServer.java Mon Aug 17 16:24:09 2009
@@ -468,7 +468,11 @@
           // then try the next port number.
           if (ex instanceof BindException) {
             if (!findPort) {
-              throw (BindException) ex;
+              BindException be = new BindException(
+                      "Port in use: " + listener.getHost()
+                              + ":" + listener.getPort());
+              be.initCause(ex);
+              throw be;
             }
           } else {
             LOG.info("HttpServer.start() threw a non Bind IOException"); 
@@ -500,6 +504,14 @@
   }
 
   /**
+   * Test for the availability of the web server
+   * @return true if the web server is started, false otherwise
+   */
+  public boolean isAlive() {
+    return webServer.isStarted();
+  }
+
+  /**
    * A very simple servlet to serve up a text representation of the current
    * stack traces. It both returns the stacks to the caller and logs them.
    * Currently the stack traces are done sequentially rather than exactly the

Added: hadoop/common/branches/HADOOP-6194/src/java/org/apache/hadoop/io/ThrowableWritable.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HADOOP-6194/src/java/org/apache/hadoop/io/ThrowableWritable.java?rev=805030&view=auto
==============================================================================
--- hadoop/common/branches/HADOOP-6194/src/java/org/apache/hadoop/io/ThrowableWritable.java (added)
+++ hadoop/common/branches/HADOOP-6194/src/java/org/apache/hadoop/io/ThrowableWritable.java Mon Aug 17 16:24:09 2009
@@ -0,0 +1,278 @@
+/**
+ * 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.io;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+/** This writable component builds from a throwable */
+
+public final class ThrowableWritable implements Writable {
+
+  /** throwable classname */
+  private String classname;
+
+  /** throwable message */
+
+  private String message;
+  /** cause: may be null */
+  private ThrowableWritable cause;
+
+  /**
+   * Stack trace as string; will be null when an empty element is created,
+   * otherwise it will be an array of length zero.
+   */
+  private String[] stack;
+
+
+  /**
+   * Empty constructor.
+   * Only use this when you are planning to deserialize data, as the object is
+   * otherwise incomplete.
+   */
+  public ThrowableWritable() {
+  }
+
+
+  /**
+   * Construct a lightweight throwable writeable with no stack trace; and the
+   * message passed in
+   *
+   * @param message message to use
+   */
+  public ThrowableWritable(String message) {
+    this.message = message;
+    stack = new String[0];
+    classname = "";
+  }
+
+  /**
+   * recursively construct from a throwable chain.
+   *
+   * @param thrown The throwable chain to build this writeable from.
+   */
+  public ThrowableWritable(Throwable thrown) {
+    classname = thrown.getClass().getName();
+    message = thrown.getMessage();
+
+    StackTraceElement[] st = thrown.getStackTrace();
+    if (st != null) {
+      int sl = st.length;
+      stack = new String[sl];
+      for (int i = 0; i < sl; i++) {
+        stack[i] = st[i].toString();
+      }
+    } else {
+      stack = new String[0];
+    }
+    Throwable rootCause = thrown.getCause();
+    if (rootCause != null && rootCause != thrown) {
+      cause = new ThrowableWritable(rootCause);
+    }
+  }
+
+  /**
+   * Copy constructor.
+   *
+   * @param that the original instance to copy
+   */
+  public ThrowableWritable(ThrowableWritable that) {
+    classname = that.classname;
+    message = that.message;
+    //copy stack trace
+    if (that.stack == null) {
+      stack = new String[0];
+    } else {
+      int l = that.stack.length;
+      stack = new String[l];
+      System.arraycopy(that.stack, 0, stack, 0, l);
+    }
+    //copy any nested cause
+    if (that.cause != null) {
+      cause = new ThrowableWritable(that.cause);
+    }
+  }
+
+
+  /**
+   * Get the classname of the underlying throwable
+   *
+   * @return the classname of the original throwable
+   */
+  public String getClassname() {
+    return classname;
+  }
+
+  /**
+   * Get the text string this instance was constructed with
+   *
+   * @return the message of the underlying throwable
+   */
+  public String getMessage() {
+    return message;
+  }
+
+  /**
+   * Get any nested cause of the exception
+   *
+   * @return any nested cause as another ThrowableWritable -or null
+   */
+  public ThrowableWritable getCause() {
+    return cause;
+  }
+
+  /**
+   * Get the stack trace of the original throwable. It may be of size 0.
+   *
+   * @return the stack trace converted to strings
+   */
+  public String[] getStack() {
+    return stack;
+  }
+
+
+  /**
+   * determine (recursively) the depth of this Throwable chain
+   *
+   * @return a number equal to or greater than 1
+   */
+  public int getDepth() {
+    return 1 + (cause == null ? 0 : cause.getDepth());
+  }
+
+  /**
+   * {@inheritDoc}
+   *
+   * @param out <code>DataOutput</code> to serialize this object into.
+   *
+   * @throws IOException IO trouble
+   */
+  public void write(DataOutput out) throws IOException {
+    out.writeUTF(classname);
+    out.writeUTF(message);
+    if (stack != null) {
+      out.writeInt(stack.length);
+      for (String call : stack) {
+        out.writeUTF(call);
+      }
+    } else {
+      out.writeInt(0);
+    }
+    //look for a cause
+    boolean hasCause = cause != null;
+    out.writeBoolean(hasCause);
+    if (hasCause) {
+      //recursively write it
+      cause.write(out);
+    }
+  }
+
+  /**
+   * {@inheritDoc}
+   *
+   * @param in <code>DataInput</code> to deseriablize this object from.
+   *
+   * @throws IOException IO trouble
+   */
+  public void readFields(DataInput in) throws IOException {
+    classname = in.readUTF();
+    message = in.readUTF();
+    int stackLength = in.readInt();
+    if (stack == null || stack.length != stackLength) {
+      //create a new stack array
+      stack = new String[stackLength];
+    }
+    //read in the stack
+    for (int i = 0; i < stackLength; i++) {
+      stack[i] = in.readUTF();
+    }
+    //look for any nested cause
+    boolean hasCause = in.readBoolean();
+    if (hasCause) {
+      if (cause == null) {
+        cause = new ThrowableWritable();
+      }
+      cause.readFields(in);
+    }
+  }
+
+  /**
+   * {@inheritDoc}
+   *
+   * @throws CloneNotSupportedException this should not happen
+   */
+  @SuppressWarnings({"CloneDoesntCallSuperClone"})
+  @Override
+  protected Object clone() throws CloneNotSupportedException {
+    return new ThrowableWritable(this);
+  }
+
+  /**
+   * {@inheritDoc}
+   *
+   * The classname and message are used for equality
+   */
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (o == null || getClass() != o.getClass()) return false;
+
+    ThrowableWritable that = (ThrowableWritable) o;
+
+    if (classname != null ? !classname.equals(that.classname) : that.classname != null)
+      return false;
+    return !(message != null ? !message.equals(that.message) : that.message != null);
+
+  }
+
+  /**
+   * {@inheritDoc}
+   *
+   * The classname and message are used in the hash
+   */
+  @Override
+  public int hashCode() {
+    int result = classname != null ? classname.hashCode() : 0;
+    result = 31 * result + (message != null ? message.hashCode() : 0);
+    return result;
+  }
+
+  /**
+   * Return the classname and message in the format classname: message The
+   * output is designed to resemble that of {@link Throwable#toString()} if the
+   * message and classname are both set. If only the message is set, only that
+   * is printed.
+   *
+   * @return a string representation of the object.
+   */
+  @Override
+  public String toString() {
+    StringBuilder builder = new StringBuilder();
+    if (classname != null) {
+      builder.append(classname);
+      if (!classname.isEmpty()) {
+        builder.append(": ");
+      }
+    }
+    if (message != null) {
+      builder.append(message);
+    }
+    return builder.toString();
+  }
+}

Modified: hadoop/common/branches/HADOOP-6194/src/java/org/apache/hadoop/security/authorize/ConfiguredPolicy.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HADOOP-6194/src/java/org/apache/hadoop/security/authorize/ConfiguredPolicy.java?rev=805030&r1=805029&r2=805030&view=diff
==============================================================================
--- hadoop/common/branches/HADOOP-6194/src/java/org/apache/hadoop/security/authorize/ConfiguredPolicy.java (original)
+++ hadoop/common/branches/HADOOP-6194/src/java/org/apache/hadoop/security/authorize/ConfiguredPolicy.java Mon Aug 17 16:24:09 2009
@@ -22,6 +22,8 @@
 import java.security.Policy;
 import java.security.Principal;
 import java.security.ProtectionDomain;
+import java.security.CodeSource;
+import java.security.Permissions;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Map;
@@ -77,7 +79,20 @@
     return super.implies(domain, permission);
   }
 
+  /**
+   * {@inheritDoc}
+   * @return a writable permission collection
+   */
   @Override
+  public PermissionCollection getPermissions(CodeSource codesource) {
+    return new Permissions();
+  }
+
+/**
+ * {@inheritDoc}
+ * @return a writable permission collection
+ */
+@Override
   public PermissionCollection getPermissions(ProtectionDomain domain) {
     PermissionCollection permissionCollection = super.getPermissions(domain);
     for (Principal principal : domain.getPrincipals()) {
@@ -153,4 +168,14 @@
       LOG.debug("Policy - Adding  " + permission + " to " + principal);
     }
   }
+
+  /**
+   * For debugging: identify ourselves and the policyproviders 
+   *
+   * @return a string representation of the object.
+   */
+  @Override
+  public String toString() {
+    return "Hadoop ConfiguredPolicy " + super.toString() + " Policy provider "+ policyProvider;
+  }
 }

Added: hadoop/common/branches/HADOOP-6194/src/java/org/apache/hadoop/util/MockService.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HADOOP-6194/src/java/org/apache/hadoop/util/MockService.java?rev=805030&view=auto
==============================================================================
--- hadoop/common/branches/HADOOP-6194/src/java/org/apache/hadoop/util/MockService.java (added)
+++ hadoop/common/branches/HADOOP-6194/src/java/org/apache/hadoop/util/MockService.java Mon Aug 17 16:24:09 2009
@@ -0,0 +1,156 @@
+/**
+ * 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.util;
+
+import org.apache.hadoop.conf.Configuration;
+
+import java.io.IOException;
+
+/**
+ * A mock service that can be set to fail in different parts of its lifecycle,
+ * and which counts the number of times its inner classes changed state.
+ */
+
+public class MockService extends Service {
+
+  /**
+   * Build from an empty configuration
+   */
+  public MockService() {
+    super(new Configuration());
+  }
+
+  /**
+   * Build from a configuration file
+   * @param conf
+   */
+  public MockService(Configuration conf) {
+    super(conf);
+  }
+
+  private boolean failOnStart, failOnPing, failOnClose;
+  private boolean goLiveInStart = true;
+  private boolean closed = true;
+  private volatile int stateChangeCount = 0;
+  private volatile int pingCount = 0;
+
+  public void setFailOnStart(boolean failOnStart) {
+    this.failOnStart = failOnStart;
+  }
+
+  public void setFailOnPing(boolean failOnPing) {
+    this.failOnPing = failOnPing;
+  }
+
+  public void setGoLiveInStart(boolean goLiveInStart) {
+    this.goLiveInStart = goLiveInStart;
+  }
+
+  public void setFailOnClose(boolean failOnClose) {
+    this.failOnClose = failOnClose;
+  }
+
+  public boolean isClosed() {
+    return closed;
+  }
+
+  /**
+   * Go live
+   *
+   * @throws ServiceStateException if we were not in a state to do so
+   */
+  public void goLive() throws ServiceStateException {
+    enterLiveState();
+  }
+
+  /**
+   * {@inheritDoc}
+   * @throws IOException  if {@link #failOnStart is set}
+   */
+  @Override
+  protected void innerStart() throws IOException {
+    if (failOnStart) {
+      throw new MockServiceException("failOnStart");
+    }
+    if (goLiveInStart) {
+      goLive();
+    }
+  }
+
+  /**
+   * {@inheritDoc}
+   * @throws IOException if {@link #failOnPing is set} @param status
+   */
+  @Override
+  protected void innerPing(ServiceStatus status) throws IOException {
+    pingCount++;
+    if (failOnPing) {
+      throw new MockServiceException("failOnPing");
+    }
+  }
+
+  /**
+   * {@inheritDoc}
+   *
+   * @throws IOException if {@link #failOnClose} is true
+   */
+  protected void innerClose() throws IOException {
+    closed = true;
+    if (failOnClose) {
+      throw new MockServiceException("failOnClose");
+    }
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  protected void onStateChange(ServiceState oldState,
+                               ServiceState newState) {
+    super.onStateChange(oldState, newState);
+    stateChangeCount++;
+  }
+
+  /**
+   * {@inheritDoc}
+   *
+   * A public method do change state
+   */
+  public void changeState(ServiceState state)
+          throws ServiceStateException {
+    setServiceState(state);
+  }
+
+  public int getStateChangeCount() {
+    return stateChangeCount;
+  }
+
+  public int getPingCount() {
+    return pingCount;
+  }
+
+  /**
+   * An exception to indicate we have triggered a mock event
+   */
+  static class MockServiceException extends IOException {
+
+    private MockServiceException(String message) {
+      super(message);
+    }
+  }
+}

Added: hadoop/common/branches/HADOOP-6194/src/java/org/apache/hadoop/util/Service.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HADOOP-6194/src/java/org/apache/hadoop/util/Service.java?rev=805030&view=auto
==============================================================================
--- hadoop/common/branches/HADOOP-6194/src/java/org/apache/hadoop/util/Service.java (added)
+++ hadoop/common/branches/HADOOP-6194/src/java/org/apache/hadoop/util/Service.java Mon Aug 17 16:24:09 2009
@@ -0,0 +1,1016 @@
+/*
+ * Copyright  2008 The Apache Software Foundation
+ *
+ *  Licensed 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.util;
+
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.io.Closeable;
+import java.util.Date;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * This is the base class for services that can be deployed. A service is any
+ * Hadoop class that has a standard lifecycle
+ *
+ * The lifecycle of a Service is:
+ *
+ * <ol>
+ *
+ * <li>Component is Created, enters the {@link ServiceState#CREATED} state.
+ * This happens in the constructor. </li>
+ *
+ * <li>Component is started
+ * through a call to {@link Service#start()} ()}. If successful, it enters the
+ * {@link ServiceState#STARTED} state. If not, it enters the {@link
+ * ServiceState#FAILED} state. </li>
+ *
+ * <li>Once the component considers itself
+ * fully started, it enters the {@link ServiceState#LIVE} state. This implies it
+ * is providing a service to external callers. </li>
+ *
+ * </ol>
+ *
+ * From any state, the service can be terminated/closed through a call to
+ * {@link Service#close()}, which may throw an {@link IOException}, or
+ * {@link Service#closeQuietly()}, which catched and logs any such exception.
+ * These are idempotent calls, and will place the service in the
+ * {@link ServiceState#CLOSED}, terminated  state, after which
+ * it can no longer be used.
+ *
+ * To implement a Service.
+ *
+ * <ol>
+ *
+ * <li>Subclass this class</li>
+ *
+ * <li>Avoid doing any initialization/startup in the constructors, as this
+ * breaks the lifecycle and prevents subclassing. </li>
+ *
+ * <li>If the service wishes to declare itself as having failed, so that
+ * {@link #ping()} operations automatically fail, call
+ * {@link #enterFailedState(Throwable)} to enter the failed state.</li>
+ *
+ * <li>Override the {@link #innerStart()} method to start the service, including
+ * starting any worker threads.</li>
+ *
+ * <li>In the {@link #innerStart()} method, if the service is immediately live
+ * to external callers, call {@link #enterLiveState()} to mark the service as
+ * live.</li>
+
+ * <li>If startup is performed in separate threads, and includes bootstrap work,
+ * call the  {@link #enterLiveState()} in the separate thread <i>when the
+ * service is ready</i></li>
+ *
+ * <li>Override {@link #innerPing(ServiceStatus)} with any health checks that a service
+ * can perform to check that it is still "alive". These should be short lasting
+ * and non-side effecting. Simple checks for valid data structures and live
+ * worker threads are good examples. When the service thinks that something
+ * has failed, throw an IOException with a meaningful error message!
+ * </li>
+ *
+ * <li>Override {@link #innerClose()} to perform all shutdown logic.
+ * Be robust here and shut down cleanly even if the service did not start up
+ * completely. Do not assume all fields are non-null</li>
+ *
+ * You should not need to worry about making these overridden methods
+ * synchronized, as they are only called when a service has entered a specific
+ * state -which is synchronized. Except for {@link #innerPing(ServiceStatus)} ,
+ * each method will only be called at most once in the life of a service instance.
+ * However, because findbugs can flag synchronization warnings, it is often
+ * simplest and safest to mark the innerX operations as synchronized.
+ */
+
+public abstract class Service extends Configured implements Closeable {
+
+  private static final Log LOG = LogFactory.getLog(Service.class);
+
+  /**
+   * The initial state of a service is {@link ServiceState#CREATED}
+   */
+  private volatile ServiceState serviceState = ServiceState.CREATED;
+
+  /**
+   * when did the state change?
+   */
+  private volatile Date lastStateChange = new Date();
+
+  /**
+   * A root cause for failure. May be null.
+   */
+  private Throwable failureCause;
+
+  /**
+   * Error string included in {@link ServiceStateException} exceptions
+   * when an operation is applied to a service that is not in the correct
+   * state for it.
+   * value: {@value}
+   */
+  public static final String ERROR_WRONG_STATE = " is in the wrong state.";
+
+  /**
+   * Error string included in {@link ServiceStateException} exceptions
+   * when a service with a null configuration is started
+   * value: {@value}
+   */
+  public static final String ERROR_NO_CONFIGURATION
+          = "Cannot initialize when unconfigured";
+
+  /**
+   * Construct a service with no configuration; one must be called with {@link
+   * #setConf(Configuration)} before the service is started
+   */
+  protected Service() {
+  }
+
+  /**
+   * Construct a Configured service
+   *
+   * @param conf the configuration
+   */
+  protected Service(Configuration conf) {
+    super(conf);
+  }
+
+  /**
+   * Start any work (usually in separate threads).
+   *
+   * When completed, the service will be in the {@link ServiceState#STARTED}
+   * state, or may have already transited to the {@link ServiceState#LIVE}
+   * state
+   *
+   * Subclasses must implement their work in {@link #innerStart()}, leaving the
+   * start() method to manage state checks and changes.
+   *
+   * @throws IOException           for any failure
+   * @throws ServiceStateException when the service is not in a state from which
+   *                               it can enter this state.
+   */
+  public void start() throws IOException {
+    synchronized (this) {
+      //this request is idempotent on either live or starting states; either
+      //state is ignored
+      ServiceState currentState = getServiceState();
+      if (currentState == ServiceState.LIVE ||
+              currentState == ServiceState.STARTED) {
+        return;
+      }
+      if (getConf() == null) {
+        throw new ServiceStateException(ERROR_NO_CONFIGURATION,
+                getServiceState());
+      }
+      //check and change states
+      enterState(ServiceState.STARTED);
+    }
+    try {
+      innerStart();
+    } catch (IOException e) {
+      enterFailedState(e);
+      throw e;
+    }
+  }
+
+  /**
+   * Ping: checks that a component considers itself live.
+   *
+   * This may trigger a health check in which the service probes its
+   * constituent parts to verify that they are themselves live.
+   * The base implementation considers any state other than
+   * {@link ServiceState#FAILED} and {@link ServiceState#CLOSED}
+   * to be valid, so it is OK to ping a
+   * component that is still starting up. However, in such situations, the inner
+   * ping health tests are skipped, because they are generally irrelevant.
+   *
+   * Subclasses should not normally override this method, but instead override
+   * {@link #innerPing(ServiceStatus)} with extra health checks that will only
+   * be called when a system is actually live.
+   * @return the current service state.
+   * @throws IOException           for any ping failure
+   * @throws ServiceStateException if the component is in a wrong state.
+   */
+  protected ServiceStatus ping() throws IOException {
+    ServiceStatus status = new ServiceStatus(this);
+    ServiceState state = status.getState();
+    if (state == ServiceState.LIVE) {
+      try {
+        innerPing(status);
+      } catch (Throwable thrown) {
+        //TODO: what happens whenthe ping() returns >0 causes of failure but 
+        //doesn't throw an exception -this method will not get called. Is 
+        //that what we want?
+        status = onInnerPingFailure(status, thrown);
+      }
+    } else {
+      //ignore the ping
+      LOG.debug("ignoring ping request while in state " + state);
+      //but tack on any non-null failure cause, which may be a valid value
+      //in FAILED or TERMINATED states.
+      status.addThrowable(getFailureCause());
+    }
+    return status;
+  }
+
+  /**
+   * This is an override point for services -handle failure of the inner
+   * ping operation.
+   * The base implementation calls {@link #enterFailedState(Throwable)} and then
+   * updates the service status with the (new) state and the throwable
+   * that was caught.
+   * @param currentStatus the current status structure
+   * @param thrown the exception from the failing ping.
+   * @return an updated service status structure.
+   * @throws IOException for IO problems
+   */
+  protected ServiceStatus onInnerPingFailure(ServiceStatus currentStatus,
+                                             Throwable thrown) 
+          throws IOException {
+    //something went wrong
+    //mark as failed
+    //TODO: don't enter failed state on a failing ping? Just report the event
+    //to the caller?
+    enterFailedState(thrown);
+    //update the state
+    currentStatus.updateState(this);
+    currentStatus.addThrowable(thrown);
+    //and return the exception.
+    return currentStatus;
+  }
+
+  /**
+   * Convert any exception to an {@link IOException}
+   * If it already is an IOException, the exception is
+   * returned as is. If it is anything else, it is wrapped, with
+   * the original message retained.
+   * @param thrown the exception to forward
+   * @return an IOException representing or containing the forwarded exception
+   */
+  @SuppressWarnings({"ThrowableInstanceNeverThrown"})
+  protected IOException forwardAsIOException(Throwable thrown) {
+    IOException newException;
+    if(thrown instanceof IOException) {
+      newException = (IOException) thrown;
+    } else {
+      IOException ioe = new IOException(thrown.toString());
+      ioe.initCause(thrown);
+      newException = ioe;
+    }
+    return newException;
+  }
+
+
+  /**
+   * Test for a service being in the {@link ServiceState#LIVE} or {@link
+   * ServiceState#STARTED}
+   *
+   * @return true if the service is in one of the two states.
+   */
+  public final boolean isRunning() {
+    ServiceState currentState = getServiceState();
+    return currentState == ServiceState.STARTED
+            || currentState == ServiceState.LIVE;
+  }
+
+  /**
+   * Shut down. This must be idempotent and turn errors into log/warn events -do
+   * your best to clean up even in the face of adversity. This method should be
+   * idempotent; if already terminated, return. Similarly, do not fail if the
+   * component never actually started.
+   *
+   * The implementation calls {@link #close()} and then
+   * {@link #logExceptionDuringQuietClose(Throwable)} if that method throws
+   * any exception.
+   */
+  public final void closeQuietly() {
+    try {
+      close();
+    } catch (Throwable e) {
+      logExceptionDuringQuietClose(e);
+    }
+  }
+
+  /**
+   * Closes this service. Subclasses are free to throw an exception, but
+   * they are expected to make a best effort attempt to close the service
+   * down as thoroughly as possible.
+   *
+   * @throws IOException if an I/O error occurs
+   */
+  public void close() throws IOException {
+    if (enterState(ServiceState.CLOSED)) {
+      innerClose();
+    }
+  }
+
+  /**
+   * This is a method called when exceptions are being logged and swallowed
+   * during termination. It logs the event at the error level.
+   *
+   * Subclasses may override this to do more advanced error handling/logging.
+   *
+   * @param thrown whatever was thrown
+   */
+  protected void logExceptionDuringQuietClose(Throwable thrown) {
+    LOG.error("Exception during termination: " + thrown,
+            thrown);
+  }
+
+  /**
+   * This method is designed for overriding, with subclasses implementing
+   * startup logic inside it. It is only called when the component is entering
+   * the running state; and will be called once only.
+   *
+   * When the work in here is completed, the component should set the service
+   * state to {@link ServiceState#LIVE} to indicate the service is now live.
+   *
+   * @throws IOException for any problem.
+   */
+  protected void innerStart() throws IOException {
+  }
+
+
+  /**
+   * This method is designed for overriding, with subclasses implementing health
+   * tests inside it.
+   *
+   * It is invoked whenever the component is called with {@link Service#ping()}
+   * and the call is not rejected.
+   * @param status the service status, which can be updated
+   * @throws IOException for any problem.
+   */
+  protected void innerPing(ServiceStatus status) throws IOException {
+  }
+
+  /**
+   * This method is designed for overriding, with subclasses implementing
+   * termination logic inside it.
+   *
+   * It is only called when the component is entering the closed state; and
+   * will be called once only.
+   *
+   * @throws IOException exceptions which will be logged
+   */
+  protected void innerClose() throws IOException {
+
+  }
+
+  /**
+   * Get the current state of the service.
+   *
+   * @return the lifecycle state
+   */
+  public final ServiceState getServiceState() {
+    return serviceState;
+  }
+
+  /**
+   * This is the state transition graph represented as some nested switches.
+   * @return true if the transition is valid. For all states, the result when
+   * oldState==newState is false: that is not a transition, after all.
+   * @param oldState the old state of the service
+   * @param newState the new state
+   */
+  protected boolean isValidStateTransition(ServiceState oldState,
+                                           ServiceState newState) {
+    switch(oldState) {
+      case CREATED:
+        switch(newState) {
+          case STARTED:
+          case FAILED:
+          case CLOSED:
+            return true;
+          default:
+            return false;
+        }
+      case STARTED:
+        switch (newState) {
+          case LIVE:
+          case FAILED:
+          case CLOSED:
+            return true;
+          default:
+            return false;
+        }
+      case LIVE:
+        switch (newState) {
+          case STARTED:
+          case FAILED:
+          case CLOSED:
+            return true;
+          default:
+            return false;
+        }
+      case UNDEFINED:
+        //if we don't know where we were before (very, very unlikely), then
+        //let's get out of it
+        return true;
+      case FAILED:
+        //failure can only enter closed state
+        return newState == ServiceState.CLOSED;
+      case CLOSED:
+        //This is the end state. There is no exit.
+      default:
+        return false;
+    }
+  }
+
+  /**
+  * Set the service state.
+  * If there is a change in state, the {@link #lastStateChange} timestamp
+  * is updated and the {@link #onStateChange(ServiceState, ServiceState)} event
+  * is invoked.
+  * @param serviceState the new state
+  */
+  protected final void setServiceState(ServiceState serviceState) {
+    ServiceState oldState;
+    synchronized (this) {
+      oldState = this.serviceState;
+      this.serviceState = serviceState;
+    }
+    if (oldState != serviceState) {
+      lastStateChange = new Date();
+      onStateChange(oldState, serviceState);
+    }
+  }
+
+  /**
+   * Override point - a method called whenever there is a state change.
+   *
+   * The base class logs the event.
+   *
+   * @param oldState existing state
+   * @param newState new state.
+   */
+  protected void onStateChange(ServiceState oldState,
+                               ServiceState newState) {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("State Change: " + toString()
+              + " transitioned from state " + oldState + " to " + newState);
+    }
+  }
+
+  /**
+   * Enter a new state if that is permitted from the current state.
+   * Does nothing if we are in that state; throws an exception if the
+   * state transition is not permitted
+   * @param  newState  the new state
+   * @return true if the service transitioned into this state, that is, it was
+   *         not already in the state
+   * @throws ServiceStateException if the service is not in either state
+   */
+  protected final synchronized boolean enterState(ServiceState newState)
+          throws ServiceStateException {
+    return enterState(getServiceState(), newState);
+  }
+
+  /**
+   * Check that a service is in a required entry state, or already in the
+   * desired exit state.
+   *
+   * @param entryState the state that is needed. If set to {@link
+   *                   ServiceState#UNDEFINED} then the entry state is not
+   *                   checked.
+   * @param exitState  the state that is desired when we exit
+   * @return true if the service transitioned into this state, that is, it was
+   *         not already in the state
+   * @throws ServiceStateException if the service is not in either state
+   */
+  protected final synchronized boolean enterState(ServiceState entryState,
+                                            ServiceState exitState)
+          throws ServiceStateException {
+    ServiceState currentState = getServiceState();
+    if (currentState == exitState) {
+      return false;
+    }
+    validateStateTransition(entryState, exitState);
+    setServiceState(exitState);
+    return true;
+  }
+
+  /**
+   * Check that the state transition is valid
+   * @param entryState the entry state
+   * @param exitState the exit state 
+   * @throws ServiceStateException if the state transition is not allowed
+   */
+  protected final void validateStateTransition(ServiceState entryState,
+                                         ServiceState exitState)
+          throws ServiceStateException {
+    if(!isValidStateTransition(entryState, exitState)) {
+      throw new ServiceStateException(toString()
+              + ERROR_WRONG_STATE
+              + " The service cannot move from the state " + entryState
+              + "to the state " + exitState,
+              entryState);
+    }
+  }
+
+
+  /**
+   * Verify that a service is in a specific state
+   *
+   * @param state the state that is required.
+   * @throws ServiceStateException if the service is in the wrong state
+   */
+  public final void verifyServiceState(ServiceState state)
+          throws ServiceStateException {
+    verifyState(getServiceState(), state, ServiceState.UNDEFINED);
+  }
+
+  /**
+   * Verify that a service is in either of two specific states
+   *
+   * @param expected  the state that is expected.
+   * @param expected2 a second state, which can be left at {@link
+   *                  ServiceState#UNDEFINED} for "do not check this"
+   * @throws ServiceStateException if the service is in the wrong state
+   */
+  public final void verifyServiceState(ServiceState expected, ServiceState expected2)
+          throws ServiceStateException {
+    verifyState(getServiceState(), expected, expected2);
+  }
+
+  /**
+   * Internal state verification test
+   *
+   * @param currentState the current state
+   * @param expected     the state that is expected.
+   * @param expected2    a second state, which can be left at {@link
+   *                     ServiceState#UNDEFINED} for "do not check this"
+   * @throws ServiceStateException if the service is in the wrong state
+   */
+  protected final void verifyState(ServiceState currentState,
+                             ServiceState expected,
+                             ServiceState expected2)
+          throws ServiceStateException {
+    boolean expected2defined = expected2 != ServiceState.UNDEFINED;
+    if (!(currentState == expected ||
+            (expected2defined && currentState == expected2))) {
+      throw new ServiceStateException(toString()
+              + ERROR_WRONG_STATE
+              + " Expected " + expected
+              + (expected2defined ? (" or " + expected2) : " ")
+              + " but the actual state is " + currentState,
+              currentState);
+    }
+  }
+
+  /**
+   * Helper method to enter the {@link ServiceState#FAILED} state.
+   *
+   * Call this whenever the service considers itself to have failed in a
+   * non-restartable manner.
+   *
+   * If the service was already terminated or failed, this operation does
+   * not trigger a state change.
+   * @param cause the cause of the failure
+   */
+  public final void enterFailedState(Throwable cause) {
+    synchronized (this) {
+      if(failureCause == null) {
+        failureCause = cause;
+      }
+    }
+    if(!isTerminated()) {
+      setServiceState(ServiceState.FAILED);
+    }
+  }
+
+
+  /**
+   * Shortcut method to enter the {@link ServiceState#LIVE} state.
+   *
+   * Call this when a service considers itself live
+   * @return true if this state was entered, false if it was already in it
+   * @throws ServiceStateException if the service is not currently in the
+   * STARTED or LIVE states
+   */
+  protected final boolean enterLiveState() throws ServiceStateException {
+    return enterState(ServiceState.LIVE);
+  }
+
+  /**
+   * Test for the service being terminated; non-blocking
+   *
+   * @return true if the service is currently terminated
+   */
+  public boolean isTerminated() {
+    return getServiceState() == ServiceState.CLOSED;
+  }
+
+
+  /**
+   * Override point: the name of this service. This is used
+   * to construct human-readable descriptions
+   * @return the name of this service for string messages
+   */
+  public String getServiceName() {
+    return "Service";
+  }
+
+  /**
+  * 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 getServiceName() + " instance " + super.toString() + " in state "
+            + getServiceState();
+  }
+
+
+  /**
+   * Get the cause of failure -will be null if not failed, and may be
+   * null even after failure.
+   * @return the exception that triggered entry into the failed state.
+   *
+   */
+  public Throwable getFailureCause() {
+    return failureCause;
+  }
+
+  /**
+   * Initialize and start a service. If the service fails to come up, it is
+   * terminated.
+   *
+   * @param service the service to deploy
+   * @throws IOException on any failure to deploy
+   */
+  public static void startService(Service service)
+          throws IOException {
+    try {
+      service.start();
+    } catch (IOException e) {
+      //mark as failed
+      service.enterFailedState(e);
+      //we assume that the service really does know how to terminate
+      service.closeQuietly();
+      throw e;
+    } catch (Throwable t) {
+      //mark as failed
+      service.enterFailedState(t);
+      //we assume that the service really does know how to terminate
+      service.closeQuietly();
+      
+      if (t instanceof IOException) {
+        //rethrow any IOException
+        throw (IOException) t;
+      } else {
+        //and wrap any other exception in an IOException that is rethrown
+        throw (IOException) new IOException(t.toString()).initCause(t);
+      }
+    }
+  }
+
+  /**
+   * Terminate a service that is not null
+   *
+   * @param service a service to terminate
+   */
+  public static void closeQuietly(Service service) {
+    if (service != null) {
+      service.closeQuietly();
+    }
+  }
+
+  /**
+   * Terminate a service or other closeable that is not null
+   *
+   * @param closeable the object to close
+   */
+  public static void close(Closeable closeable) {
+    if (closeable != null) {
+      try {
+        closeable.close();
+      } catch (IOException e) {
+        LOG.info("when closing :" + closeable+ ":" + e, e);
+      }
+    }
+  }
+
+
+  /**
+   * An exception that indicates there is something wrong with the state of the
+   * service
+   */
+  public static class ServiceStateException extends IOException {
+    private ServiceState state;
+
+
+    /**
+     * Create a service state exception with a standard message {@link
+     * Service#ERROR_WRONG_STATE} including the string value of the owning
+     * service, and the supplied state value
+     *
+     * @param service owning service
+     * @param state current state
+     */
+    public ServiceStateException(Service service, ServiceState state) {
+      this(service.toString()
+              + ERROR_WRONG_STATE + " : " + state,
+              null,
+              state);
+    }
+
+    /**
+     * Constructs an Exception with the specified detail message and service
+     * state.
+     *
+     * @param message The detail message (which is saved for later retrieval by
+     *                the {@link #getMessage()} method)
+     * @param state   the current state of the service
+     */
+    public ServiceStateException(String message, ServiceState state) {
+      this(message, null, state);
+    }
+
+    /**
+     * Constructs an Exception with the specified detail message, cause and
+     * service state.
+     *
+     * @param message message
+     * @param cause   optional root cause
+     * @param state   the state of the component
+     */
+    public ServiceStateException(String message,
+                                 Throwable cause,
+                                 ServiceState state) {
+      super(message, cause);
+      this.state = state;
+    }
+
+    /**
+     * Construct an exception. The lifecycle state of the specific component is
+     * extracted
+     *
+     * @param message message
+     * @param cause   optional root cause
+     * @param service originating service
+     */
+    public ServiceStateException(String message,
+                                 Throwable cause,
+                                 Service service) {
+      this(message, cause, service.getServiceState());
+    }
+
+    /**
+     * Get the state when this exception was raised
+     *
+     * @return the state of the service
+     */
+    public ServiceState getState() {
+      return state;
+    }
+
+
+  }
+
+  /**
+   * This is an exception that can be raised on a liveness failure.
+   */
+  public static class LivenessException extends IOException {
+
+    /**
+     * Constructs an exception with {@code null} as its error detail message.
+     */
+    public LivenessException() {
+    }
+
+    /**
+     * Constructs an Exception with the specified detail message.
+     *
+     * @param message The detail message (which is saved for later retrieval by
+     *                the {@link #getMessage()} method)
+     */
+    public LivenessException(String message) {
+      super(message);
+    }
+
+    /**
+     * Constructs an exception with the specified detail message and cause.
+     *
+     * <p> The detail message associated with {@code cause} is only incorporated
+     * into this exception's detail message when the message parameter is null.
+     *
+     * @param message The detail message (which is saved for later retrieval by
+     *                the {@link #getMessage()} method)
+     * @param cause   The cause (which is saved for later retrieval by the
+     *                {@link #getCause()} method).  (A null value is permitted,
+     *                and indicates that the cause is nonexistent or unknown.)
+     */
+    public LivenessException(String message, Throwable cause) {
+      super(message, cause);
+    }
+
+    /**
+     * Constructs an exception with the specified cause and a detail message of
+     * {@code cause.toString())}. A null cause is allowed.
+     *
+     * @param cause The cause (which is saved for later retrieval by the {@link
+     *              #getCause()} method). Can be null.
+     */
+    public LivenessException(Throwable cause) {
+      super(cause);
+    }
+  }
+
+  /**
+   * The state of the service as perceived by the service itself. Failure is the
+   * odd one as it often takes a side effecting test (or an outsider) to
+   * observe.
+   */
+  public enum ServiceState {
+    /**
+     * we don't know or care what state the service is in
+     */
+    UNDEFINED,
+    /**
+     * The service has been created
+     */
+    CREATED,
+
+    /**
+     * The service is starting up.
+     * Its {@link Service#start()} method has been called.
+     * When it is ready for work, it will declare itself LIVE.
+     */
+    STARTED,
+    /**
+     * The service is now live and available for external use
+     */
+    LIVE,
+    /**
+     * The service has failed
+     */
+    FAILED,
+      /**
+     * the service has been shut down
+     * The container process may now destroy the instance
+     * Its {@link Service#close()} ()} method has been called.
+     */
+    CLOSED
+  }
+
+  /**
+   * This is the full service status
+   */
+  public static final class ServiceStatus implements Serializable {
+    /** enumerated state */
+    private ServiceState state;
+
+    /** name of the service */
+    private String name;
+
+    /** when did the state change?  */
+    private Date lastStateChange;
+
+    /**
+     * a possibly null array of exceptions that caused a system failure
+     */
+    public ArrayList<Throwable> throwables = new ArrayList<Throwable>(0);
+
+    /**
+     * Create an empty service status instance
+     */
+    public ServiceStatus() {
+    }
+
+    /**
+     * Create a service status instance with the base values set
+     * @param name service name
+     * @param state current state
+     * @param lastStateChange when did the state last change?
+     */
+    public ServiceStatus(String name, ServiceState state,
+                         Date lastStateChange) {
+      this.state = state;
+      this.name = name;
+      this.lastStateChange = lastStateChange;
+    }
+
+    /**
+     * Create a service status instance from the given service
+     *
+     * @param service service to read from
+     */
+    public ServiceStatus(Service service) {
+      name = service.getServiceName();
+      updateState(service);
+    }
+
+    /**
+     * Add a new throwable to the list. This is a no-op if it is null.
+     * To be safely sent over a network connection, the Throwable (and any
+     * chained causes) must be fully serializable.
+     * @param thrown the throwable. Can be null; will not be cloned.
+     */
+    public void addThrowable(Throwable thrown) {
+      if (thrown != null) {
+        throwables.add(thrown);
+      }
+    }
+
+    /**
+     * Get the list of throwables. This may be null.
+     * @return A list of throwables or null
+     */
+    public List<Throwable> getThrowables() {
+      return throwables;
+    }
+
+    /**
+     * Get the current state
+     * @return the state
+     */
+    public ServiceState getState() {
+      return state;
+    }
+
+    /**
+     * set the state
+     * @param state new state
+     */
+    public void setState(ServiceState state) {
+      this.state = state;
+    }
+
+    /**
+     * Get the name of the service
+     * @return the service name
+     */
+    public String getName() {
+      return name;
+    }
+
+    /**
+     * Set the name of the service
+     * @param name the service name
+     */
+    public void setName(String name) {
+      this.name = name;
+    }
+
+    /**
+     * Get the date of the last state change
+     * @return when the service state last changed
+     */
+    public Date getLastStateChange() {
+      return lastStateChange;
+    }
+
+    /**
+     * Set the last state change
+     * @param lastStateChange the timestamp of the last state change
+     */
+    public void setLastStateChange(Date lastStateChange) {
+      this.lastStateChange = lastStateChange;
+    }
+
+    /**
+     * Update the service state
+     * @param service the service to update from
+     */
+    public void updateState(Service service) {
+      synchronized (service) {
+        setState(service.getServiceState());
+        setLastStateChange(service.lastStateChange);
+      }
+    }
+
+    /**
+     * The string operator includes the messages of every throwable
+     * in the list of failures
+     * @return the list of throwables
+     */
+    @Override
+    public String toString() {
+      StringBuilder builder = new StringBuilder();
+      builder.append(getName()).append(" in state ").append(getState());
+      for (Throwable t : throwables) {
+        builder.append("\n ").append(t.toString());
+      }
+      return builder.toString();
+    }
+  }
+}

Added: hadoop/common/branches/HADOOP-6194/src/test/core/org/apache/hadoop/io/TestThrowableWritable.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HADOOP-6194/src/test/core/org/apache/hadoop/io/TestThrowableWritable.java?rev=805030&view=auto
==============================================================================
--- hadoop/common/branches/HADOOP-6194/src/test/core/org/apache/hadoop/io/TestThrowableWritable.java (added)
+++ hadoop/common/branches/HADOOP-6194/src/test/core/org/apache/hadoop/io/TestThrowableWritable.java Mon Aug 17 16:24:09 2009
@@ -0,0 +1,154 @@
+/**
+ * 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.io;
+
+import junit.framework.TestCase;
+
+import java.io.IOException;
+
+public class TestThrowableWritable extends TestCase {
+
+  private ThrowableWritable simple, messageOnly, chained, empty;
+  private static final String SIMPLE = "simple";
+  private static final String MESSAGE_ONLY = "messageOnly";
+  private static final String OUTER = "outer";
+  private static final String INNER = "inner";
+
+  public TestThrowableWritable() {
+  }
+
+  public TestThrowableWritable(String s) {
+    super(s);
+  }
+
+  /** {@inheritDoc} */
+  @Override
+  protected void setUp() throws Exception {
+    super.setUp();
+
+    simple = new ThrowableWritable(new Throwable(SIMPLE));
+    messageOnly = new ThrowableWritable(MESSAGE_ONLY);
+    empty = new ThrowableWritable();
+    chained = new ThrowableWritable(new Throwable(OUTER,
+        new IOException(INNER)));
+  }
+
+  private void assertEmptyStack(ThrowableWritable throwableWritable) {
+    assertEquals(0, throwableWritable.getStack().length);
+  }
+
+  private void assertCopyWorks(ThrowableWritable instance) throws CloneNotSupportedException {
+    Object cloned = instance.clone();
+    ThrowableWritable copy = new ThrowableWritable(instance);
+    assertEquals(cloned, copy);
+    assertEquals(instance, copy);
+    assertEquals(instance.hashCode(), copy.hashCode());
+    assertEquals(instance.getDepth(), copy.getDepth());
+  }
+
+  private void assertStackSetUp(ThrowableWritable instance) {
+    assertTrue(instance.getStack().length > 0);
+    String topEntry = instance.getStack()[0];
+    assertTrue("No stack in "+topEntry,
+        topEntry.contains("TestThrowableWritable"));
+  }
+
+  private void assertMessageEquals(String message, ThrowableWritable instance) {
+    assertEquals(message,instance.getMessage());
+  }
+
+  private void assertDepth(int depth, ThrowableWritable instance) {
+    assertEquals(depth, instance.getDepth());
+  }
+
+  private void assertClassnameContains(String classname, ThrowableWritable instance) {
+    assertNotNull(instance.getClassname());
+    assertContains(classname, instance.getClassname());
+  }
+
+  private void assertContains(String expected, String source) {
+    assertNotNull(source);
+    assertTrue("Did not find "+expected+ " in "+source,source.contains(expected));
+  }
+
+  private void close(java.io.Closeable c) throws IOException {
+    if(c!=null) {
+      c.close();
+    }
+  }
+
+  private void assertRoundTrips(ThrowableWritable source) throws IOException {
+    DataOutputBuffer out = null;
+    DataInputBuffer in = null;
+    ThrowableWritable dest;
+    try {
+      out = new DataOutputBuffer();
+      in = new DataInputBuffer();
+      out.reset();
+      source.write(out);
+      in.reset(out.getData(), out.getLength());
+      dest = new ThrowableWritable();
+      dest.readFields(in);
+    } finally {
+      close(in);
+      close(out);
+    }
+    assertEquals(source, dest);
+  }
+
+  public void testEmptyInstance() throws Throwable {
+    assertNotNull(empty.toString());
+    assertNull(empty.getClassname());
+    assertEquals(empty, empty);
+    assertNull(empty.getMessage());
+    assertCopyWorks(empty);
+    assertDepth(1, empty);
+  }
+
+  public void testSimple() throws Throwable {
+    assertMessageEquals(SIMPLE, simple);
+    assertClassnameContains("Throwable", simple);
+    assertStackSetUp(simple);
+    assertDepth(1, simple);
+    assertCopyWorks(simple);
+    assertRoundTrips(simple);
+  }
+
+  public void testMessageOnly() throws Throwable {
+    assertMessageEquals(MESSAGE_ONLY, messageOnly);
+    assertEmptyStack(messageOnly);
+    assertDepth(1, messageOnly);
+    assertCopyWorks(messageOnly);
+    assertRoundTrips(messageOnly);
+  }
+
+  public void testChained() throws Throwable {
+    assertContains(OUTER, chained.toString());
+    assertClassnameContains("Throwable", chained);
+    assertStackSetUp(chained);
+    assertDepth(2, chained);
+    assertCopyWorks(chained);
+    ThrowableWritable cause = chained.getCause();
+    assertContains(INNER, cause.toString());
+    assertClassnameContains("IOException", cause);
+    assertRoundTrips(chained);
+  }
+
+
+}

Added: hadoop/common/branches/HADOOP-6194/src/test/core/org/apache/hadoop/util/TestServiceLifecycle.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HADOOP-6194/src/test/core/org/apache/hadoop/util/TestServiceLifecycle.java?rev=805030&view=auto
==============================================================================
--- hadoop/common/branches/HADOOP-6194/src/test/core/org/apache/hadoop/util/TestServiceLifecycle.java (added)
+++ hadoop/common/branches/HADOOP-6194/src/test/core/org/apache/hadoop/util/TestServiceLifecycle.java Mon Aug 17 16:24:09 2009
@@ -0,0 +1,350 @@
+/**
+ * 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.util;
+
+import junit.framework.TestCase;
+
+import java.io.IOException;
+import java.util.List;
+
+
+/**
+ * Test service transitions in a mock service
+ */
+
+public class TestServiceLifecycle extends TestCase {
+  private MockService service;
+
+  public TestServiceLifecycle(String name) {
+    super(name);
+  }
+
+  @Override
+  protected void setUp() throws Exception {
+    super.setUp();
+    service = new MockService();
+  }
+
+  @Override
+  protected void tearDown() throws Exception {
+    Service.close(service);
+    super.tearDown();
+  }
+
+  private void ping() throws IOException {
+    service.ping();
+  }
+
+  private void start() throws IOException {
+    service.start();
+  }
+
+  private void close() throws IOException {
+    service.close();
+    assertInTerminatedState();
+  }
+
+  protected void assertInState(Service.ServiceState state)
+          throws Service.ServiceStateException {
+    service.verifyServiceState(state);
+  }
+
+  private void assertInLiveState() throws Service.ServiceStateException {
+    assertInState(Service.ServiceState.LIVE);
+  }
+
+  private void assertInCreatedState() throws Service.ServiceStateException {
+    assertInState(Service.ServiceState.CREATED);
+  }
+
+  private void assertInFailedState() throws Service.ServiceStateException {
+    assertInState(Service.ServiceState.FAILED);
+  }
+
+  private void assertInTerminatedState() throws Service.ServiceStateException {
+    assertInState(Service.ServiceState.CLOSED);
+  }
+
+  private void assertRunning() {
+    assertTrue("Service is not running: " + service, service.isRunning());
+  }
+
+  private void assertNotRunning() {
+    assertFalse("Service is running: " + service, service.isRunning());
+  }
+
+  private void enterState(Service.ServiceState state)
+          throws Service.ServiceStateException {
+    service.changeState(state);
+    assertInState(state);
+  }
+
+
+  private void enterFailedState() throws Service.ServiceStateException {
+    enterState(Service.ServiceState.FAILED);
+  }
+
+  private void enterTerminatedState() throws Service.ServiceStateException {
+    enterState(Service.ServiceState.CLOSED);
+  }
+
+  private void assertStateChangeCount(int expected) {
+    assertEquals("Wrong state change count for " + service,
+            expected,
+            service.getStateChangeCount());
+  }
+
+  private void assertPingCount(int expected) {
+    assertEquals("Wrong pingchange count for " + service,
+            expected,
+            service.getPingCount());
+  }
+
+  private void assertNoStartFromState(Service.ServiceState serviceState)
+          throws IOException {
+    enterState(serviceState);
+    try {
+      service.start();
+      failShouldNotGetHere();
+    } catch (Service.ServiceStateException expected) {
+      //expected
+    }
+  }
+
+  private void failShouldNotGetHere() {
+    fail("expected failure, but service is in " + service.getServiceState());
+  }
+
+  /**
+   * Test that the ping operation returns a mock exception
+   * @return the service status
+   * @throws IOException IO problems
+   */
+  private Service.ServiceStatus assertPingContainsMockException()
+          throws IOException {
+    Service.ServiceStatus serviceStatus = service.ping();
+    List<Throwable> thrown = serviceStatus.getThrowables();
+    assertFalse("No nested exceptions in service status", thrown.isEmpty());
+    Throwable throwable = thrown.get(0);
+    assertTrue(
+            "Nested exception is not a MockServiceException : "+throwable,
+            throwable instanceof MockService.MockServiceException);
+    return serviceStatus;
+  }
+
+  /**
+   * Walk through the lifecycle and check it changes visible state
+   */
+  public void testBasicLifecycle() throws Throwable {
+    assertInCreatedState();
+    assertNotRunning();
+    assertNotRunning();
+    start();
+    assertInLiveState();
+    assertRunning();
+    ping();
+    ping();
+    assertPingCount(2);
+    close();
+    assertStateChangeCount(3);
+    assertNotRunning();
+  }
+
+  /**
+   * Assert that a state changing operation is idempotent
+   * @throws Throwable if something went wrong
+   */
+  public void testStartIdempotent() throws Throwable {
+    start();
+    int count = service.getStateChangeCount();
+    //declare that we want to fail in our start operation
+    service.setFailOnStart(true);
+    //then start. If the innerStart() method is called: failure
+    start();
+    //check that the state count has not changed either.
+    assertStateChangeCount(count);
+    assertInLiveState();
+  }
+
+  public void testTerminateIdempotent() throws Throwable {
+    close();
+    int count = service.getStateChangeCount();
+    close();
+    assertStateChangeCount(count);
+  }
+
+  public void testCloseFromCreated() throws Throwable {
+    close();
+  }
+
+  public void testStaticCloseHandlesNull() throws Throwable {
+    Service.close(null);
+  }
+
+
+  public void testStaticCloseOperation() throws Throwable {
+    Service.close(service);
+    assertInTerminatedState();
+    Service.close(service);
+  }
+
+  public void testFailInStart() throws Throwable {
+    service.setFailOnStart(true);
+    try {
+      start();
+      failShouldNotGetHere();
+    } catch (MockService.MockServiceException e) {
+      assertInFailedState();
+    }
+  }
+
+  public void testPingInFailedReturnsException() throws Throwable {
+    service.setFailOnStart(true);
+    try {
+      start();
+      failShouldNotGetHere();
+    } catch (MockService.MockServiceException e) {
+      assertInFailedState();
+      //and test that the ping works out
+      Service.ServiceStatus serviceStatus = assertPingContainsMockException();
+      assertEquals(Service.ServiceState.FAILED, serviceStatus.getState());
+    }
+  }
+
+  public void testTerminateFromFailure() throws Throwable {
+    enterFailedState();
+    //test that we can get from failed to terminated
+    close();
+  }
+
+  public void testFailInPing() throws Throwable {
+    service.setFailOnPing(true);
+    start();
+    Service.ServiceStatus serviceStatus = service.ping();
+    assertEquals(Service.ServiceState.FAILED, serviceStatus.getState());
+    assertPingCount(1);
+    List<Throwable> thrown = serviceStatus.getThrowables();
+    assertEquals(1, thrown.size());
+    Throwable throwable = thrown.get(0);
+    assertTrue(throwable instanceof MockService.MockServiceException);
+  }
+
+  public void testPingInCreated() throws Throwable {
+    service.setFailOnPing(true);
+    ping();
+    assertPingCount(0);
+  }
+
+
+  /**
+   * Test that when in a failed state, you can't ping the service
+   *
+   * @throws Throwable if needed
+   */
+  public void testPingInFailedStateIsNoop() throws Throwable {
+    enterFailedState();
+    assertInFailedState();
+    Service.ServiceStatus serviceStatus = service.ping();
+    assertEquals(Service.ServiceState.FAILED, serviceStatus.getState());
+    assertPingCount(0);
+  }
+
+  /**
+   * Test that when in a terminated state, you can't ping the service
+   *
+   * @throws Throwable if needed
+   */
+  public void testPingInTerminatedStateIsNoop() throws Throwable {
+    enterTerminatedState();
+    assertInTerminatedState();
+    Service.ServiceStatus serviceStatus = service.ping();
+    assertEquals(Service.ServiceState.CLOSED, serviceStatus.getState());
+    assertPingCount(0);
+  }
+
+  public void testDeploy() throws Throwable {
+    Service.startService(service);
+    assertInLiveState();
+  }
+
+  public void testDeployFailingStart() throws Throwable {
+    service.setFailOnStart(true);
+    try {
+      Service.startService(service);
+    } catch (MockService.MockServiceException e) {
+      assertInTerminatedState();
+    }
+  }
+
+  public void testNoStartFromTerminated() throws Throwable {
+    assertNoStartFromState(Service.ServiceState.CLOSED);
+  }
+
+  public void testNoStartFromFailed() throws Throwable {
+    assertNoStartFromState(Service.ServiceState.CLOSED);
+  }
+
+  public void testStartFromLiveIdempotent() throws Throwable {
+    enterState(Service.ServiceState.LIVE);
+    int count = service.getStateChangeCount();
+    start();
+    assertStateChangeCount(count);
+  }
+
+  public void testFailOnClose() throws Throwable {
+    service.setFailOnClose(true);
+    try {
+      service.close();
+      fail("Should have thrown an exception");
+    } catch (IOException e) {
+      assertInTerminatedState();
+      assertTrue(service.isClosed());
+    }
+    //the second call should be a no-op; no exceptions get thrown
+    service.close();
+  }
+
+  public void testFailIdempotent() throws Throwable {
+    Exception cause = new Exception("test");
+    service.enterFailedState(null);
+    int count = service.getStateChangeCount();
+    service.enterFailedState(cause);
+    assertStateChangeCount(count);
+    assertEquals(cause, service.getFailureCause());
+  }
+
+  public void testFailFromTerminatedDoesNotChangeState() throws Throwable {
+    Service.startService(service);
+    service.close();
+    assertInTerminatedState();
+    Exception cause = new Exception("test");
+    service.enterFailedState(cause);
+    assertInTerminatedState();
+    assertEquals(cause,service.getFailureCause());
+  }
+
+  public void testFailFromFailedDoesNotChangeCause() throws Throwable {
+    Exception cause = new Exception("test");
+    service.enterFailedState(cause);
+    assertInFailedState();
+    service.enterFailedState(new Exception("test2"));
+    assertInFailedState();
+    assertEquals(cause, service.getFailureCause());
+  }
+
+}