You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by jb...@apache.org on 2009/05/27 21:59:34 UTC

svn commit: r779290 - in /incubator/cassandra/trunk/src/java/org/apache/cassandra: analytics/ db/ gms/ net/http/ service/

Author: jbellis
Date: Wed May 27 19:59:33 2009
New Revision: 779290

URL: http://svn.apache.org/viewvc?rev=779290&view=rev
Log:
r/m shutdown code (making Cassandra crash-only, which it has been de facto for a while).
patch by jbellis; reviewed by Eric Evans for CASSANDRA-115

Removed:
    incubator/cassandra/trunk/src/java/org/apache/cassandra/service/IComponentShutdown.java
Modified:
    incubator/cassandra/trunk/src/java/org/apache/cassandra/analytics/AnalyticsContext.java
    incubator/cassandra/trunk/src/java/org/apache/cassandra/db/HintedHandOffManager.java
    incubator/cassandra/trunk/src/java/org/apache/cassandra/db/MinorCompactionManager.java
    incubator/cassandra/trunk/src/java/org/apache/cassandra/db/PeriodicFlushManager.java
    incubator/cassandra/trunk/src/java/org/apache/cassandra/gms/Gossiper.java
    incubator/cassandra/trunk/src/java/org/apache/cassandra/net/http/HttpRequestHandler.java
    incubator/cassandra/trunk/src/java/org/apache/cassandra/service/StorageLoadBalancer.java
    incubator/cassandra/trunk/src/java/org/apache/cassandra/service/StorageService.java

Modified: incubator/cassandra/trunk/src/java/org/apache/cassandra/analytics/AnalyticsContext.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/analytics/AnalyticsContext.java?rev=779290&r1=779289&r2=779290&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/java/org/apache/cassandra/analytics/AnalyticsContext.java (original)
+++ incubator/cassandra/trunk/src/java/org/apache/cassandra/analytics/AnalyticsContext.java Wed May 27 19:59:33 2009
@@ -40,7 +40,6 @@
 import java.util.concurrent.locks.ReentrantLock;
 
 import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.service.IComponentShutdown;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.utils.LogUtil;
 import org.apache.log4j.Logger;
@@ -52,7 +51,7 @@
  *
  * Author : Avinash Lakshman ( alakshman@facebook.com) & Prashant Malik ( pmalik@facebook.com ) & Karthik Ranganathan ( kranganathan@facebook.com )
  */
-public class AnalyticsContext implements IComponentShutdown
+public class AnalyticsContext
 {
 	private static Logger logger_ = Logger.getLogger(AnalyticsContext.class);
 
@@ -141,14 +140,6 @@
 
 
 	/**
-	 * Creates a new instance of AnalyticsReporter
-	 */
-	public AnalyticsContext()
-	{
-		StorageService.instance().registerComponentForShutdown(this);
-	}
-
-	/**
 	* Initializes the context.
 	*/
 	public void init(String contextName, String serverSpecList)
@@ -418,7 +409,11 @@
 	public void stopMonitoring() {
 		if (isMonitoring)
 		{
-			shutdown();
+            if (timer != null)
+            {
+                timer.cancel();
+                timer = null;
+            }
 			isMonitoring = false;
 		}
 	}
@@ -524,18 +519,6 @@
 	}
 
 	/**
-	 * Stops timer if it is running
-	 */
-	public void shutdown()
-	{
-		if (timer != null)
-		{
-			timer.cancel();
-			timer = null;
-		}
-	}
-
-	/**
 	 * Timer callback.
 	 */
 	private void timerEvent() throws IOException

Modified: incubator/cassandra/trunk/src/java/org/apache/cassandra/db/HintedHandOffManager.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/db/HintedHandOffManager.java?rev=779290&r1=779289&r2=779290&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/java/org/apache/cassandra/db/HintedHandOffManager.java (original)
+++ incubator/cassandra/trunk/src/java/org/apache/cassandra/db/HintedHandOffManager.java Wed May 27 19:59:33 2009
@@ -48,7 +48,7 @@
  * runDelieverHints() is called when some other node starts up (potentially
  * from a failure) and delivers the hinted data just to that node.
  */
-public class HintedHandOffManager implements IComponentShutdown
+public class HintedHandOffManager
 {
     private static HintedHandOffManager instance_;
     private static Lock lock_ = new ReentrantLock();
@@ -247,11 +247,6 @@
         }
     }
 
-    public HintedHandOffManager()
-    {
-    	StorageService.instance().registerComponentForShutdown(this);
-    }
-
     public void submit(final ColumnFamilyStore columnFamilyStore)
     {
         Runnable r = new Runnable()
@@ -280,9 +275,4 @@
         };
     	executor_.submit(r);
     }
-
-    public void shutdown()
-    {
-    	executor_.shutdownNow();
-    }
 }

Modified: incubator/cassandra/trunk/src/java/org/apache/cassandra/db/MinorCompactionManager.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/db/MinorCompactionManager.java?rev=779290&r1=779289&r2=779290&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/java/org/apache/cassandra/db/MinorCompactionManager.java (original)
+++ incubator/cassandra/trunk/src/java/org/apache/cassandra/db/MinorCompactionManager.java Wed May 27 19:59:33 2009
@@ -33,7 +33,6 @@
 import org.apache.cassandra.concurrent.ThreadFactoryImpl;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.net.EndPoint;
-import org.apache.cassandra.service.IComponentShutdown;
 import org.apache.cassandra.service.StorageService;
 import org.apache.log4j.Logger;
 
@@ -41,7 +40,7 @@
  * Author : Avinash Lakshman ( alakshman@facebook.com) & Prashant Malik ( pmalik@facebook.com )
  */
 
-class MinorCompactionManager implements IComponentShutdown
+class MinorCompactionManager
 {
     private static MinorCompactionManager instance_;
     private static Lock lock_ = new ReentrantLock();
@@ -152,16 +151,6 @@
     
     private ScheduledExecutorService compactor_ = new DebuggableScheduledThreadPoolExecutor(1, new ThreadFactoryImpl("MINOR-COMPACTION-POOL"));
 
-    public MinorCompactionManager()
-    {
-    	StorageService.instance().registerComponentForShutdown(this);
-	}
-
-    public void shutdown()
-    {
-    	compactor_.shutdownNow();
-    }
-
     public void submitPeriodicCompaction(final ColumnFamilyStore columnFamilyStore)
     {
         Runnable runnable = new Runnable()

Modified: incubator/cassandra/trunk/src/java/org/apache/cassandra/db/PeriodicFlushManager.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/db/PeriodicFlushManager.java?rev=779290&r1=779289&r2=779290&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/java/org/apache/cassandra/db/PeriodicFlushManager.java (original)
+++ incubator/cassandra/trunk/src/java/org/apache/cassandra/db/PeriodicFlushManager.java Wed May 27 19:59:33 2009
@@ -26,13 +26,12 @@
 import org.apache.log4j.Logger;
 
 import org.apache.cassandra.concurrent.*;
-import org.apache.cassandra.service.IComponentShutdown;
 import org.apache.cassandra.service.StorageService;
 
 /**
  *  Background flusher that force-flushes a column family periodically.
  */
-class PeriodicFlushManager implements IComponentShutdown
+class PeriodicFlushManager
 {
     private static Logger logger_ = Logger.getLogger(PeriodicFlushManager.class);
     private static PeriodicFlushManager instance_;
@@ -57,16 +56,6 @@
         return instance_;
     }
 
-    public PeriodicFlushManager()
-    {
-        StorageService.instance().registerComponentForShutdown(this);
-    }
-
-    public void shutdown()
-    {
-        flusher_.shutdownNow();
-    }
-
     public void submitPeriodicFlusher(final ColumnFamilyStore columnFamilyStore, int flushPeriodInMinutes)
     {        
         Runnable runnable= new Runnable()

Modified: incubator/cassandra/trunk/src/java/org/apache/cassandra/gms/Gossiper.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/gms/Gossiper.java?rev=779290&r1=779289&r2=779290&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/java/org/apache/cassandra/gms/Gossiper.java (original)
+++ incubator/cassandra/trunk/src/java/org/apache/cassandra/gms/Gossiper.java Wed May 27 19:59:33 2009
@@ -29,7 +29,6 @@
 import org.apache.cassandra.net.IVerbHandler;
 import org.apache.cassandra.net.Message;
 import org.apache.cassandra.net.MessagingService;
-import org.apache.cassandra.service.IComponentShutdown;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.utils.LogUtil;
 import org.apache.log4j.Logger;
@@ -47,7 +46,7 @@
  * Author : Avinash Lakshman ( alakshman@facebook.com) & Prashant Malik ( pmalik@facebook.com )
  */
 
-public class Gossiper implements IFailureDetectionEventListener, IEndPointStateChangePublisher, IComponentShutdown
+public class Gossiper implements IFailureDetectionEventListener, IEndPointStateChangePublisher
 {
     private class GossipTimerTask extends TimerTask
     {
@@ -146,8 +145,6 @@
         MessagingService.getMessagingInstance().registerVerbHandlers(GOSSIP_DIGEST_ACK2_VERB, new GossipDigestAck2VerbHandler());
         /* register the Gossip stage */
         StageManager.registerStage( Gossiper.GOSSIP_STAGE, new SingleThreadedStage("GMFD") );
-        /* register with Storage Service for shutdown */
-        StorageService.instance().registerComponentForShutdown(this);
     }
 
     public void register(IEndPointStateChangeSubscriber subscriber)
@@ -909,16 +906,6 @@
         gossipTimer_.schedule( new GossipTimerTask(), Gossiper.intervalInMillis_, Gossiper.intervalInMillis_);
     }
 
-    public void shutdown()
-    {
-    	/* This prevents this guy from responding to Gossip messages */
-    	MessagingService.getMessagingInstance().deregisterVerbHandlers(GOSSIP_DIGEST_SYN_VERB);
-        MessagingService.getMessagingInstance().deregisterVerbHandlers(GOSSIP_DIGEST_ACK_VERB);
-        MessagingService.getMessagingInstance().deregisterVerbHandlers(GOSSIP_DIGEST_ACK2_VERB);
-    	/* This prevents this guy from Gossiping */
-        gossipTimer_.cancel();
-    }
-
     public synchronized void addApplicationState(String key, ApplicationState appState)
     {
         EndPointState epState = endPointStateMap_.get(localEndPoint_);

Modified: incubator/cassandra/trunk/src/java/org/apache/cassandra/net/http/HttpRequestHandler.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/net/http/HttpRequestHandler.java?rev=779290&r1=779289&r2=779290&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/java/org/apache/cassandra/net/http/HttpRequestHandler.java (original)
+++ incubator/cassandra/trunk/src/java/org/apache/cassandra/net/http/HttpRequestHandler.java Wed May 27 19:59:33 2009
@@ -682,8 +682,6 @@
 
     private String handleLBHealthCheck()
     {
-    	if(StorageService.instance().isShutdown())
-    		return "";
     	return LB_HEALTH_CHECK_RESPONSE;
     }
 
@@ -692,24 +690,23 @@
      */
     private String handleKillMe()
     {
-    	if(StorageService.instance().isShutdown())
-    		return "Already scheduled for being shutdown";
-    	/*
-    	 * The storage service will wait for a period of time to let the
-    	 * VIP know that we are shutting down, then will perform an actual
-    	 * shutdown on a separate thread.
-    	 */
-        String status = "Service has been killed";
-        try
-        {
-            StorageService.instance().killMe();
-        }
-        catch( Throwable th )
-        {
-            logger_.warn(LogUtil.throwableToString(th));
-            status = "Failed to kill service.";
-        }
-    	return status;
+        logger_.info("Shutting down due to http request");
+        new Thread() {
+            @Override
+            public void run()
+            {
+                try
+                {
+                    Thread.sleep(1000);
+                }
+                catch (InterruptedException e)
+                {
+                    throw new RuntimeException(e);
+                }
+                System.exit(0);
+            }
+        };
+    	return "Service will be killed in one second.";
     }
 
 }

Modified: incubator/cassandra/trunk/src/java/org/apache/cassandra/service/StorageLoadBalancer.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/service/StorageLoadBalancer.java?rev=779290&r1=779289&r2=779290&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/java/org/apache/cassandra/service/StorageLoadBalancer.java (original)
+++ incubator/cassandra/trunk/src/java/org/apache/cassandra/service/StorageLoadBalancer.java Wed May 27 19:59:33 2009
@@ -56,7 +56,7 @@
  * 
  * Author : Avinash Lakshman ( alakshman@facebook.com) & Prashant Malik ( pmalik@facebook.com )
  */
-final class StorageLoadBalancer implements IEndPointStateChangeSubscriber, IComponentShutdown
+final class StorageLoadBalancer implements IEndPointStateChangeSubscriber
 {
     class LoadBalancer implements Runnable
     {
@@ -196,8 +196,6 @@
         StageManager.registerStage(StorageLoadBalancer.lbStage_, new SingleThreadedStage(StorageLoadBalancer.lbStage_));
         /* register the load balancer verb handler */
         MessagingService.getMessagingInstance().registerVerbHandlers(StorageLoadBalancer.moveMessageVerbHandler_, new MoveMessageVerbHandler());
-        /* register with the StorageService */
-        storageService_.registerComponentForShutdown(this);
     }
 
     public void start()
@@ -206,12 +204,6 @@
         Gossiper.instance().register(this);
     }
 
-    public void shutdown()
-    {
-        lbOperations_.shutdownNow();
-        lb_.shutdownNow();
-    }
-
     public void onChange(EndPoint endpoint, EndPointState epState)
     {
         // load information for this specified endpoint for load balancing 

Modified: incubator/cassandra/trunk/src/java/org/apache/cassandra/service/StorageService.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/service/StorageService.java?rev=779290&r1=779289&r2=779290&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/java/org/apache/cassandra/service/StorageService.java (original)
+++ incubator/cassandra/trunk/src/java/org/apache/cassandra/service/StorageService.java Wed May 27 19:59:33 2009
@@ -231,20 +231,9 @@
     private TokenMetadata tokenMetadata_ = new TokenMetadata();
     private DBManager.StorageMetadata storageMetadata_;
 
-    /*
-     * Maintains a list of all components that need to be shutdown
-     * for a clean exit.
-    */
-    private Set<IComponentShutdown> components_ = new HashSet<IComponentShutdown>();
     /* Timer is used to disseminate load information */
     private Timer loadTimer_ = new Timer(false);
 
-    /*
-     * This variable indicates if the local storage instance
-     * has been shutdown.
-    */
-    private AtomicBoolean isShutdown_ = new AtomicBoolean(false);
-
     /* This thread pool is used to do the bootstrap for a new node */
     private ExecutorService bootStrapper_ = new DebuggableThreadPoolExecutor("BOOT-STRAPPER");
     
@@ -322,11 +311,6 @@
     {
         return zk_;
     }
-    
-    public void registerComponentForShutdown(IComponentShutdown component)
-    {
-    	components_.add(component);
-    }
 
     static
     {
@@ -376,50 +360,6 @@
         Gossiper.instance().addApplicationState(StorageService.nodeId_, state);
     }
 
-    public void killMe() throws Throwable
-    {
-        isShutdown_.set(true);
-        /* 
-         * Shutdown the Gossiper to stop responding/sending Gossip messages.
-         * This causes other nodes to detect you as dead and starting hinting
-         * data for the local endpoint. 
-        */
-        Gossiper.instance().shutdown();
-        final long nodeDeadDetectionTime = 25000L;
-        Thread.sleep(nodeDeadDetectionTime);
-        /* Now perform a force flush of the table */
-        String table = DatabaseDescriptor.getTables().get(0);
-        Table.open(table).flush(false);
-        /* Now wait for the flush to complete */
-        Thread.sleep(nodeDeadDetectionTime);
-        /* Shutdown all other components */
-        StorageService.instance().shutdown();
-    }
-
-    public boolean isShutdown()
-    {
-    	return isShutdown_.get();
-    }
-
-    public void shutdown()
-    {
-        bootStrapper_.shutdownNow();
-        /* shut down all stages */
-        StageManager.shutdown();
-        /* shut down the messaging service */
-        MessagingService.shutdown();
-        /* shut down the load disseminator */
-        loadTimer_.cancel();
-        /* shut down the cleaner thread in FileUtils */
-        FileUtils.shutdown();
-
-        /* shut down all registered components */
-        for ( IComponentShutdown component : components_ )
-        {
-        	component.shutdown();
-        }
-    }
-
     public TokenMetadata getTokenMetadata()
     {
         return tokenMetadata_.cloneMe();