You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by gd...@apache.org on 2010/06/15 17:04:46 UTC

svn commit: r954928 - in /cassandra/trunk: src/java/org/apache/cassandra/avro/ src/java/org/apache/cassandra/service/ src/java/org/apache/cassandra/thrift/ test/unit/org/apache/cassandra/service/

Author: gdusbabek
Date: Tue Jun 15 15:04:46 2010
New Revision: 954928

URL: http://svn.apache.org/viewvc?rev=954928&view=rev
Log:
begin process of consolidating common parts of CassandraDaemon. Patch by Karthick Sankarachary, reviewed by Gary Dusbabek. CASSANDRA-1131

Added:
    cassandra/trunk/src/java/org/apache/cassandra/service/AbstractCassandraDaemon.java
    cassandra/trunk/src/java/org/apache/cassandra/service/CassandraDaemon.java
Modified:
    cassandra/trunk/src/java/org/apache/cassandra/avro/CassandraDaemon.java
    cassandra/trunk/src/java/org/apache/cassandra/thrift/CassandraDaemon.java
    cassandra/trunk/test/unit/org/apache/cassandra/service/EmbeddedCassandraServiceTest.java

Modified: cassandra/trunk/src/java/org/apache/cassandra/avro/CassandraDaemon.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/avro/CassandraDaemon.java?rev=954928&r1=954927&r2=954928&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/avro/CassandraDaemon.java (original)
+++ cassandra/trunk/src/java/org/apache/cassandra/avro/CassandraDaemon.java Tue Jun 15 15:04:46 2010
@@ -18,24 +18,15 @@
 
 package org.apache.cassandra.avro;
 
-import java.io.File;
 import java.io.IOException;
 import java.net.InetAddress;
-import java.net.InetSocketAddress;
-import java.util.ArrayList;
-import java.util.List;
 import java.util.UUID;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.Future;
 
-import org.apache.avro.ipc.SocketServer;
 import org.apache.avro.ipc.HttpServer;
 import org.apache.avro.specific.SpecificResponder;
-import org.apache.cassandra.concurrent.StageManager;
 import org.apache.cassandra.config.ConfigurationException;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.CompactionManager;
-import org.apache.cassandra.db.DefsTable;
 import org.apache.cassandra.db.SystemTable;
 import org.apache.cassandra.db.Table;
 import org.apache.cassandra.db.commitlog.CommitLog;
@@ -44,24 +35,20 @@ import org.apache.cassandra.service.Migr
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.Mx4jTool;
-
-import org.apache.cassandra.utils.WrappedRunnable;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.log4j.PropertyConfigurator;
-
 /**
  * The Avro analogue to org.apache.cassandra.service.CassandraDaemon.
  *
  */
-public class CassandraDaemon {
+public class CassandraDaemon extends org.apache.cassandra.service.AbstractCassandraDaemon {
     private static Logger logger = LoggerFactory.getLogger(CassandraDaemon.class);
     private HttpServer server;
     private InetAddress listenAddr;
     private int listenPort;
     
-    private void setup() throws IOException
+    protected void setup() throws IOException
     {
         listenPort = DatabaseDescriptor.getRpcPort();
         listenAddr = DatabaseDescriptor.getRpcAddress();
@@ -134,12 +121,6 @@ public class CassandraDaemon {
     }
     
     /** hook for JSVC */
-    public void load(String[] arguments) throws IOException
-    {
-        setup();
-    }
-    
-    /** hook for JSVC */
     public void start() throws IOException
     {
         if (logger.isDebugEnabled())
@@ -159,43 +140,8 @@ public class CassandraDaemon {
         server.close();
     }
     
-    /** 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 (Throwable 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);
-        }
+        new CassandraDaemon().activate();
     }
 
 }

Added: cassandra/trunk/src/java/org/apache/cassandra/service/AbstractCassandraDaemon.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/service/AbstractCassandraDaemon.java?rev=954928&view=auto
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/service/AbstractCassandraDaemon.java (added)
+++ cassandra/trunk/src/java/org/apache/cassandra/service/AbstractCassandraDaemon.java Tue Jun 15 15:04:46 2010
@@ -0,0 +1,126 @@
+/**
+ * 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.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * The <code>CassandraDaemon</code> is an abstraction for a Cassandra daemon
+ * service, which defines not only a way to activate and deactivate it, but also
+ * hooks into its lifecycle methods (see {@link #setup()}, {@link #start()},
+ * {@link #stop()} and {@link #setup()}).
+ * 
+ */
+public abstract class AbstractCassandraDaemon implements CassandraDaemon
+{
+    private static Logger logger = LoggerFactory
+            .getLogger(AbstractCassandraDaemon.class);
+    
+    /**
+     * This is a hook for concrete daemons to initialize themselves suitably.
+     * 
+     * @throws IOException
+     */
+    protected abstract void setup() throws IOException;
+    
+    /**
+     * Initialize the Cassandra Daemon based on the given <a
+     * href="http://commons.apache.org/daemon/jsvc.html">Commons
+     * Daemon</a>-specific arguments. To clarify, this is a hook for JSVC.
+     * 
+     * @param arguments
+     *            the arguments passed in from JSVC
+     * @throws IOException
+     */
+    public void init(String[] arguments) throws IOException
+    {
+        setup();
+    }
+    
+    /**
+     * Start the Cassandra Daemon, assuming that it has already been
+     * initialized, via either {@link #init(String[])} or
+     * {@link #load(String[])}.
+     * 
+     * @throws IOException
+     */
+    public abstract void start() throws IOException;
+    
+    /**
+     * Stop the daemon, ideally in an idempotent manner.
+     */
+    public abstract void stop();
+    
+    /**
+     * Clean up all resources obtained during the lifetime of the daemon. This
+     * is a hook for JSVC.
+     */
+    public void destroy()
+    {}
+    
+    /**
+     * A convenience method to initialize and start the daemon in one shot.
+     */
+    public void activate()
+    {
+        String pidFile = System.getProperty("cassandra-pidfile");
+        
+        try
+        {
+            setup();
+            
+            if (pidFile != null)
+            {
+                new File(pidFile).deleteOnExit();
+            }
+            
+            if (System.getProperty("cassandra-foreground") == null)
+            {
+                System.out.close();
+                System.err.close();
+            }
+            
+            start();
+        } catch (Throwable 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);
+        }
+    }
+    
+    /**
+     * A convenience method to stop and destroy the daemon in one shot.
+     */
+    public void deactivate()
+    {
+        stop();
+        destroy();
+    }
+    
+}

Added: cassandra/trunk/src/java/org/apache/cassandra/service/CassandraDaemon.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/service/CassandraDaemon.java?rev=954928&view=auto
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/service/CassandraDaemon.java (added)
+++ cassandra/trunk/src/java/org/apache/cassandra/service/CassandraDaemon.java Tue Jun 15 15:04:46 2010
@@ -0,0 +1,53 @@
+package org.apache.cassandra.service;
+
+import java.io.IOException;
+
+/**
+ * The <code>CassandraDaemon</code> interface captures the lifecycle of a
+ * Cassandra daemon that runs on a single node.
+ * 
+ */
+public interface CassandraDaemon
+{
+    /**
+     * Initialize the Cassandra Daemon based on the given <a
+     * href="http://commons.apache.org/daemon/jsvc.html">Commons
+     * Daemon</a>-specific arguments. To clarify, this is a hook for JSVC.
+     * 
+     * @param arguments
+     *            the arguments passed in from JSVC
+     * @throws IOException
+     */
+    public void init(String[] arguments) throws IOException;
+    
+    /**
+     * Start the Cassandra Daemon, assuming that it has already been
+     * initialized, via either {@link #init(String[])} or
+     * {@link #load(String[])}.
+     * 
+     * @throws IOException
+     */
+    public void start() throws IOException;
+    
+    /**
+     * Stop the daemon, ideally in an idempotent manner.
+     */
+    public void stop();
+    
+    /**
+     * Clean up all resources obtained during the lifetime of the daemon. Just
+     * to clarify, this is a hook for JSVC.
+     */
+    public void destroy();
+    
+    /**
+     * A convenience method to initialize and start the daemon in one shot.
+     */
+    public void activate();
+    
+    /**
+     * A convenience method to stop and destroy the daemon in one shot.
+     */
+    public void deactivate();
+    
+}
\ No newline at end of file

Modified: cassandra/trunk/src/java/org/apache/cassandra/thrift/CassandraDaemon.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/thrift/CassandraDaemon.java?rev=954928&r1=954927&r2=954928&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/thrift/CassandraDaemon.java (original)
+++ cassandra/trunk/src/java/org/apache/cassandra/thrift/CassandraDaemon.java Tue Jun 15 15:04:46 2010
@@ -18,38 +18,36 @@
 
 package org.apache.cassandra.thrift;
 
-import java.io.File;
 import java.io.IOException;
-import java.net.InetSocketAddress;
 import java.net.InetAddress;
+import java.net.InetSocketAddress;
 import java.util.UUID;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.SynchronousQueue;
 import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
 
-import org.apache.cassandra.db.SystemTable;
-import org.apache.thrift.server.TServer;
 import org.apache.cassandra.config.ConfigurationException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.cassandra.utils.Mx4jTool;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.CompactionManager;
+import org.apache.cassandra.db.SystemTable;
+import org.apache.cassandra.db.Table;
 import org.apache.cassandra.db.commitlog.CommitLog;
 import org.apache.cassandra.db.migration.Migration;
+import org.apache.cassandra.service.MigrationManager;
+import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.Mx4jTool;
+import org.apache.thrift.TProcessorFactory;
 import org.apache.thrift.protocol.TBinaryProtocol;
 import org.apache.thrift.protocol.TProtocolFactory;
+import org.apache.thrift.server.TServer;
+import org.apache.thrift.transport.TFramedTransport;
 import org.apache.thrift.transport.TServerSocket;
 import org.apache.thrift.transport.TTransportException;
 import org.apache.thrift.transport.TTransportFactory;
-import org.apache.thrift.transport.TFramedTransport;
-import org.apache.thrift.TProcessorFactory;
-import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.service.MigrationManager;
-import org.apache.cassandra.service.StorageService;
-import org.apache.cassandra.utils.FBUtilities;
-import org.apache.cassandra.db.Table;
-import org.apache.cassandra.db.CompactionManager;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * This class supports two methods for creating a Cassandra node daemon, 
@@ -60,12 +58,12 @@ import org.apache.cassandra.db.Compactio
  * documentation).
  */
 
-public class CassandraDaemon
+public class CassandraDaemon extends org.apache.cassandra.service.AbstractCassandraDaemon
 {
     private static Logger logger = LoggerFactory.getLogger(CassandraDaemon.class);
     private TServer serverEngine;
 
-    private void setup() throws IOException, TTransportException
+    protected void setup() throws IOException
     {
         int listenPort = DatabaseDescriptor.getRpcPort();
         InetAddress listenAddr = DatabaseDescriptor.getRpcAddress();
@@ -140,7 +138,17 @@ public class CassandraDaemon
         Cassandra.Processor processor = new Cassandra.Processor(cassandraServer);
 
         // Transport
-        TServerSocket tServerSocket = new TServerSocket(new InetSocketAddress(listenAddr, listenPort));
+        TServerSocket tServerSocket = null;
+        
+        try
+        {
+            tServerSocket = new TServerSocket(new InetSocketAddress(listenAddr, listenPort));
+        } catch (TTransportException e)
+        {
+            throw new IOException(String.format(
+                    "Unable to create thrift socket to %s:%s", listenAddr,
+                    listenPort), e);
+        }
         
         logger.info(String.format("Binding thrift service to %s:%s", listenAddr, listenPort));
 
@@ -192,12 +200,6 @@ public class CassandraDaemon
     }
 
     /** hook for JSVC */
-    public void init(String[] args) throws IOException, TTransportException
-    {  
-        setup();
-    }
-
-    /** hook for JSVC */
     public void start()
     {
         logger.info("Cassandra starting up...");
@@ -214,47 +216,8 @@ public class CassandraDaemon
         serverEngine.stop();
     }
     
-    
-    /** hook for JSVC */
-    public void destroy()
-    {
-        // this is supposed to "destroy any object created in init", but
-        // StorageService et al. are crash-only, so we no-op here.
-    }
-    
     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 (Throwable 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);
-        }
+        new CassandraDaemon().activate();
     }
 }

Modified: cassandra/trunk/test/unit/org/apache/cassandra/service/EmbeddedCassandraServiceTest.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/test/unit/org/apache/cassandra/service/EmbeddedCassandraServiceTest.java?rev=954928&r1=954927&r2=954928&view=diff
==============================================================================
--- cassandra/trunk/test/unit/org/apache/cassandra/service/EmbeddedCassandraServiceTest.java (original)
+++ cassandra/trunk/test/unit/org/apache/cassandra/service/EmbeddedCassandraServiceTest.java Tue Jun 15 15:04:46 2010
@@ -134,7 +134,7 @@ public class EmbeddedCassandraServiceTes
      */
     private Cassandra.Client getClient() throws TTransportException
     {
-        TTransport tr = new TSocket("localhost", 9170);
+        TTransport tr = new TSocket("localhost", DatabaseDescriptor.getRpcPort());
         TProtocol proto = new TBinaryProtocol(tr);
         Cassandra.Client client = new Cassandra.Client(proto);
         tr.open();