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/26 18:58:14 UTC

svn commit: r808124 - in /hadoop/common/branches/HADOOP-6194/src: java/org/apache/hadoop/util/MockService.java java/org/apache/hadoop/util/Service.java test/core/org/apache/hadoop/util/TestServiceLifecycle.java

Author: stevel
Date: Wed Aug 26 16:58:14 2009
New Revision: 808124

URL: http://svn.apache.org/viewvc?rev=808124&view=rev
Log:
HADOOP-6194 SFOS-1270 strip the ping operation from the service lifecycle proposal.

Modified:
    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/util/TestServiceLifecycle.java

Modified: 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=808124&r1=808123&r2=808124&view=diff
==============================================================================
--- hadoop/common/branches/HADOOP-6194/src/java/org/apache/hadoop/util/MockService.java (original)
+++ hadoop/common/branches/HADOOP-6194/src/java/org/apache/hadoop/util/MockService.java Wed Aug 26 16:58:14 2009
@@ -43,20 +43,15 @@
     super(conf);
   }
 
-  private boolean failOnStart, failOnPing, failOnClose;
+  private boolean failOnStart, 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;
   }
@@ -94,18 +89,6 @@
 
   /**
    * {@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
    */
@@ -140,16 +123,14 @@
     return stateChangeCount;
   }
 
-  public int getPingCount() {
-    return pingCount;
-  }
+
 
   /**
    * An exception to indicate we have triggered a mock event
    */
-  static class MockServiceException extends IOException {
+  public static class MockServiceException extends IOException {
 
-    private MockServiceException(String message) {
+    public MockServiceException(String message) {
       super(message);
     }
   }

Modified: 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=808124&r1=808123&r2=808124&view=diff
==============================================================================
--- hadoop/common/branches/HADOOP-6194/src/java/org/apache/hadoop/util/Service.java (original)
+++ hadoop/common/branches/HADOOP-6194/src/java/org/apache/hadoop/util/Service.java Wed Aug 26 16:58:14 2009
@@ -23,11 +23,8 @@
 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
@@ -67,8 +64,7 @@
  * <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
+ * <li>If the service wishes to declare itself as having failed, call
  * {@link #enterFailedState(Throwable)} to enter the failed state.</li>
  *
  * <li>Override the {@link #innerStart()} method to start the service, including
@@ -82,20 +78,14 @@
  * 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)} ,
+ * state -which is synchronized. 
  * 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.
@@ -134,7 +124,7 @@
    * value: {@value}
    */
   public static final String ERROR_NO_CONFIGURATION
-          = "Cannot initialize when unconfigured";
+          = "Cannot initialize no Configuration has been provided";
 
   /**
    * Construct a service with no configuration; one must be called with {@link
@@ -190,71 +180,6 @@
     }
   }
 
-  /**
-   * 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}
@@ -265,7 +190,7 @@
    * @return an IOException representing or containing the forwarded exception
    */
   @SuppressWarnings({"ThrowableInstanceNeverThrown"})
-  protected IOException forwardAsIOException(Throwable thrown) {
+  protected static IOException forwardAsIOException(Throwable thrown) {
     IOException newException;
     if(thrown instanceof IOException) {
       newException = (IOException) thrown;
@@ -347,19 +272,6 @@
   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.
@@ -469,6 +381,14 @@
   }
 
   /**
+   * When did the service last change state
+   * @return the last state change of this service
+   */
+  public Date getLastStateChange() {
+    return lastStateChange;
+  }
+
+  /**
    * 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
@@ -524,7 +444,6 @@
     }
   }
 
-
   /**
    * Verify that a service is in a specific state
    *
@@ -672,14 +591,7 @@
       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);
-      }
+      throw forwardAsIOException(t);
     }
   }
 
@@ -711,6 +623,16 @@
 
 
   /**
+   * Get the current number of workers
+   * @return the worker count, or -1 for "this service has no workers"
+   */
+
+  public int getLiveWorkerCount() {
+    return -1;
+  }
+
+
+  /**
    * An exception that indicates there is something wrong with the state of the
    * service
    */
@@ -787,55 +709,6 @@
   }
 
   /**
-   * 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.
@@ -872,145 +745,4 @@
     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();
-    }
-  }
 }

Modified: 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=808124&r1=808123&r2=808124&view=diff
==============================================================================
--- hadoop/common/branches/HADOOP-6194/src/test/core/org/apache/hadoop/util/TestServiceLifecycle.java (original)
+++ hadoop/common/branches/HADOOP-6194/src/test/core/org/apache/hadoop/util/TestServiceLifecycle.java Wed Aug 26 16:58:14 2009
@@ -46,10 +46,6 @@
     super.tearDown();
   }
 
-  private void ping() throws IOException {
-    service.ping();
-  }
-
   private void start() throws IOException {
     service.start();
   }
@@ -109,12 +105,6 @@
             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);
@@ -130,22 +120,6 @@
     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
@@ -157,9 +131,6 @@
     start();
     assertInLiveState();
     assertRunning();
-    ping();
-    ping();
-    assertPingCount(2);
     close();
     assertStateChangeCount(3);
     assertNotRunning();
@@ -213,18 +184,6 @@
     }
   }
 
-  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();
@@ -232,50 +191,6 @@
     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);