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/27 19:04:59 UTC

svn commit: r808526 - in /hadoop/common/branches/HADOOP-6194: ./ src/contrib/ec2/ src/docs/ src/java/ src/java/org/apache/hadoop/util/ src/test/core/ src/test/core/org/apache/hadoop/util/

Author: stevel
Date: Thu Aug 27 17:04:58 2009
New Revision: 808526

URL: http://svn.apache.org/viewvc?rev=808526&view=rev
Log:
HADOOP-6194 adding InterruptedException on startup, listeners

Modified:
    hadoop/common/branches/HADOOP-6194/   (props changed)
    hadoop/common/branches/HADOOP-6194/src/contrib/ec2/   (props changed)
    hadoop/common/branches/HADOOP-6194/src/docs/   (props changed)
    hadoop/common/branches/HADOOP-6194/src/java/   (props changed)
    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/   (props changed)
    hadoop/common/branches/HADOOP-6194/src/test/core/org/apache/hadoop/util/TestServiceLifecycle.java

Propchange: hadoop/common/branches/HADOOP-6194/
------------------------------------------------------------------------------
--- svn:mergeinfo (original)
+++ svn:mergeinfo Thu Aug 27 17:04:58 2009
@@ -1,2 +1,2 @@
-/hadoop/common/trunk:804966-806688
+/hadoop/common/trunk:804966-807681
 /hadoop/core/branches/branch-0.19/core:713112

Propchange: hadoop/common/branches/HADOOP-6194/src/contrib/ec2/
------------------------------------------------------------------------------
--- svn:mergeinfo (original)
+++ svn:mergeinfo Thu Aug 27 17:04:58 2009
@@ -1,3 +1,3 @@
-/hadoop/common/trunk/src/contrib/ec2:804966-806688
+/hadoop/common/trunk/src/contrib/ec2:804966-807681
 /hadoop/core/branches/branch-0.19/core/src/contrib/ec2:713112
 /hadoop/core/trunk/src/contrib/ec2:776175-784663

Propchange: hadoop/common/branches/HADOOP-6194/src/docs/
------------------------------------------------------------------------------
--- svn:mergeinfo (original)
+++ svn:mergeinfo Thu Aug 27 17:04:58 2009
@@ -1,3 +1,3 @@
-/hadoop/common/trunk/src/docs:804966-806688
+/hadoop/common/trunk/src/docs:804966-807681
 /hadoop/core/branches/HADOOP-4687/core/src/docs:776175-786719
 /hadoop/core/branches/branch-0.19/src/docs:713112

Propchange: hadoop/common/branches/HADOOP-6194/src/java/
------------------------------------------------------------------------------
--- svn:mergeinfo (original)
+++ svn:mergeinfo Thu Aug 27 17:04:58 2009
@@ -1,3 +1,3 @@
-/hadoop/common/trunk/src/java:804966-806688
+/hadoop/common/trunk/src/java:804966-807681
 /hadoop/core/branches/branch-0.19/core/src/java:713112
 /hadoop/core/trunk/src/core:776175-785643,785929-786278

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=808526&r1=808525&r2=808526&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 Thu Aug 27 17:04:58 2009
@@ -45,6 +45,7 @@
 
   private boolean failOnStart, failOnClose;
   private boolean goLiveInStart = true;
+  private int startupDelay; 
   private boolean closed = true;
   private volatile int stateChangeCount = 0;
 
@@ -60,6 +61,10 @@
     this.failOnClose = failOnClose;
   }
 
+  public void setStartupDelay(int startupDelay) {
+    this.startupDelay = startupDelay;
+  }
+
   public boolean isClosed() {
     return closed;
   }
@@ -76,12 +81,17 @@
   /**
    * {@inheritDoc}
    * @throws IOException  if {@link #failOnStart is set}
+   * @throws InterruptedException if the service was to delay its startup, and it
+   * was interrupted
    */
   @Override
-  protected void innerStart() throws IOException {
+  protected void innerStart() throws IOException, InterruptedException {
     if (failOnStart) {
       throw new MockServiceException("failOnStart");
     }
+    if (startupDelay>0) {
+      Thread.sleep(startupDelay);
+    }
     if (goLiveInStart) {
       goLive();
     }

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=808526&r1=808525&r2=808526&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 Thu Aug 27 17:04:58 2009
@@ -25,6 +25,8 @@
 import java.io.IOException;
 import java.io.Closeable;
 import java.util.Date;
+import java.util.List;
+import java.util.ArrayList;
 
 /**
  * This is the base class for services that can be deployed. A service is any
@@ -109,6 +111,8 @@
    * A root cause for failure. May be null.
    */
   private Throwable failureCause;
+  
+  private List<StateChangeListener> stateListeners;
 
   /**
    * Error string included in {@link ServiceStateException} exceptions
@@ -145,18 +149,22 @@
   /**
    * Start any work (usually in separate threads).
    *
-   * When completed, the service will be in the {@link ServiceState#STARTED}
+   * When successful, the service will be in the {@link ServiceState#STARTED}
    * state, or may have already transited to the {@link ServiceState#LIVE}
    * state
    *
+   * When unsuccessful, the service will have entered the FAILED state and
+   * then attempted to close down.
    * 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.
+   * @throws InterruptedException if the thread was interrupted on startup
    */
-  public void start() throws IOException {
+  public final void start() throws IOException, InterruptedException {
     synchronized (this) {
       //this request is idempotent on either live or starting states; either
       //state is ignored
@@ -165,6 +173,7 @@
               currentState == ServiceState.STARTED) {
         return;
       }
+      //sanity check: make sure that we are configured
       if (getConf() == null) {
         throw new ServiceStateException(ERROR_NO_CONFIGURATION,
                 getServiceState());
@@ -177,6 +186,10 @@
     } catch (IOException e) {
       enterFailedState(e);
       throw e;
+    } catch (InterruptedException e) {
+      //interruptions mean "stop trying to start the service"
+      enterFailedState(e);
+      throw e;
     }
   }
 
@@ -189,7 +202,6 @@
    * @param thrown the exception to forward
    * @return an IOException representing or containing the forwarded exception
    */
-  @SuppressWarnings({"ThrowableInstanceNeverThrown"})
   protected static IOException forwardAsIOException(Throwable thrown) {
     IOException newException;
     if(thrown instanceof IOException) {
@@ -202,7 +214,6 @@
     return newException;
   }
 
-
   /**
    * Test for a service being in the {@link ServiceState#LIVE} or {@link
    * ServiceState#STARTED}
@@ -268,8 +279,9 @@
    * state to {@link ServiceState#LIVE} to indicate the service is now live.
    *
    * @throws IOException for any problem.
+   * @throws InterruptedException if the thread was interrupted on startup
    */
-  protected void innerStart() throws IOException {
+  protected void innerStart() throws IOException, InterruptedException {
   }
 
   /**
@@ -364,21 +376,7 @@
     }
   }
 
-  /**
-   * 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);
-    }
-  }
+
 
   /**
    * When did the service last change state
@@ -631,8 +629,66 @@
     return -1;
   }
 
+  /**
+   * Override point - a method called whenever there is a state change.
+   *
+   * The base class logs the event and notifies all state change listeners.
+   *
+   * @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);
+    }
+    
+    //copy all the listeners out of the list
+    //this is to give us access to an unsynchronized copy of the listeners, which
+    //can then have the state notifications called outside of any synchronized 
+    //sectoin
+    StateChangeListener[] listeners = null;
+    synchronized (this) {
+      if (stateListeners != null) {
+        listeners = new StateChangeListener[stateListeners
+                .size()];
+        stateListeners.toArray(listeners);
+      } else {
+        //no listeners, exit here
+        return;
+      }
+    }
+    // issue the notifications
+    for (StateChangeListener listener : listeners) {
+      listener.onStateChange(this, oldState, newState);
+    }
+  }
 
   /**
+   * Add a new state change listener
+   * @param listener a new state change listener
+   */
+  public synchronized void addStateChangeListener(StateChangeListener listener) {
+    if(stateListeners==null) {
+      stateListeners = new ArrayList<StateChangeListener>(1);
+    }
+    stateListeners.add(listener);
+  }
+
+  /**
+   * Remove a state change listener. This is an idempotent operation; it is 
+   * not an error to attempt to remove a listener which is not present
+   * @param listener a state change listener
+   */
+  public synchronized void removeStateChangeListener(StateChangeListener listener) {
+    if (stateListeners != null) {
+      stateListeners.remove(listener);
+    }
+  }
+
+  
+  /**
    * An exception that indicates there is something wrong with the state of the
    * service
    */
@@ -745,4 +801,19 @@
     CLOSED
   }
 
+  public interface StateChangeListener {
+    
+    /**
+     * This method is called for any listener. 
+     *
+     * The base class logs the event.
+     * @param service the service whose state is changing
+     * @param oldState existing state
+     * @param newState new state.
+     */
+    void onStateChange(Service service, 
+                       ServiceState oldState,
+                       ServiceState newState);
+
+  }
 }

Propchange: hadoop/common/branches/HADOOP-6194/src/test/core/
------------------------------------------------------------------------------
--- svn:mergeinfo (original)
+++ svn:mergeinfo Thu Aug 27 17:04:58 2009
@@ -1,3 +1,3 @@
-/hadoop/common/trunk/src/test/core:804966-806688
+/hadoop/common/trunk/src/test/core:804966-807681
 /hadoop/core/branches/branch-0.19/core/src/test/core:713112
 /hadoop/core/trunk/src/test/core:776175-785643,785929-786278

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=808526&r1=808525&r2=808526&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 Thu Aug 27 17:04:58 2009
@@ -46,7 +46,7 @@
     super.tearDown();
   }
 
-  private void start() throws IOException {
+  private void start() throws IOException, InterruptedException {
     service.start();
   }
 
@@ -106,7 +106,7 @@
   }
 
   private void assertNoStartFromState(Service.ServiceState serviceState)
-          throws IOException {
+          throws Throwable {
     enterState(serviceState);
     try {
       service.start();