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/04/01 00:17:20 UTC

svn commit: r760679 - in /incubator/cassandra/trunk: bin/ src/org/apache/cassandra/db/ src/org/apache/cassandra/service/

Author: jbellis
Date: Tue Mar 31 22:17:20 2009
New Revision: 760679

URL: http://svn.apache.org/viewvc?rev=760679&view=rev
Log:
Move startup code into CassandraDaemon.  Drop stdout and stderr.  Add hooks for jsvc, which allows more thorough daemonization (such as dropping priviledges and setting process name).  New startup script bin/casssandra provides out-of-the-box background mode without hacks like screen or nohup; -f flag restores old foreground mode.  Also -p <filename> to log pid.  Patch by Eric Evans for #20

Added:
    incubator/cassandra/trunk/src/org/apache/cassandra/service/CassandraDaemon.java
Removed:
    incubator/cassandra/trunk/bin/start-server
Modified:
    incubator/cassandra/trunk/bin/stop-server
    incubator/cassandra/trunk/src/org/apache/cassandra/db/DBManager.java
    incubator/cassandra/trunk/src/org/apache/cassandra/service/CassandraServer.java
    incubator/cassandra/trunk/src/org/apache/cassandra/service/StorageService.java

Modified: incubator/cassandra/trunk/bin/stop-server
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/bin/stop-server?rev=760679&r1=760678&r2=760679&view=diff
==============================================================================
--- incubator/cassandra/trunk/bin/stop-server (original)
+++ incubator/cassandra/trunk/bin/stop-server Tue Mar 31 22:17:20 2009
@@ -1,2 +1,14 @@
-user=`whoami`
-pgrep -u $user -f cassandra | xargs kill -9
+echo "please read the stop-server script before use"
+
+# if you are using the cassandra start script with -p, this
+# is the best way to stop:
+
+# kill `cat <pidfile>` 
+
+
+# otherwise, you can run something like this, but
+# this is a shotgun approach and will kill other processes
+# with cassandra in their name or arguments too:
+
+# user=`whoami`
+# pgrep -u $user -f cassandra | xargs kill

Modified: incubator/cassandra/trunk/src/org/apache/cassandra/db/DBManager.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/org/apache/cassandra/db/DBManager.java?rev=760679&r1=760678&r2=760679&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/org/apache/cassandra/db/DBManager.java (original)
+++ incubator/cassandra/trunk/src/org/apache/cassandra/db/DBManager.java Tue Mar 31 22:17:20 2009
@@ -44,7 +44,7 @@
     private static DBManager dbMgr_;
     private static Lock lock_ = new ReentrantLock();
 
-    public static DBManager instance() throws Throwable
+    public static DBManager instance() throws IOException
     {
         if ( dbMgr_ == null )
         {
@@ -89,7 +89,7 @@
         }
     }
 
-    public DBManager() throws Throwable
+    public DBManager() throws IOException
     {
         Set<String> tables = DatabaseDescriptor.getTableToColumnFamilyMap().keySet();
         

Added: incubator/cassandra/trunk/src/org/apache/cassandra/service/CassandraDaemon.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/org/apache/cassandra/service/CassandraDaemon.java?rev=760679&view=auto
==============================================================================
--- incubator/cassandra/trunk/src/org/apache/cassandra/service/CassandraDaemon.java (added)
+++ incubator/cassandra/trunk/src/org/apache/cassandra/service/CassandraDaemon.java Tue Mar 31 22:17:20 2009
@@ -0,0 +1,137 @@
+/**
+ * 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.cassandra.service;
+
+import java.io.File;
+import java.io.IOException;
+
+import org.apache.log4j.Logger;
+import com.facebook.thrift.protocol.TBinaryProtocol;
+import com.facebook.thrift.protocol.TProtocolFactory;
+import com.facebook.thrift.server.TThreadPoolServer;
+import com.facebook.thrift.transport.TServerSocket;
+import com.facebook.thrift.transport.TTransportException;
+import org.apache.cassandra.config.DatabaseDescriptor;
+
+/**
+ * This class supports two methods for creating a Cassandra node daemon, 
+ * invoking the class's main method, and using the jsvc wrapper from 
+ * commons-daemon, (for more information on using this class with the 
+ * jsvc wrapper, see the 
+ * <a href="http://commons.apache.org/daemon/jsvc.html">Commons Daemon</a>
+ * documentation).
+ * 
+ * @author Eric Evans <ee...@racklabs.com>
+ * 
+ */
+
+public class CassandraDaemon
+{
+    private static Logger logger = Logger.getLogger(CassandraDaemon.class);
+    private TThreadPoolServer serverEngine;
+
+    private void setup() throws IOException, TTransportException
+    {
+        int listenPort = DatabaseDescriptor.getThriftPort();
+        
+        Thread.setDefaultUncaughtExceptionHandler(new Thread.UncaughtExceptionHandler()
+        {
+            public void uncaughtException(Thread t, Throwable e)
+            {
+                logger.error("Fatal exception in thread " + t, e);
+            }
+        });
+        
+        CassandraServer peerStorageServer = new CassandraServer();
+        peerStorageServer.start();
+        Cassandra.Processor processor = new Cassandra.Processor(peerStorageServer);
+
+        // Transport
+        TServerSocket tServerSocket =  new TServerSocket(listenPort);
+
+        // Protocol factory
+        TProtocolFactory tProtocolFactory = new TBinaryProtocol.Factory();
+
+        // ThreadPool Server
+        TThreadPoolServer.Options options = new TThreadPoolServer.Options();
+        options.minWorkerThreads = 64;
+        serverEngine = new TThreadPoolServer(processor, tServerSocket, tProtocolFactory);
+    }
+
+    /** hook for JSVC */
+    public void load(String[] args) throws IOException, TTransportException
+    {  
+        setup();
+    }
+
+    /** hook for JSVC */
+    public void start()
+    {
+        logger.info("Cassandra starting up...");
+        serverEngine.serve();
+    }
+
+    /** hook for JSVC */
+    public void stop()
+    {
+        logger.info("Cassandra shutting down...");
+        serverEngine.stop();
+    }
+    
+    
+    /** hook for JSVC */
+    public void destroy()
+    {        
+    }
+    
+    public static void main(String[] args)
+    {
+        CassandraDaemon daemon = new CassandraDaemon();
+        String pidFile = System.getProperty("cassandra-pidfile");
+        
+        try
+        {   
+            daemon.setup();
+
+            if (pidFile != null)
+            {
+                new File(pidFile).deleteOnExit();
+            }
+
+            if (System.getProperty("cassandra-foreground") == null)
+            {
+                System.out.close();
+                System.err.close();
+            }
+
+            daemon.start();
+        }
+        catch (Exception e)
+        {
+            String msg = "Exception encountered during startup.";
+            logger.error(msg, e);
+
+            // try to warn user on stdout too, if we haven't already detached
+            System.out.println(msg);
+            e.printStackTrace();
+
+            System.exit(3);
+        }
+    }
+}

Modified: incubator/cassandra/trunk/src/org/apache/cassandra/service/CassandraServer.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/org/apache/cassandra/service/CassandraServer.java?rev=760679&r1=760678&r2=760679&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/org/apache/cassandra/service/CassandraServer.java (original)
+++ incubator/cassandra/trunk/src/org/apache/cassandra/service/CassandraServer.java Tue Mar 31 22:17:20 2009
@@ -90,7 +90,7 @@
 	 * The start function initializes the server and start's listening on the
 	 * specified port.
 	 */
-	public void start() throws Throwable
+	public void start() throws IOException
     {
 		LogUtil.init();
 		//LogUtil.setLogLevel("com.facebook", "DEBUG");
@@ -850,42 +850,5 @@
 		return null;
 	}
 
-	public static void main(String[] args) throws Throwable
-	{
-		int port = DatabaseDescriptor.getThriftPort();
-
-        Thread.setDefaultUncaughtExceptionHandler(new Thread.UncaughtExceptionHandler()
-        {
-            public void uncaughtException(Thread t, Throwable e)
-            {
-                logger_.error("Fatal exception in thread " + t, e);
-            }
-        });
-
-		try
-		{
-			CassandraServer peerStorageServer = new CassandraServer();
-			peerStorageServer.start();
-			Cassandra.Processor processor = new Cassandra.Processor(
-					peerStorageServer);
-			// Transport
-			TServerSocket tServerSocket =  new TServerSocket(port);
-			 // Protocol factory
-			TProtocolFactory tProtocolFactory = new TBinaryProtocol.Factory();
-			 // ThreadPool Server
-			Options options = new Options();
-			options.minWorkerThreads = 64;
-			TThreadPoolServer serverEngine = new TThreadPoolServer(processor, tServerSocket, tProtocolFactory);
-			serverEngine.serve();
-
-		}
-		catch (Exception x)
-		{
-			System.err.println("UNCAUGHT EXCEPTION IN main()");
-			x.printStackTrace();
-			System.exit(1);
-		}
-
-	}
-
+    // main method moved to CassandraDaemon
 }

Modified: incubator/cassandra/trunk/src/org/apache/cassandra/service/StorageService.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/org/apache/cassandra/service/StorageService.java?rev=760679&r1=760678&r2=760679&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/org/apache/cassandra/service/StorageService.java (original)
+++ incubator/cassandra/trunk/src/org/apache/cassandra/service/StorageService.java Tue Mar 31 22:17:20 2009
@@ -448,7 +448,7 @@
         }
     }
     
-    public void start() throws Throwable
+    public void start() throws IOException
     {
         /* Start the DB */
         storageMetadata_ = DBManager.instance().start();