You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by "Gary Dusbabek (JIRA)" <ji...@apache.org> on 2010/01/26 13:50:34 UTC
[jira] Commented: (CASSANDRA-740) Create InProcessCassandraServer
for unit tests
[ https://issues.apache.org/jira/browse/CASSANDRA-740?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12805010#action_12805010 ]
Gary Dusbabek commented on CASSANDRA-740:
-----------------------------------------
Have you found that all threads associated with spinning up cassandra are stopped when CassandraDaemon.stop() is called? I was working on something a few months ago and discovered that there were still large parts of cassandra that were not reentrant, meaning that it was nearly impossible to start a cassanrdra process, shut it down, and then start it up again all within the same jvm instance.
> Create InProcessCassandraServer for unit tests
> ----------------------------------------------
>
> Key: CASSANDRA-740
> URL: https://issues.apache.org/jira/browse/CASSANDRA-740
> Project: Cassandra
> Issue Type: New Feature
> Components: Contrib
> Reporter: Ran Tavory
> Fix For: 0.6
>
>
> I've been personally using an in-process cassandra server and found it useful so I was ask to make it available publicly, here goes.
> When unit-testing with cassandra I create an in process cassandra instance. That's nice since it lets you isolate tests, and you don't have to worry about a server being available for your unit tests.
> The code goes more or less like this (I'll attach a patch when the work is done after cleanup etc)
> /**
> * An in-memory cassandra storage service that listens to the thrift interface.
> * Useful for unit testing,
> *
> * @author Ran Tavory (rantav@gmail.com)
> *
> */
> public class InProcessCassandraServer implements Runnable {
> private static final Logger log = LoggerFactory.getLogger(InProcessCassandraServer.class);
> CassandraDaemon cassandraDaemon;
> public void init() {
> try {
> prepare();
> } catch (IOException e) {
> log.error("Cannot prepare cassandra.", e);
> }
> try {
> cassandraDaemon = new CassandraDaemon();
> cassandraDaemon.init(null);
> } catch (TTransportException e) {
> log.error("TTransportException", e);
> } catch (IOException e) {
> log.error("IOException", e);
> }
> }
> @Override
> public void run() {
> cassandraDaemon.start();
> }
> public void stop() {
> cassandraDaemon.stop();
> rmdir("tmp");
> }
> /**
> * Creates all files and directories needed
> * @throws IOException
> */
> private void prepare() throws IOException {
> // delete tmp dir first
> rmdir("tmp");
> // make a tmp dir and copy storag-conf.xml and log4j.properties to it
> copy("/cassandra/storage-conf.xml", "tmp");
> copy("/cassandra/log4j.properties", "tmp");
> System.setProperty("storage-config", "tmp");
> // make cassandra directories.
> for (String s: DatabaseDescriptor.getAllDataFileLocations()) {
> mkdir(s);
> }
> mkdir(DatabaseDescriptor.getBootstrapFileLocation());
> mkdir(DatabaseDescriptor.getLogFileLocation());
> }
> /**
> * Copies a resource from within the jar to a directory.
> *
> * @param resourceName
> * @param directory
> * @throws IOException
> */
> private void copy(String resource, String directory) throws IOException {
> mkdir(directory);
> InputStream is = getClass().getResourceAsStream(resource);
> String fileName = resource.substring(resource.lastIndexOf("/") + 1);
> File file = new File(directory + System.getProperty("file.separator") + fileName);
> OutputStream out = new FileOutputStream(file);
> byte buf[] = new byte[1024];
> int len;
> while ((len = is.read(buf)) > 0) {
> out.write(buf, 0, len);
> }
> out.close();
> is.close();
> }
> /**
> * Creates a directory
> * @param dir
> * @throws IOException
> */
> private void mkdir(String dir) throws IOException {
> FileUtils.createDirectory(dir);
> }
> /**
> * Removes a directory from file system
> * @param dir
> */
> private void rmdir(String dir) {
> FileUtils.deleteDir(new File(dir));
> }
> }
> And test code using this class looks like this:
> public class XxxTest {
> private static InProcessCassandraServer cassandra;
> @BeforeClass
> public static void setup() throws TTransportException, IOException, InterruptedException {
> cassandra = new InProcessCassandraServer();
> cassandra.init();
> Thread t = new Thread(cassandra);
> t.setDaemon(true);
> t.start();
> }
> @AfterClass
> public static void shutdown() {
> cassandra.stop();
> }
> public void testX() {
> // connect to cassandra at localhost:9160
> }
> }
> note: I've set Fix Version to 6.0, hope it's correct
--
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.