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 2010/10/19 17:09:24 UTC

svn commit: r1024262 - in /cassandra/trunk: CHANGES.txt src/java/org/apache/cassandra/cli/CliMain.java src/java/org/apache/cassandra/cli/CliSessionState.java test/unit/org/apache/cassandra/cli/ test/unit/org/apache/cassandra/cli/CliTest.java

Author: jbellis
Date: Tue Oct 19 15:09:24 2010
New Revision: 1024262

URL: http://svn.apache.org/viewvc?rev=1024262&view=rev
Log:
add cli sanity tests.
patch by Pavel Yaskevich; reviewed by jbellis for CASSANDRA-1582

Added:
    cassandra/trunk/test/unit/org/apache/cassandra/cli/
    cassandra/trunk/test/unit/org/apache/cassandra/cli/CliTest.java
Modified:
    cassandra/trunk/CHANGES.txt
    cassandra/trunk/src/java/org/apache/cassandra/cli/CliMain.java
    cassandra/trunk/src/java/org/apache/cassandra/cli/CliSessionState.java

Modified: cassandra/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/cassandra/trunk/CHANGES.txt?rev=1024262&r1=1024261&r2=1024262&view=diff
==============================================================================
--- cassandra/trunk/CHANGES.txt (original)
+++ cassandra/trunk/CHANGES.txt Tue Oct 19 15:09:24 2010
@@ -47,6 +47,7 @@ dev
  * add jmx histogram of sstables accessed per read (CASSANDRA-1624)
  * remove system_rename_column_family and system_rename_keyspace from the
    client API until races can be fixed (CASSANDRA-1630, CASSANDRA-1585)
+ * add cli sanity tests (CASSANDRA-1582)
 
 
 0.7-beta2

Modified: cassandra/trunk/src/java/org/apache/cassandra/cli/CliMain.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/cli/CliMain.java?rev=1024262&r1=1024261&r2=1024262&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/cli/CliMain.java (original)
+++ cassandra/trunk/src/java/org/apache/cassandra/cli/CliMain.java Tue Oct 19 15:09:24 2010
@@ -28,8 +28,7 @@ import org.apache.thrift.transport.TFram
 import org.apache.thrift.transport.TSocket;
 import org.apache.thrift.transport.TTransport;
 
-import java.io.File;
-import java.io.IOException;
+import java.io.*;
 import java.util.Arrays;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -43,11 +42,11 @@ public class CliMain
 {
     public final static String HISTORYFILE = ".cassandra.history";
 
-    private static TTransport transport_ = null;
-    private static Cassandra.Client thriftClient_ = null;
-    private static CliSessionState css_ = new CliSessionState();
-    private static CliClient cliClient_;
-    private static CliCompleter completer_ = new CliCompleter();
+    private static TTransport transport = null;
+    private static Cassandra.Client thriftClient = null;
+    public  static CliSessionState sessionState = new CliSessionState();
+    private static CliClient cliClient;
+    private static CliCompleter completer = new CliCompleter();
 
     /**
      * Establish a thrift connection to cassandra instance
@@ -60,47 +59,45 @@ public class CliMain
 
         TSocket socket = new TSocket(server, port);
 
-        if (transport_ != null)
-            transport_.close();
+        if (transport != null)
+            transport.close();
 
-        if (css_.framed)
+        if (sessionState.framed)
         {
-            transport_ = new TFramedTransport(socket);
+            transport = new TFramedTransport(socket);
         }
         else 
         {
-            transport_ = socket;
+            transport = socket;
         }
 
-        TBinaryProtocol binaryProtocol = new TBinaryProtocol(transport_, true, true);
+        TBinaryProtocol binaryProtocol = new TBinaryProtocol(transport, true, true);
         Cassandra.Client cassandraClient = new Cassandra.Client(binaryProtocol);
 
         try
         {
-            transport_.open();
+            transport.open();
         }
         catch (Exception e)
         {
-            // Should move this to Log4J as well probably...
-            css_.err.format("Exception connecting to %s/%d - %s\n", server, port, e.getMessage());
-
-            if (css_.debug)
+            if (sessionState.debug)
                 e.printStackTrace();
 
-            return;
+            String error = (e.getCause() == null) ? e.getMessage() : e.getCause().getMessage();
+            throw new RuntimeException("Exception connecting to " + server + "/" + port + ". Reason: " + error + ".");
         }
 
-        thriftClient_ = cassandraClient;
-        cliClient_ = new CliClient(css_, thriftClient_);
+        thriftClient = cassandraClient;
+        cliClient = new CliClient(sessionState, thriftClient);
         
-        if (css_.keyspace != null)
+        if (sessionState.keyspace != null)
         {
             try {
-                thriftClient_.set_keyspace(css_.keyspace);
-                cliClient_.setKeyspace(css_.keyspace);
+                thriftClient.set_keyspace(sessionState.keyspace);
+                cliClient.setKeyspace(sessionState.keyspace);
 
                 Set<String> cfnames = new HashSet<String>();
-                KsDef ksd = cliClient_.getKSMetaData(css_.keyspace);
+                KsDef ksd = cliClient.getKSMetaData(sessionState.keyspace);
                 for (CfDef cfd : ksd.cf_defs) {
                     cfnames.add(cfd.name);
                 }
@@ -109,47 +106,47 @@ public class CliMain
             }
             catch (InvalidRequestException e)
             {
-                css_.err.println("Keyspace " + css_.keyspace + " not found");
+                sessionState.err.println("Keyspace " + sessionState.keyspace + " not found");
                 return;
             }
             catch (TException e)
             {
-                css_.err.println("Did you specify 'keyspace'?");
+                sessionState.err.println("Did you specify 'keyspace'?");
                 return;
             }
             catch (NotFoundException e)
             {
-                css_.err.println("Keyspace " + css_.keyspace + " not found");
+                sessionState.err.println("Keyspace " + sessionState.keyspace + " not found");
                 return;
             }
         }
         
-        if ((css_.username != null) && (css_.password != null))
+        if ((sessionState.username != null) && (sessionState.password != null))
         {
             // Authenticate 
             Map<String, String> credentials = new HashMap<String, String>();
-            credentials.put(SimpleAuthenticator.USERNAME_KEY, css_.username);
-            credentials.put(SimpleAuthenticator.PASSWORD_KEY, css_.password);
+            credentials.put(SimpleAuthenticator.USERNAME_KEY, sessionState.username);
+            credentials.put(SimpleAuthenticator.PASSWORD_KEY, sessionState.password);
             AuthenticationRequest authRequest = new AuthenticationRequest(credentials);
             try 
             {
-                thriftClient_.login(authRequest);
-                cliClient_.setUsername(css_.username);
+                thriftClient.login(authRequest);
+                cliClient.setUsername(sessionState.username);
             } 
             catch (AuthenticationException e) 
             {
-                css_.err.println("Exception during authentication to the cassandra node, " +
+                sessionState.err.println("Exception during authentication to the cassandra node, " +
                 		"Verify the keyspace exists, and that you are using the correct credentials.");
                 return;
             } 
             catch (AuthorizationException e) 
             {
-                css_.err.println("You are not authorized to use keyspace: " + css_.keyspace);
+                sessionState.err.println("You are not authorized to use keyspace: " + sessionState.keyspace);
                 return;
             }
             catch (TException e) 
             {
-                css_.err.println("Login failure. Did you specify 'keyspace', 'username' and 'password'?");
+                sessionState.err.println("Login failure. Did you specify 'keyspace', 'username' and 'password'?");
                 return;
             } 
         }
@@ -159,20 +156,20 @@ public class CliMain
 
         try
         {
-            clusterName = thriftClient_.describe_cluster_name();
+            clusterName = thriftClient.describe_cluster_name();
         }
         catch (Exception e)
         {
 
-            css_.err.println("Exception retrieving information about the cassandra node, check you have connected to the thrift port.");
+            sessionState.err.println("Exception retrieving information about the cassandra node, check you have connected to the thrift port.");
 
-            if (css_.debug)
+            if (sessionState.debug)
                 e.printStackTrace();
 
             return;
         }
 
-        css_.out.printf("Connected to: \"%s\" on %s/%d%n", clusterName, server, port);
+        sessionState.out.printf("Connected to: \"%s\" on %s/%d%n", clusterName, server, port);
     }
 
     /**
@@ -180,27 +177,28 @@ public class CliMain
      */
     public static void disconnect()
     {
-        if (transport_ != null)
+        if (transport != null)
         {
-            transport_.close();
-            transport_ = null;
+            transport.close();
+            transport = null;
         }
     }
 
     private static void printBanner()
     {
-        css_.out.println("Welcome to cassandra CLI.\n");
-        css_.out.println("Type 'help' or '?' for help. Type 'quit' or 'exit' to quit.");
+        sessionState.out.println("Welcome to cassandra CLI.\n");
+        sessionState.out.println("Type 'help' or '?' for help. Type 'quit' or 'exit' to quit.");
     }
 
     /**
      * Checks whether the thrift client is connected.
+     * @return boolean - true when connected, false otherwise
      */
     public static boolean isConnected()
     {
-        if (thriftClient_ == null)
+        if (thriftClient == null)
         {
-            css_.out.println("Not connected to a cassandra instance.");
+            sessionState.out.println("Not connected to a cassandra instance.");
             return false;
         }
         return true;
@@ -211,68 +209,76 @@ public class CliMain
         Set<String> actions = new HashSet<String>();
         for (String cf : candidates)
         {
-            for (String cmd : completer_.getKeyspaceCommands())
+            for (String cmd : completer.getKeyspaceCommands())
                 actions.add(String.format("%s %s", cmd, cf));
         }
         
         String[] strs = Arrays.copyOf(actions.toArray(), actions.toArray().length, String[].class);
         
-        completer_.setCandidateStrings(strs);
+        completer.setCandidateStrings(strs);
     }
 
-    private static void processCLIStmt(String query)
+    public static void processStatement(String query)
     {
         try
         {
-            cliClient_.executeCLIStmt(query);
+            cliClient.executeCLIStmt(query);
         }
         catch (InvalidRequestException ire)
         {
-            css_.err.println(ire.why);
-            if (css_.debug)
-                ire.printStackTrace(css_.err);
+            sessionState.err.println(ire.why);
+            if (sessionState.debug)
+                ire.printStackTrace(sessionState.err);
             
             // Abort a batch run when errors are encountered
-            if (css_.batch)
+            if (sessionState.batch)
                 System.exit(4);
         }
         catch (Throwable e)
         {
-            css_.err.println((e.getCause() == null) ? e.getMessage() : e.getCause().getMessage());
+            sessionState.err.println((e.getCause() == null) ? e.getMessage() : e.getCause().getMessage());
             
-            if (css_.debug)
-                e.printStackTrace(css_.err);
+            if (sessionState.debug)
+                e.printStackTrace(sessionState.err);
             
             // Abort a batch run when errors are encountered
-            if (css_.batch)
+            if (sessionState.batch)
                 System.exit(8);
         }
     }
 
     public static void main(String args[]) throws IOException
     {
-        // process command line args
+        // process command line arguments
         CliOptions cliOptions = new CliOptions();
-        cliOptions.processArgs(css_, args);
+        cliOptions.processArgs(sessionState, args);
 
         // connect to cassandra server if host argument specified.
-        if (css_.hostName != null)
+        if (sessionState.hostName != null)
         {
-            connect(css_.hostName, css_.thriftPort);
+            try
+            {
+                connect(sessionState.hostName, sessionState.thriftPort);   
+            }
+            catch (RuntimeException e)
+            {
+                sessionState.err.println(e.getMessage());
+                System.exit(-1);
+            }
         }
         
-        if ( cliClient_ == null )
+        if ( cliClient == null )
         {
             // Connection parameter was either invalid or not present.
             // User must connect explicitly using the "connect" CLI statement.
-            cliClient_ = new CliClient(css_, null);
+            cliClient = new CliClient(sessionState, null);
         }
 
         ConsoleReader reader = new ConsoleReader();
         
-        if (!css_.batch)
+        if (!sessionState.batch)
         {
-            reader.addCompletor(completer_);
+            reader.addCompletor(completer);
             reader.setBellEnabled(false);
             
             String historyFile = System.getProperty("user.home") + File.separator + HISTORYFILE;
@@ -284,20 +290,31 @@ public class CliMain
             }
             catch (IOException exp)
             {
-                css_.err.printf("Unable to open %s for writing %n", historyFile);
+                sessionState.err.printf("Unable to open %s for writing %n", historyFile);
             }
         }
         else
         {
-            css_.out.close();
+            sessionState.out.close();
         }
 
         printBanner();
 
         String line;
-        while ((line = reader.readLine("[" + cliClient_.getUsername() + "@" + cliClient_.getKeySpace() + "] ")) != null)
+        while ((line = reader.readLine(getPrompt(cliClient))) != null)
         {
-            processCLIStmt(line);
+            processStatement(line);
         }
     }
+
+    /**
+     * Returns prompt for current connection
+     * @param client - currently connected client
+     * @return String - prompt with username and keyspace (if any)
+     */
+    private static String getPrompt(CliClient client)
+    {
+        return "[" + client.getUsername() + "@" + client.getKeySpace() + "] ";
+    }
+
 }

Modified: cassandra/trunk/src/java/org/apache/cassandra/cli/CliSessionState.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/cli/CliSessionState.java?rev=1024262&r1=1024261&r2=1024262&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/cli/CliSessionState.java (original)
+++ cassandra/trunk/src/java/org/apache/cassandra/cli/CliSessionState.java Tue Oct 19 15:09:24 2010
@@ -48,4 +48,14 @@ public class CliSessionState
         out = System.out;
         err = System.err;
     }
+
+    public void setOut(PrintStream newOut)
+    {
+        this.out = newOut;   
+    }
+
+    public void setErr(PrintStream newErr)
+    {
+        this.err = newErr;
+    }
 }

Added: cassandra/trunk/test/unit/org/apache/cassandra/cli/CliTest.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/test/unit/org/apache/cassandra/cli/CliTest.java?rev=1024262&view=auto
==============================================================================
--- cassandra/trunk/test/unit/org/apache/cassandra/cli/CliTest.java (added)
+++ cassandra/trunk/test/unit/org/apache/cassandra/cli/CliTest.java Tue Oct 19 15:09:24 2010
@@ -0,0 +1,116 @@
+/**
+ * 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.cli;
+
+import junit.framework.TestCase;
+import org.apache.cassandra.config.ConfigurationException;
+import org.apache.cassandra.service.EmbeddedCassandraService;
+import org.apache.thrift.transport.TTransportException;
+import org.junit.Test;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.PrintStream;
+
+public class CliTest extends TestCase
+{
+    // please add new statements here so they could be auto-runned by this test.
+    private String[] statements = {
+        "use TestKeySpace",
+        "create column family CF1 with comparator=UTF8Type and column_metadata=[{ column_name:world, validation_class:IntegerType }]",
+        "set CF1[hello][world] = 123848374878933948398384",
+        "get CF1[hello][world]",
+        "set CF1['hello'][time_spent_uuid] = timeuuid(a8098c1a-f86e-11da-bd1a-00112444be1e)",
+        "get CF1['hello'][time_spent_uuid] as LexicalUUIDType",
+        "create column family CF2 with comparator=IntegerType",
+        "set CF2['key'][98349387493847748398334] = 'some text'",
+        "get CF2['key'][98349387493847748398334]",
+        "set CF2['key'][98349387493] = 'some text other'",
+        "get CF2['key'][98349387493]",
+        "create column family CF3 with comparator=UTF8Type and column_metadata=[{column_name:'big world', validation_class:LongType}]",
+        "set CF3['hello']['big world'] = 3748",
+        "get CF3['hello']['big world']",
+        "update keyspace TestKeySpace with placement_strategy='org.apache.cassandra.locator.LocalStrategy'",
+        "update keyspace TestKeySpace with replication_factor=1 and strategy_options=[{DC1:3, DC2:4, DC5:1}]"
+    };
+    
+    @Test
+    public void testCli() throws IOException, TTransportException, ConfigurationException
+    {
+        setup();
+
+        // new error/output streams for CliSessionState
+        ByteArrayOutputStream errStream = new ByteArrayOutputStream();
+        ByteArrayOutputStream outStream = new ByteArrayOutputStream();
+
+        // checking if we can connect to the running cassandra node on localhost
+        CliMain.connect("127.0.0.1", 9170);
+
+        // setting new output stream
+        CliMain.sessionState.setOut(new PrintStream(outStream));
+        CliMain.sessionState.setErr(new PrintStream(errStream));
+
+        // re-creating keyspace for tests
+        // dropping in case it exists e.g. could be left from previous run
+        CliMain.processStatement("drop keyspace TestKeySpace");
+        CliMain.processStatement("create keyspace TestKeySpace");
+
+        for (String statement : statements)
+        {
+            CliMain.processStatement(statement);
+            String result = outStream.toString();
+
+            if (statement.startsWith("drop ") || statement.startsWith("create ") || statement.startsWith("update "))
+            {
+                assertTrue(result.matches("(.{8})-(.{4})-(.{4})-(.{4})-(.{12})\n"));
+            }
+            else if (statement.startsWith("set "))
+            {
+                assertEquals(result, "Value inserted.\n");
+            }
+            else if (statement.startsWith("get "))
+            {
+                assertTrue(result.startsWith("=> (column="));
+            }
+
+            outStream.reset(); // reset stream so we have only output from next statement all the time
+            errStream.reset(); // no errors to the end user.
+        }
+    }
+
+    /**
+     * Setup embedded cassandra instance using test config.
+     * @throws TTransportException - when trying to bind address
+     * @throws IOException - when reading config file
+     * @throws ConfigurationException - when can set up configuration
+     */
+    private void setup() throws TTransportException, IOException, ConfigurationException
+    {
+        EmbeddedCassandraService cassandra;
+
+        cassandra = new EmbeddedCassandraService();
+        cassandra.init();
+
+        // spawn cassandra in a new thread
+        Thread t = new Thread(cassandra);
+        t.setDaemon(true);
+        t.start();
+    }
+
+}