You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by pm...@apache.org on 2009/03/02 08:57:31 UTC

svn commit: r749218 [3/34] - in /incubator/cassandra: branches/ dist/ nightly/ site/ tags/ trunk/ trunk/lib/ trunk/src/ trunk/src/org/ trunk/src/org/apache/ trunk/src/org/apache/cassandra/ trunk/src/org/apache/cassandra/analytics/ trunk/src/org/apache/...

Added: incubator/cassandra/trunk/src/org/apache/cassandra/cli/CliMain.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/org/apache/cassandra/cli/CliMain.java?rev=749218&view=auto
==============================================================================
--- incubator/cassandra/trunk/src/org/apache/cassandra/cli/CliMain.java (added)
+++ incubator/cassandra/trunk/src/org/apache/cassandra/cli/CliMain.java Mon Mar  2 07:57:22 2009
@@ -0,0 +1,190 @@
+/**
+ * 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 com.facebook.thrift.protocol.TBinaryProtocol;
+import com.facebook.thrift.transport.TSocket;
+import com.facebook.thrift.transport.TTransport;
+
+import jline.*;
+import java.io.*;
+import java.util.*;
+
+import org.apache.cassandra.service.Cassandra;
+
+//
+// Cassandra Command Line Interface (CLI) Main
+//
+public class CliMain
+{
+    public final static String PROMPT = "cassandra";
+    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_;
+
+    // Establish a thrift connection to cassandra instance
+    public static void connect(String server, int port)
+    {
+        TSocket socket = new TSocket(server, port);
+
+        if (transport_ != null)
+            transport_.close();
+
+        transport_ = socket;
+
+        TBinaryProtocol binaryProtocol = new TBinaryProtocol(transport_, false, false);
+        Cassandra.Client cassandraClient = new Cassandra.Client(binaryProtocol);
+
+        try
+        {
+            transport_.open();
+        }
+        catch(Exception e)
+        {
+            // Should move this to Log4J as well probably...
+            System.err.println("Exception " + e.getMessage());            
+            e.printStackTrace();
+        }
+
+        thriftClient_ = cassandraClient;
+        cliClient_ = new CliClient(css_, thriftClient_);
+
+        css_.out.printf("Connected to %s/%d\n", server, port);
+    }
+
+    // Disconnect thrift connection to cassandra instance
+    public static void disconnect()
+    {
+        if (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.");
+    }
+
+    public static boolean isConnected()
+    {
+        if (thriftClient_ == null)
+        {
+            css_.out.println("Not connected to a cassandra instance.");
+            return false;
+        }
+        return true;
+    }
+    
+    private static void processServerQuery(String query)
+    {
+        if (!isConnected())
+            return;
+
+        try
+        {
+            cliClient_.executeQueryOnServer(query);
+        }
+        catch(Exception e)
+        {
+            System.err.println("Exception " + e.getMessage());
+            e.printStackTrace(System.err);
+        }
+        return;
+    }
+
+    private static void processCLIStmt(String query)
+    {
+        try
+        {
+            cliClient_.executeCLIStmt(query);
+        }
+        catch(Exception e)
+        {
+            System.err.println("Exception " + e.getMessage());
+            e.printStackTrace(System.err);
+        }
+        return;
+    }
+
+    private static void processLine(String line)
+    {
+        StringTokenizer tokenizer = new StringTokenizer(line);
+        if (tokenizer.hasMoreTokens())
+        {
+            // Use first token for now to determine if this statement is
+            // a CQL statement. Technically, the line could start with
+            // a comment token followed by a CQL statement. That case
+            // isn't handled right now.
+            String token = tokenizer.nextToken().toUpperCase();
+            if (token.startsWith("GET")
+                || token.startsWith("SELECT")
+                || token.startsWith("SET")
+                || token.startsWith("DELETE")
+                || token.startsWith("EXPLAIN")) // explain plan statement
+            {
+                // these are CQL Statements that are compiled and executed on server-side
+                processServerQuery(line);
+            }
+            else 
+            {
+                // These are CLI statements processed locally
+                processCLIStmt(line);
+            }
+        }
+    } 
+
+    public static void main(String args[]) throws IOException  
+    {
+        // process command line args
+        CliOptions cliOptions = new CliOptions();
+        cliOptions.processArgs(css_, args);
+
+        // connect to cassandra server if host argument specified.
+        if (css_.hostName != null)
+        {
+            connect(css_.hostName, css_.thriftPort);
+        }
+        else
+        {
+            // If not, client must connect explicitly using the "connect" CLI statement.
+            cliClient_ = new CliClient(css_, null);
+        }
+
+        ConsoleReader reader = new ConsoleReader(); 
+        reader.setBellEnabled(false);
+
+        String historyFile = System.getProperty("user.home") + File.separator  + HISTORYFILE;
+
+        reader.setHistory(new History(new File(historyFile)));
+
+        printBanner();
+
+        String line;
+        while ((line = reader.readLine(PROMPT+"> ")) != null)
+        {
+            processLine(line);
+        }
+    }
+}

Added: incubator/cassandra/trunk/src/org/apache/cassandra/cli/CliOptions.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/org/apache/cassandra/cli/CliOptions.java?rev=749218&view=auto
==============================================================================
--- incubator/cassandra/trunk/src/org/apache/cassandra/cli/CliOptions.java (added)
+++ incubator/cassandra/trunk/src/org/apache/cassandra/cli/CliOptions.java Mon Mar  2 07:57:22 2009
@@ -0,0 +1,91 @@
+/**
+ * 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 org.apache.commons.cli.*;
+
+public class CliOptions {
+
+    private static Options options = null; // Info about command line options
+    private CommandLine cmd = null;        // Command Line arguments
+
+    // Command line options
+    private static final String HOST_OPTION = "host";
+    private static final String PORT_OPTION = "port";
+
+    // Default values for optional command line arguments
+    private static final int    DEFAULT_THRIFT_PORT = 9160;
+
+    // Register the command line options and their properties (such as
+    // whether they take an extra argument, etc.
+    static
+    {
+        options = new Options();
+        options.addOption(HOST_OPTION, true, "cassandra server's host name");
+        options.addOption(PORT_OPTION, true, "cassandra server's thrift port");  
+    }
+
+    private static void printUsage()
+    {
+        System.err.println("");
+        System.err.println("Usage: cascli --host hostname [--port <portname>]");
+        System.err.println("");
+    }
+
+    public void processArgs(CliSessionState css, String[] args)
+    {
+        CommandLineParser parser = new PosixParser();
+        try
+        {
+            cmd = parser.parse(options, args);
+        }
+        catch (ParseException e)
+        {
+            printUsage();
+            e.printStackTrace();
+            System.exit(1);
+        }
+
+        if (!cmd.hasOption(HOST_OPTION))
+        {
+            // host name not specified in command line.
+            // In this case, we don't implicitly connect at CLI startup. In this case,
+            // the user must use the "connect" CLI statement to connect.
+            //
+            css.hostName = null;
+            
+            // HelpFormatter formatter = new HelpFormatter();
+            // formatter.printHelp("java com.facebook.infrastructure.cli.CliMain ", options);
+            // System.exit(1);
+        }
+        else 
+        {
+            css.hostName = cmd.getOptionValue(HOST_OPTION);
+        }
+
+        // Look for optional args.
+        if (cmd.hasOption(PORT_OPTION))
+        {
+            css.thriftPort = Integer.parseInt(cmd.getOptionValue(PORT_OPTION));
+        }
+        else
+        {
+            css.thriftPort = DEFAULT_THRIFT_PORT;
+        }
+    }
+}