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 2011/11/08 00:05:55 UTC

svn commit: r1198994 [1/3] - in /cassandra/branches/cassandra-1.0: ./ bin/ doc/cql/ pylib/ pylib/cqlshlib/

Author: jbellis
Date: Mon Nov  7 23:05:54 2011
New Revision: 1198994

URL: http://svn.apache.org/viewvc?rev=1198994&view=rev
Log:
add next-gen cqlsh
patch by Paul Cannon; reviewed by jbellis for CASSANDRA-3188

Added:
    cassandra/branches/cassandra-1.0/bin/cqlsh
    cassandra/branches/cassandra-1.0/bin/cqlshrc.sample
    cassandra/branches/cassandra-1.0/doc/cql/CQL.css
    cassandra/branches/cassandra-1.0/pylib/
    cassandra/branches/cassandra-1.0/pylib/cqlshlib/
    cassandra/branches/cassandra-1.0/pylib/cqlshlib/cqlhandling.py
    cassandra/branches/cassandra-1.0/pylib/cqlshlib/pylexotron.py
    cassandra/branches/cassandra-1.0/pylib/cqlshlib/saferscanner.py
Modified:
    cassandra/branches/cassandra-1.0/CHANGES.txt
    cassandra/branches/cassandra-1.0/build.xml
    cassandra/branches/cassandra-1.0/doc/cql/CQL.textile

Modified: cassandra/branches/cassandra-1.0/CHANGES.txt
URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-1.0/CHANGES.txt?rev=1198994&r1=1198993&r2=1198994&view=diff
==============================================================================
--- cassandra/branches/cassandra-1.0/CHANGES.txt (original)
+++ cassandra/branches/cassandra-1.0/CHANGES.txt Mon Nov  7 23:05:54 2011
@@ -1,6 +1,7 @@
 1.0.3
  * (Hadoop) Fix empty row filtering (CASSANDRA-3450)
  * fix invalidate-related test failures (CASSANDRA-3437)
+ * add next-gen cqlsh to bin/
 Merged from 0.8:
  * Make counter shard merging thread safe (CASSANDRA-3178)
 

Added: cassandra/branches/cassandra-1.0/bin/cqlsh
URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-1.0/bin/cqlsh?rev=1198994&view=auto
==============================================================================
--- cassandra/branches/cassandra-1.0/bin/cqlsh (added)
+++ cassandra/branches/cassandra-1.0/bin/cqlsh Mon Nov  7 23:05:54 2011
@@ -0,0 +1,1500 @@
+#!/usr/bin/env python
+
+# 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.
+
+description = "CQL Shell for Apache Cassandra"
+version = "2.0.0"
+
+from collections import defaultdict
+from StringIO import StringIO
+from itertools import groupby
+from functools import partial
+
+import cmd
+import sys
+import os
+import string
+import time
+import optparse
+import ConfigParser
+
+# cqlsh should run correctly when run out of a Cassandra source tree,
+# out of an unpacked Cassandra tarball, and after a proper package install.
+cqlshlibdir = os.path.join(os.path.dirname(__file__), '..', 'pylib')
+if os.path.isdir(cqlshlibdir):
+    sys.path.insert(0, cqlshlibdir)
+
+from cqlshlib import cqlhandling, pylexotron
+from cqlshlib.cqlhandling import (token_dequote, cql_dequote, cql_escape,
+                                  maybe_cql_escape, cql_typename)
+
+try:
+    import readline
+except ImportError:
+    readline = None
+
+try:
+    import cql
+except ImportError:
+    sys.stderr.write("\nPython CQL driver not installed, or not on PYTHONPATH.\n")
+    sys.stderr.write('You might try "easy_install cql".\n\n')
+    sys.exit(1)
+
+import cql.decoders
+from cql.cursor import _COUNT_DESCRIPTION, _VOID_DESCRIPTION
+
+
+CONFIG_FILE = os.path.expanduser(os.path.join('~', '.cqlshrc'))
+HISTORY = os.path.expanduser(os.path.join('~', '.cqlsh_history'))
+DEFAULT_HOST = 'localhost'
+DEFAULT_PORT = 9160
+
+epilog = """Connects to %(DEFAULT_HOST)s:%(DEFAULT_PORT)d by default. These
+defaults can be changed by setting $CQLSH_HOST and/or $CQLSH_PORT. When a
+host (and optional port number) are given on the command line, they take
+precedence over any defaults.""" % globals()
+
+parser = optparse.OptionParser(description=description, epilog=epilog,
+                               usage="Usage: %prog [options] [host [port]]",
+                               version='cqlsh ' + version)
+parser.add_option("-C", "--color", action="store_true",
+                  help="Enable color output.")
+parser.add_option("-u", "--username", help="Authenticate as user.")
+parser.add_option("-p", "--password", help="Authenticate using password.")
+parser.add_option('--debug', action='store_true',
+                  help='Show additional debugging information')
+
+
+RED = "\033[1;31m%s\033[0m"
+GREEN = "\033[1;32m%s\033[0m"
+BLUE = "\033[1;34m%s\033[0m"
+YELLOW = "\033[1;33m%s\033[0m"
+CYAN = "\033[1;36m%s\033[0m"
+MAGENTA = "\033[1;35m%s\033[0m"
+
+CQL_ERRORS = (cql.Error,)
+try:
+    from thrift.Thrift import TException
+except ImportError:
+    pass
+else:
+    CQL_ERRORS += (TException,)
+
+debug_completion = bool(os.environ.get('CQLSH_DEBUG_COMPLETION', '') == 'YES')
+
+# we want the cql parser to understand our cqlsh-specific commands too
+cqlhandling.commands_end_with_newline.update((
+    'help',
+    '?',
+    'describe',
+    'show',
+    'assume',
+    'eof',
+    'exit',
+    'quit'
+))
+
+cqlhandling.CqlRuleSet.append_rules(r'''
+<cqlshCommand> ::= <CQL_Statement>
+                 | <specialCommand> ( ";" | "\n" )
+                 ;
+<specialCommand> ::= <describeCommand>
+                   | <showCommand>
+                   | <assumeCommand>
+                   | <helpCommand>
+                   | <exitCommand>
+                   ;
+
+<describeCommand> ::= "DESCRIBE" ( "KEYSPACE" ksname=<name>?
+                                  | "COLUMNFAMILY" cfname=<name>
+                                  | "SCHEMA"
+                                  | "CLUSTER" )
+                    ;
+
+<showCommand> ::= "SHOW" what=( "VERSION" | "HOST" | "ASSUMPTIONS" )
+                ;
+
+<assumeCommand> ::= "ASSUME" ( ks=<name> "." )? cf=<name> <assumeTypeDef>
+                                                          ( "," <assumeTypeDef> )*
+                  ;
+
+<assumeTypeDef> ::= "NAMES" "ARE" names=<storageType>
+                  | "VALUES" "ARE" values=<storageType>
+                  | "(" colname=<name> ")" "VALUES" "ARE" colvalues=<storageType>
+                  ;
+
+<helpCommand> ::= "HELP" [topic]=( <identifier> | <stringLiteral> )*
+                | "?"
+                ;
+
+<exitCommand> ::= ( eof=@"EOF" | "exit" | "quit" )
+                ;
+
+<qmark> ::= "?" ;
+''')
+
+@cqlhandling.cql_add_completer('exitCommand', 'eof')
+def hide_eof_from_completion(ctxt, cqlsh):
+    return ()
+
+@cqlhandling.cql_add_completer('helpCommand', 'topic')
+def complete_help(ctxt, cqlsh):
+    helpfuncs = [n[5:].upper() for n in cqlsh.get_names() if n.startswith('help_')]
+    funcs_with_docstrings = [n[3:].upper() for n in cqlsh.get_names()
+                             if n.startswith('do_') and getattr(cqlsh, n, None).__doc__]
+    return sorted(helpfuncs + funcs_with_docstrings)
+
+@cqlhandling.cql_add_completer('describeCommand', 'ksname')
+def complete_describe_ks(ctxt, cqlsh):
+    return map(maybe_cql_escape, cqlsh.get_keyspace_names())
+
+@cqlhandling.cql_add_completer('describeCommand', 'cfname')
+def complete_describe_cf(ctxt, cqlsh):
+    return map(maybe_cql_escape, cqlsh.get_columnfamily_names())
+
+@cqlhandling.cql_add_completer('assumeCommand', 'ks')
+def complete_assume_ks(ctxt, cqlsh):
+    return [maybe_cql_escape(ks) + '.' for ks in cqlsh.get_keyspace_names()]
+
+@cqlhandling.cql_add_completer('assumeCommand', 'cf')
+def complete_assume_cf(ctxt, cqlsh):
+    ks = ctxt.get_binding('ks', None)
+    if ks is not None:
+        ks = cql_dequote(ks)
+    return map(maybe_cql_escape, cqlsh.get_columnfamily_names(ks))
+
+@cqlhandling.cql_add_completer('assumeTypeDef', 'colname')
+def complete_assume_col(ctxt, cqlsh):
+    ks = ctxt.get_binding('ks', None)
+    ks = cql_dequote(ks) if ks is not None else None
+    cf = cql_dequote(ctxt.get_binding('cf'))
+    cfdef = cqlsh.get_columnfamily(cf, ksname=ks)
+    return map(maybe_cql_escape, [cm.name for cm in cfdef.column_metadata])
+
+class NoKeyspaceError(Exception):
+    pass
+
+def trim_if_present(s, prefix):
+    if s.startswith(prefix):
+        return s[len(prefix):]
+    return s
+
+class Shell(cmd.Cmd):
+    default_prompt  = "cqlsh> "
+    continue_prompt = "   ... "
+    keyspace_prompt          = "cqlsh:%s> "
+    keyspace_continue_prompt = "%s    ... "
+    num_retries = 4
+    debug = False
+
+    def __init__(self, hostname, port, color=False, username=None,
+            password=None, completekey='tab'):
+        cmd.Cmd.__init__(self, completekey=completekey)
+        self.hostname = hostname
+        self.port = port
+        self.conn = cql.connect(hostname, port, user=username, password=password)
+        self.cursor = self.conn.cursor()
+
+        self.current_keyspace = None
+
+        if sys.stdin.isatty():
+            self.prompt = Shell.default_prompt
+        else:
+            self.prompt = ""
+
+        self.statement = StringIO()
+        self.color = color
+        self.in_comment = False
+        self.schema_overrides = {}
+
+        if os.isatty(0):
+            self.report_connection()
+            self.printout('Use HELP for help.')
+
+    def report_connection(self):
+        self.show_host()
+        self.show_version()
+
+    def show_host(self):
+        self.printout("Connected to ", newline=False)
+        self.printout(self.get_cluster_name(), color=BLUE, newline=False)
+        self.printout(" at %s:%d." % (self.hostname, self.port))
+
+    def show_version(self):
+        vers = self.get_cluster_versions()
+        vers['shver'] = version
+        self.printout("[cqlsh %(shver)s | Cassandra %(build)s | CQL spec %(cql)s | Thrift protocol %(thrift)s]" % vers)
+
+    def show_assumptions(self):
+        all_overrides = self.schema_overrides.items()
+        all_overrides.sort()
+        if all_overrides:
+            self.printout('')
+        else:
+            self.printout('No overrides.')
+            return
+        for keyspace, ksoverrides in groupby(all_overrides, key=lambda x:x[0][0]):
+            keyspace = maybe_cql_escape(keyspace)
+            self.printout('USE %s;' % keyspace)
+            self.printout('')
+            for (ks, cf), override in ksoverrides:
+                cf = maybe_cql_escape(cf)
+                if override.default_name_type:
+                    self.printout('ASSUME %s NAMES ARE %s;'
+                                  % (cf, cql_typename(override.default_name_type)))
+                if override.default_value_type:
+                    self.printout('ASSUME %s VALUES ARE %s;'
+                                  % (cf, cql_typename(override.default_value_type)))
+                for colname, vtype in override.value_types.items():
+                    colname = maybe_cql_escape(colname)
+                    self.printout('ASSUME %s(%s) VALUES ARE %s;'
+                                  % (cf, colname, cql_typename(vtype)))
+        self.printout('')
+
+    def get_cluster_versions(self):
+        try:
+            self.cursor.execute('select component, version from system.Versions')
+            vers = dict(self.cursor)
+        except cql.ProgrammingError:
+            # older Cassandra; doesn't have system.Versions
+            thrift_ver = self.get_thrift_version()
+            return {'build': 'unknown', 'cql': 'unknown', 'thrift': thrift_ver}
+        return vers
+
+    def get_keyspace_names(self):
+        return [k.name for k in self.get_keyspaces()]
+
+    def get_columnfamilies(self, ksname=None):
+        if ksname is None:
+            ksname = self.current_keyspace
+            if ksname is None:
+                raise NoKeyspaceError("Not in any keyspace.")
+        return self.get_keyspace(ksname).cf_defs
+
+    def get_columnfamily(self, cfname, ksname=None):
+        if ksname is None:
+            ksname = self.current_keyspace
+        cf_defs = self.get_columnfamilies(ksname)
+        for c in cf_defs:
+            if c.name == cfname:
+                return c
+        raise KeyError("Unconfigured column family %r" % (cfname,))
+
+    def get_columnfamily_names(self, ksname=None):
+        return [c.name for c in self.get_columnfamilies(ksname)]
+
+    def get_index_names(self, ksname=None):
+        indnames = []
+        for c in self.get_columnfamilies(ksname):
+            for md in c.column_metadata:
+                if md.index_name is not None:
+                    indnames.append(md.index_name)
+        return indnames
+
+    def filterable_column_names(self, cfname, ksname=None):
+        cfdef = self.get_columnfamily(cfname, ksname=ksname)
+        filterable = set()
+        if cfdef.key_alias is not None and cfdef.key_alias != 'KEY':
+            filterable.add(cfdef.key_alias)
+        else:
+            filterable.add('KEY')
+        for cm in cfdef.column_metadata:
+            if cm.index_name is not None:
+                filterable.add(cm.name)
+        return filterable
+
+    # ===== thrift-dependent parts =====
+
+    def get_cluster_name(self):
+        return self.make_hacktastic_thrift_call('describe_cluster_name')
+
+    def get_partitioner(self):
+        return self.make_hacktastic_thrift_call('describe_partitioner')
+
+    def get_snitch(self):
+        return self.make_hacktastic_thrift_call('describe_snitch')
+
+    def get_thrift_version(self):
+        return self.make_hacktastic_thrift_call('describe_version')
+
+    def get_ring(self):
+        if self.current_keyspace is None:
+            raise NoKeyspaceError("Ring view requires a current non-system keyspace")
+        return self.make_hacktastic_thrift_call('describe_ring', self.current_keyspace)
+
+    def get_keyspace(self, ksname):
+        return self.make_hacktastic_thrift_call('describe_keyspace', ksname)
+
+    def get_keyspaces(self):
+        return self.make_hacktastic_thrift_call('describe_keyspaces')
+
+    def get_schema_versions(self):
+        return self.make_hacktastic_thrift_call('describe_schema_versions')
+
+    def make_hacktastic_thrift_call(self, call, *args):
+        client = self.cursor._connection.client
+        return getattr(client, call)(*args)
+
+    # ===== end thrift-dependent parts =====
+
+    def reset_statement(self):
+        if self.current_keyspace is None:
+            self.set_prompt(Shell.default_prompt)
+        else:
+            self.set_prompt(Shell.keyspace_prompt % self.current_keyspace)
+        self.statement.truncate(0)
+
+    def continue_statement(self):
+        if self.current_keyspace is None:
+            self.set_prompt(Shell.continue_prompt)
+        else:
+            spaces = ' ' * len(str(self.current_keyspace))
+            self.set_prompt(Shell.keyspace_continue_prompt % spaces)
+
+    def precmd(self, line):
+        self.statement.write(line + '\n')
+        return self.statement.getvalue()
+
+    def onecmd(self, line):
+        try:
+            statements, in_batch = cqlhandling.cql_split_statements(line)
+        except pylexotron.LexingError, e:
+            self.printerr('Invalid syntax at line %d, char %d' % (e.linenum, e.charnum))
+            line = line.split('\n')[e.linenum - 1]
+            self.printerr('  %s' % line)
+            self.printerr(' %s^' % (' ' * e.charnum))
+            self.reset_statement()
+            return
+
+        while statements and not statements[-1]:
+            statements = statements[:-1]
+        if not statements:
+            self.reset_statement()
+            return
+        if in_batch or statements[-1][-1][0] != 'endtoken':
+            self.continue_statement()
+            return
+        try:
+            for st in statements:
+                try:
+                    self.handle_statement(st)
+                except Exception, e:
+                    if self.debug:
+                        import traceback
+                        traceback.print_exc()
+                    else:
+                        self.printerr(e)
+        finally:
+            self.reset_statement()
+
+    def handle_statement(self, tokens):
+        cmdword = tokens[0][1]
+        if cmdword != 'EOF':
+            # and why yes, it /is/ brain-molestingly stupid that cmd uses
+            # the string "EOF" as a sentinel, so that there's no clear way
+            # to tell the difference between someone typing "EOF" and a
+            # real EOF.
+            cmdword = cmdword.lower()
+        if cmdword == '?':
+            cmdword = 'help'
+        custom_handler = getattr(self, 'do_' + cmdword, None)
+        if custom_handler:
+            parsed = cqlhandling.cql_whole_parse_tokens(tokens, startsymbol='cqlshCommand')
+            if parsed and not parsed.remainder:
+                # successful complete parse
+                return custom_handler(parsed)
+            else:
+                return self.handle_parse_error(cmdword, tokens, parsed)
+        return self.perform_statement_as_tokens(tokens)
+
+    def handle_parse_error(self, cmdword, tokens, parsed):
+        if cmdword == 'select':
+            # hey, maybe they know about some new syntax we don't. type
+            # assumptions won't work, but maybe the query will.
+            return self.perform_statement_as_tokens(tokens)
+        if parsed:
+            self.printerr('Improper %s command (problem at %r).' % (cmdword, parsed.remainder[0]))
+        else:
+            self.printerr('Improper %s command.' % cmdword)
+
+    def do_use(self, parsed):
+        """
+        USE <keyspacename>;
+
+        Tells cqlsh and the connected Cassandra instance that you will be
+        working in the given keyspace. All subsequent operations on column
+        families or indexes will be in the context of this keyspace, unless
+        otherwise specified, until another USE command is issued or the
+        connection terminates.
+
+        As always, when a keyspace name does not work as a normal identifier or
+        number, it can be enclosed in quotes and expressed as a string literal.
+        """
+        ksname = parsed.get_binding('ksname')
+        if self.perform_statement_as_tokens(parsed.matched):
+            self.current_keyspace = cql_dequote(ksname)
+
+    def do_select(self, parsed):
+        """
+        SELECT [FIRST n] [REVERSED] <selectExpr>
+          FROM [<keyspace>.]<columnFamily>
+            [USING CONSISTENCY <consistencylevel>]
+            [WHERE <clause>]
+            [LIMIT m];
+
+        SELECT is used to read one or more records from a Cassandra column
+        family. It returns a result-set of rows, where each row consists of a
+        key and a collection of columns corresponding to the query.
+
+        For more information, see one of the following:
+
+          HELP SELECT_EXPR
+          HELP SELECT_COLUMNFAMILY
+          HELP SELECT_WHERE
+          HELP SELECT_LIMIT
+          HELP CONSISTENCYLEVEL
+        """
+        cfname = cql_dequote(parsed.get_binding('selectsource'))
+        decoder = self.determine_decoder_for(cfname)
+        self.perform_statement_as_tokens(parsed.matched, decoder=decoder)
+
+    def perform_statement_as_tokens(self, tokens, decoder=None):
+        return self.perform_statement(cqlhandling.cql_detokenize(tokens), decoder=decoder)
+
+    def perform_statement(self, statement, decoder=None):
+        if not statement:
+            return False
+        trynum = 1
+        while True:
+            try:
+                self.cursor.execute(statement, decoder=decoder)
+                break
+            except cql.IntegrityError, err:
+                self.printerr("Attempt #%d: %s" % (trynum, str(err)))
+                trynum += 1
+                if trynum > self.num_retries:
+                    return False
+                time.sleep(1*trynum)
+            except CQL_ERRORS, err:
+                self.printerr(str(err))
+                return False
+            except Exception, err:
+                import traceback
+                self.printerr(traceback.format_exc())
+                return False
+
+        if self.cursor.description is _COUNT_DESCRIPTION:
+            self.print_count_result()
+        elif self.cursor.description is not _VOID_DESCRIPTION:
+            self.print_result()
+        return True
+
+    def determine_decoder_for(self, cfname):
+        schema = self.schema_overrides.get((self.current_keyspace, cfname), None)
+        if schema:
+            def use_my_schema_decoder(real_schema):
+                return cql.decoders.SchemaDecoder(schema.join(real_schema))
+            return use_my_schema_decoder
+
+    def print_count_result(self):
+        if not self.cursor.result:
+            return
+        self.printout('count')
+        self.printout('-----')
+        self.printout(self.cursor.result[0])
+        self.printout("")
+
+    def print_result(self):
+        # first pass: see if we have a static column set
+        last_description = None
+        for row in self.cursor:
+            if last_description is not None and self.cursor.description != last_description:
+                static = False
+                break
+            last_description = self.cursor.description
+        else:
+            static = True
+        self.cursor._reset()
+
+        if static:
+            self.print_static_result()
+        else:
+            self.print_dynamic_result()
+        self.printout("")
+
+    def print_static_result(self):
+        # first pass, get widths
+        widths = defaultdict(lambda: 0)
+        for row in self.cursor:
+            for desc, value in zip(self.cursor.description, row):
+                name = desc[0]
+                widths[name] = max(widths[name], len(str(name)), len(str(value)))
+        self.cursor._reset()
+
+        # print header
+        for desc in self.cursor.description:
+            name = desc[0]
+            width = widths[name]
+            self.printout(" ", newline=False)
+            self.printout(string.rjust(str(name), width), MAGENTA, False)
+            self.printout(" |", newline=False)
+        self.printout("")
+
+        # print row data
+        for row in self.cursor:
+            for desc, value in zip(self.cursor.description, row):
+                name = desc[0]
+                width = widths[desc[0]]
+                self.printout(" ", newline=False)
+                self.printout(string.rjust(str(value), width), YELLOW, False)
+                self.printout(" |", newline=False)
+            self.printout("")
+
+    def print_dynamic_result(self):
+        for row in self.cursor:
+            self.printout(" ", newline=False)
+            for desc, value in zip(self.cursor.description, row):
+                name = desc[0]
+                self.printout(str(name), MAGENTA, False)
+                self.printout(",", newline=False)
+                self.printout(str(value), YELLOW, False)
+                self.printout(" | ", newline=False)
+            self.printout("")
+
+    def emptyline(self):
+        pass
+
+    def parseline(self, line):
+        # this shouldn't be needed
+        raise NotImplementedError
+
+    def complete(self, text, state):
+        if readline is None:
+            return
+        if state == 0:
+            try:
+                self.completion_matches = self.find_completions(text)
+            except Exception:
+                if debug_completion:
+                    import traceback
+                    traceback.print_exc()
+                else:
+                    raise
+        try:
+            return self.completion_matches[state]
+        except IndexError:
+            return None
+
+    def find_completions(self, text):
+        curline = readline.get_line_buffer()
+        prevlines = self.statement.getvalue()
+        wholestmt = prevlines + curline
+        begidx = readline.get_begidx() + len(prevlines)
+        endidx = readline.get_endidx() + len(prevlines)
+        stuff_to_complete = wholestmt[:begidx]
+        return cqlhandling.cql_complete(stuff_to_complete, text, cassandra_conn=self,
+                                        debug=debug_completion, startsymbol='cqlshCommand')
+
+    def set_prompt(self, prompt):
+        if sys.stdin.isatty():
+            self.prompt = prompt
+
+    def print_recreate_keyspace(self, ksdef):
+        stratclass = trim_if_present(ksdef.strategy_class, 'org.apache.cassandra.locator.')
+        ksname = maybe_cql_escape(ksdef.name)
+        self.printout("CREATE KEYSPACE %s WITH strategy_class = %s"
+                         % (ksname, cql_escape(stratclass)),
+                      newline=False)
+        for opname, opval in ksdef.strategy_options.iteritems():
+            self.printout("\n  AND strategy_options:%s = %s" % (opname, cql_escape(opval)),
+                          newline=False)
+        self.printout(';')
+
+        if ksdef.cf_defs:
+            self.printout('\nUSE %s;' % ksname)
+            for cf in ksdef.cf_defs:
+                self.printout('')
+                self.print_recreate_columnfamily(cf)
+
+    def print_recreate_columnfamily(self, cfdef):
+        cfname = maybe_cql_escape(cfdef.name)
+        self.printout("CREATE COLUMNFAMILY %s (" % cfname)
+        alias = cfdef.key_alias if cfdef.key_alias else 'KEY'
+        keytype = cql_typename(cfdef.key_validation_class)
+        self.printout("  %s %s PRIMARY KEY" % (alias, keytype), newline=False)
+        indexed_columns = []
+        for col in cfdef.column_metadata:
+            colname = maybe_cql_escape(col.name)
+            self.printout(",\n  %s %s" % (colname, cql_typename(col.validation_class)),
+                          newline=False)
+            if col.index_name is not None:
+                indexed_columns.append(col)
+        notable_columns = []
+        for (option, thriftname) in cqlhandling.columnfamily_options:
+            optval = getattr(cfdef, thriftname or option)
+            if option in ('comparator', 'default_validation'):
+                optval = cql_typename(optval)
+            elif option == 'row_cache_provider':
+                optval = cql_escape(trim_if_present(optval, 'org.apache.cassandra.cache.'))
+            else:
+                optval = cql_escape(optval)
+            notable_columns.append((option, optval))
+        self.printout('\n)', newline=False)
+        if notable_columns:
+            joiner = 'WITH'
+            for optname, optval in notable_columns:
+                self.printout(" %s\n  %s=%s" % (joiner, optname, optval), newline=False)
+                joiner = 'AND'
+        self.printout(";")
+
+        for col in indexed_columns:
+            self.printout('')
+            # guess CQL can't represent index_type or index_options
+            self.printout('CREATE INDEX %s ON %s (%s);'
+                          % (col.index_name, cfname, maybe_cql_escape(col.name)))
+
+    def describe_keyspace(self, ksname):
+        self.printout('')
+        self.print_recreate_keyspace(self.get_keyspace(ksname))
+        self.printout('')
+
+    def describe_columnfamily(self, cfname):
+        self.printout('')
+        self.print_recreate_columnfamily(self.get_columnfamily(cfname))
+        self.printout('')
+
+    def describe_cluster(self):
+        self.printout('Cluster: %s' % self.get_cluster_name())
+        p = trim_if_present(self.get_partitioner(), 'org.apache.cassandra.dht.')
+        self.printout('Partitioner: %s' % p)
+        snitch = trim_if_present(self.get_snitch(), 'org.apache.cassandra.locator.')
+        self.printout('Snitch: %s' % snitch)
+        self.printout('')
+        if self.current_keyspace is not None and self.current_keyspace != 'system':
+            self.printout("Range ownership:")
+            ring = self.get_ring()
+            for entry in ring:
+                self.printout(' %39s  [%s]' % (entry.start_token, ', '.join(entry.endpoints)))
+            self.printout('')
+
+    def describe_schema(self):
+        self.printout('')
+        for k in self.get_keyspaces():
+            self.print_recreate_keyspace(k)
+            self.printout('')
+
+    def do_describe(self, parsed):
+        """
+        DESCRIBE [cqlsh only]
+
+          Outputs information about the connected Cassandra cluster, or about
+          the data stored on it. Use in one of the following ways:
+
+        DESCRIBE KEYSPACE <keyspacename>
+
+          Output CQL commands that could be used to recreate the given
+          keyspace, and the columnfamilies in it. In some cases, as the CQL
+          interface matures, there will be some metadata about a keyspace that
+          is not representable with CQL. That metadata will not be shown.
+
+          The '<keyspacename>' argument may be omitted when using a non-system
+          keyspace; in that case, the current keyspace will be described.
+
+        DESCRIBE COLUMNFAMILY <columnfamilyname>
+
+          Output CQL commands that could be used to recreate the given
+          columnfamily. In some cases, as above, there may be columnfamily
+          metadata which is not representable and which will not be shown.
+
+        DESCRIBE CLUSTER
+
+          Output information about the connected Cassandra cluster, such as the
+          cluster name, and the partitioner and snitch in use. When you are
+          connected to a non-system keyspace, also shows endpoint-range
+          ownership information for the Cassandra ring.
+
+        DESCRIBE SCHEMA
+
+          Output CQL commands that could be used to recreate the entire schema.
+          Works as though "DESCRIBE KEYSPACE k" was invoked for each keyspace
+          k.
+        """
+
+        what = parsed.matched[1][1].lower()
+        if what == 'keyspace':
+            ksname = cql_dequote(parsed.get_binding('ksname', ''))
+            if not ksname:
+                ksname = self.current_keyspace
+                if ksname is None:
+                    self.printerr('Not in any keyspace.')
+                    return
+            self.describe_keyspace(ksname)
+        elif what == 'columnfamily':
+            cfname = cql_dequote(parsed.get_binding('cfname'))
+            self.describe_columnfamily(cfname)
+        elif what == 'cluster':
+            self.describe_cluster()
+        elif what == 'schema':
+            self.describe_schema()
+
+    def do_show(self, parsed):
+        """
+        SHOW [cqlsh only]
+
+          Displays information about the current cqlsh session. Can be called in
+          the following ways:
+
+        SHOW VERSION
+
+          Shows the version and build of the connected Cassandra instance, as
+          well as the versions of the CQL spec and the Thrift protocol that
+          the connected Cassandra instance understands.
+
+        SHOW HOST
+
+          Shows where cqlsh is currently connected.
+
+        SHOW ASSUMPTIONS
+
+          Outputs the current list of type assumptions as specified by the
+          user. See the help for the ASSUME command for more information.
+        """
+
+        showwhat = parsed.get_binding('what').lower()
+        if showwhat == 'version':
+            self.show_version()
+        elif showwhat == 'host':
+            self.show_host()
+        elif showwhat == 'assumptions':
+            self.show_assumptions()
+        else:
+            self.printerr('Wait, how do I show %r?' % (showwhat,))
+
+    def do_assume(self, parsed):
+        """
+        ASSUME [cqlsh only]
+
+          Instruct cqlsh to consider certain column names or values to be of a
+          specified type, even if that type information is not specified in
+          the columnfamily's metadata. Data will be deserialized according to
+          the given type, and displayed appropriately when retrieved.
+
+          Use thus:
+
+        ASSUME [<keyspace>.]<columnfamily> NAMES ARE <type>;
+
+          Treat all column names in the given columnfamily as being of the
+          given type.
+
+        ASSUME [<keyspace>.]<columnfamily> VALUES ARE <type>;
+
+          Treat all column values in the given columnfamily as being of the
+          given type, unless there is more information about the specific
+          column being deserialized. That is, a column-specific ASSUME will
+          take precedence here, as will column-specific metadata in the
+          columnfamily's definition.
+
+        ASSUME [<keyspace>.]<columnfamily>(<colname>) VALUES ARE <type>;
+
+          Treat all values in the given column in the given columnfamily as
+          being of the specified type. This overrides any other information
+          about the type of a value.
+
+        Assign multiple overrides at once for the same columnfamily by
+        separating with commas:
+
+          ASSUME ks.cf NAMES ARE uuid, VALUES ARE int, (col) VALUES ARE ascii
+
+        See HELP TYPES for information on the supported data storage types.
+        """
+
+        params = {}
+        for paramname in ('ks', 'cf', 'colname', 'names', 'values', 'colvalues'):
+            val = parsed.get_binding(paramname, None)
+            if val is not None:
+                val = cql_dequote(val)
+            params[paramname] = val
+
+        if params['ks'] is None:
+            if self.current_keyspace is None:
+                self.printerr('Error: not in any keyspace.')
+                return
+            params['ks'] = self.current_keyspace
+
+        for overridetype in ('names', 'values', 'colvalues'):
+            cqltype = params[overridetype]
+            if cqltype is None:
+                continue
+            try:
+                validator_class = cqlhandling.find_validator_class(cqltype)
+            except KeyError:
+                self.printerr('Error: validator type %s not found.' % cqltype)
+            self.add_assumption(params['ks'], params['cf'], params['colname'],
+                                overridetype, validator_class)
+
+    def do_EOF(self, parsed):
+        """
+        EOF [cqlsh only]
+
+        An end-of-file condition on the input stream causes cqlsh to exit.
+
+        The command 'EOF' also exits cqlsh, but this is only because of an
+        annoying feature of Python's cmd.Cmd, and it is not expected to
+        stay this way. See also 'EXIT', which will continue to work.
+        """
+
+        if sys.stdin.isatty(): print
+        self.do_exit(None)
+
+    def do_exit(self, parsed):
+        """
+        EXIT/QUIT [cqlsh only]
+
+        Exits cqlsh.
+        """
+
+        sys.exit()
+    do_quit = do_exit
+
+    def get_names(self):
+        names = cmd.Cmd.get_names(self)
+        for hide_from_help in ('do_EOF', 'do_quit'):
+            names.remove(hide_from_help)
+        return names
+
+    def columnize(self, slist, *a, **kw):
+        return cmd.Cmd.columnize(self, [u.upper() for u in slist], *a, **kw)
+
+    def do_help(self, parsed):
+        """
+        HELP [cqlsh only]
+
+        Gives information about cqlsh commands. To see available topics,
+        enter "HELP" without any arguments. To see help on a topic,
+        use "HELP <topic>".
+        """
+        topics = parsed.get_binding('topic', ())
+        if not topics:
+            return cmd.Cmd.do_help(self, '')
+        for t in topics:
+            cmd.Cmd.do_help(self, cql_dequote(t).lower())
+
+    def help_types(self):
+        self.printout("CQL types recognized by this version of cqlsh:\n")
+        for t in cqlhandling.cql_types:
+            self.printout('  ' + t)
+        self.printout('')
+
+    def help_select_expr(self):
+        print """
+        SELECT: Specifying Columns
+
+          SELECT [FIRST n] [REVERSED] name1, name2, name3 FROM ...
+          SELECT [FIRST n] [REVERSED] name1..nameN FROM ...
+          SELECT COUNT(*) FROM ...
+
+        The SELECT expression determines which columns will appear in the
+        results and takes the form of either a comma separated list of names,
+        or a range. The range notation consists of a start and end column name
+        separated by two periods (..). The set of columns returned for a
+        range is start and end inclusive.
+
+        The FIRST option accepts an integer argument and can be used to apply a
+        limit to the number of columns returned per row.  When this limit is
+        left unset, it defaults to 10,000 columns.
+
+        The REVERSED option causes the sort order of the results to be
+        reversed.
+
+        It is worth noting that unlike the projection in a SQL SELECT, there is
+        no guarantee that the results will contain all of the columns
+        specified. This is because Cassandra is schema-less and there are no
+        guarantees that a given column exists.
+
+        When the COUNT aggregate function is specified as a column to fetch, a
+        single row will be returned, with a single column named "count" whose
+        value is the number of rows from the pre-aggregation resultset.
+
+        Currently, COUNT is the only function supported by CQL.
+        """
+
+    def help_select_columnfamily(self):
+        print """
+        SELECT: Specifying Column Family
+
+          SELECT ... FROM [<keyspace>.]<columnFamily> ...
+
+        The FROM clause is used to specify the Cassandra column family
+        applicable to a SELECT query. The keyspace in which the column family
+        exists can optionally be specified along with the column family name,
+        separated by a dot (.). This will not change the current keyspace of
+        the session (see HELP USE).
+        """
+
+    def help_select_where(self):
+        print """
+        SELECT: Filtering rows
+
+          SELECT ... WHERE <key> = keyname AND name1 = value1
+          SELECT ... WHERE <key> >= startkey and <key> =< endkey AND name1 = value1
+          SELECT ... WHERE <key> IN ('<key>', '<key>', '<key>', ...)
+
+        The WHERE clause provides for filtering the rows that appear in
+        results.  The clause can filter on a key name, or range of keys, and in
+        the case of indexed columns, on column values.  Key filters are
+        specified using the KEY keyword or key alias name, a relational
+        operator (one of =, >, >=, <, and <=), and a term value.  When terms
+        appear on both sides of a relational operator it is assumed the filter
+        applies to an indexed column. With column index filters, the term on
+        the left of the operator is the name, the term on the right is the
+        value to filter _on_.
+
+        Note: The greater-than and less-than operators (> and <) result in key
+        ranges that are inclusive of the terms. There is no supported notion of
+        "strictly" greater-than or less-than; these operators are merely
+        supported as aliases to >= and <=.
+        """
+
+    def help_select_limit(self):
+        print """
+        SELECT: Limiting results
+
+          SELECT ... WHERE <clause> [LIMIT n] ...
+
+        Limiting the number of rows returned can be achieved by adding the
+        LIMIT option to a SELECT expression. LIMIT defaults to 10,000 when left
+        unset.
+        """
+
+    def help_consistencylevel(self):
+        print """
+        Consistency Level Specification
+
+          ... USING CONSISTENCY <consistencylevel> ...
+
+        Consistency level specifications are made up of keyword USING,
+        followed by a consistency level identifier. Valid consistency level
+        identifiers are as follows:
+
+         * ANY
+         * ONE (default)
+         * QUORUM
+         * ALL
+         * LOCAL_QUORUM
+         * EACH_QUORUM
+
+        For more information on how consistency levels work, consult your
+        Cassandra documentation.
+        """
+
+    def help_update(self):
+        print """
+        UPDATE <columnFamily> [USING CONSISTENCY <consistencylevel>
+                                [AND TIMESTAMP <timestamp>]
+                                [AND TTL <timeToLive>]]
+               SET name1 = value1, name2 = value2 WHERE <KEY> = keyname;
+
+        An UPDATE is used to write one or more columns to a record in a
+        Cassandra column family. No results are returned. Key can be given
+        using the KEY keyword or by an alias set per columnfamily.
+
+        Statements begin with the UPDATE keyword followed by a Cassandra
+        column family name.
+
+        For more information, see one of the following:
+
+          HELP UPDATE_USING
+          HELP UPDATE_SET
+          HELP UPDATE_COUNTERS
+          HELP UPDATE_WHERE
+          HELP CONSISTENCYLEVEL
+        """
+
+    def help_update_using(self):
+        print """
+        UPDATE: the USING clause
+
+          UPDATE ... USING TIMESTAMP <timestamp>;
+          UPDATE ... USING TTL <timeToLive>;
+          UPDATE ... USING CONSISTENCY <consistencylevel>;
+
+        The USING clause allows setting of certain query and data parameters.
+        If multiple parameters need to be set, these may be joined using AND.
+        Example:
+
+          UPDATE ... USING TTL 43200 AND CONSISTENCY LOCAL_QUORUM;
+
+        <timestamp> defines the optional timestamp for the new column value(s).
+        It must be an integer.
+
+        <timeToLive> defines the optional time to live (TTL) for the new column
+        value(s). It must be an integer.
+        """
+
+    def help_update_set(self):
+        print """
+        UPDATE: Specifying Columns and Row
+
+          UPDATE ... SET name1 = value1, name2 = value2
+                   WHERE <key> = keyname;
+          UPDATE ... SET name1 = value1, name2 = value2
+                   WHERE <key> IN ('<key1>', '<key2>', ...)
+
+        Rows are created or updated by supplying column names and values in
+        term assignment format.  Multiple columns can be set by separating the
+        name/value pairs using commas.
+        """
+
+    def help_update_counters(self):
+        print """
+        UPDATE: Updating Counter Columns
+
+          UPDATE ... SET name1 = name1 + <value> ...
+          UPDATE ... SET name1 = name1 - <value> ...
+
+        Counter columns can be incremented or decremented by an arbitrary
+        numeric value though the assignment of an expression that adds or
+        substracts the value.
+        """
+
+    def help_update_where(self):
+        print """
+        UPDATE: Selecting rows to update
+
+          UPDATE ... WHERE <keyname> = <keyval>;
+          UPDATE ... WHERE <keyname> IN (<keyval1>, <keyval2>, ...);
+
+        Each update statement requires a precise set of keys to be specified
+        using a WHERE clause.
+
+        <keyname> can be the keyword KEY or the key alias for the column
+        family.
+        """
+
+    def help_delete(self):
+        print """
+        DELETE [<col1> [, <col2>, ...] FROM <columnFamily>
+               [USING CONSISTENCY <consistencylevel>
+                   [AND TIMESTAMP <timestamp>]]
+            WHERE <keyname> = <keyvalue>;
+
+        A DELETE is used to perform the removal of one or more columns from one
+        or more rows. Each DELETE statement requires a precise set of row keys
+        to be specified using a WHERE clause and the KEY keyword or key alias.
+
+        For more information, see one of the following:
+
+          HELP DELETE_USING
+          HELP DELETE_COLUMNS
+          HELP DELETE_WHERE
+          HELP CONSISTENCYLEVEL
+        """
+
+    def help_delete_using(self):
+        """
+        DELETE: the USING clause
+
+          DELETE ... USING CONSISTENCY <consistencylevel>;
+          DELETE ... USING TIMESTAMP <timestamp>;
+
+        The USING clause allows setting of certain query and data parameters.
+        If multiple parameters need to be set, these may be joined using AND.
+        Example:
+
+          DELETE ... CONSISTENCY LOCAL_QUORUM AND TIMESTAMP 1318452291034;
+
+        <timestamp> defines the optional timestamp for the new tombstone
+        record. It must be an integer.
+        """
+
+    def help_delete_columns(self):
+        """
+        DELETE: specifying columns
+
+          DELETE col1, 'col2 name', 3 FROM ...
+
+        Following the DELETE keyword is an optional comma-delimited list of
+        column name terms. When no column names are given, the remove applies
+        to the entire row(s) matched by the WHERE clause.
+        """
+
+    def help_delete_where(self):
+        print """
+        DELETE: specifying rows
+
+          DELETE ... WHERE KEY = 'some_key_value';
+          DELETE ... WHERE keyalias IN (key1, key2);
+
+        The WHERE clause is used to determine to which row(s) a DELETE
+        applies. The first form allows the specification of a single keyname
+        using the KEY keyword (or the key alias) and the = operator. The
+        second form allows a list of keyname terms to be specified using the
+        IN operator and a parenthesized list of comma-delimited keyname
+        terms.
+        """
+
+    def help_create(self):
+        print """
+        There are different variants of CREATE. For more information, see
+        one of the following:
+
+          HELP CREATE_KEYSPACE;
+          HELP CREATE_COLUMNFAMILY;
+          HELP CREATE_INDEX;
+        """
+
+    def help_create_keyspace(self):
+        print """
+        CREATE KEYSPACE <ksname> WITH strategy_class = '<strategy>'
+                                 [AND strategy_options:<option> = <val>];
+
+        The CREATE KEYSPACE statement creates a new top-level namespace (aka
+        "keyspace"). Valid names are any string constructed of alphanumeric
+        characters and underscores. Names which do not work as valid
+        identifiers or integers should be quoted as string literals. Properties
+        such as replication strategy and count are specified during creation
+        using the following accepted keyword arguments:
+
+          strategy_class [required]: The name of the replication strategy class
+          which should be used for the new keyspace. Some often-used classes
+          are SimpleStrategy and NetworkTopologyStrategy.
+
+          strategy_options: Most strategies require additional arguments which
+          can be supplied by appending the option name to "strategy_options",
+          separated by a colon (:). For example, a strategy option of "DC1"
+          with a value of "1" would be specified as "strategy_options:DC1 = 1".
+          The replication factor option for SimpleStrategy could be
+          "strategy_options:replication_factor=3".
+        """
+
+    def help_create_columnfamily(self):
+        print """
+        CREATE COLUMNFAMILY <cfname> ( <colname> <type> PRIMARY KEY [,
+                                       <colname> <type> [, ...]] )
+               [WITH <optionname> = <val> [AND <optionname> = <val> [...]]];
+
+        CREATE COLUMNFAMILY statements create a new column family under the
+        current keyspace. Valid column family names are strings of alphanumeric
+        characters and underscores, which begin with a letter.
+
+        Each columnfamily requires at least one column definition and type,
+        which will correspond to the columnfamily key and key validator. It's
+        important to note that the key type you use must be compatible with the
+        partitioner in use. For example, OrderPreservingPartitioner and
+        CollatingOrderPreservingPartitioner both require UTF-8 keys. If you
+        use an identifier for the primary key name, instead of the KEY
+        keyword, a key alias will be set automatically.
+
+        For more information, see one of the following:
+
+          HELP CREATE_COLUMNFAMILY_TYPES;
+          HELP CREATE_COLUMNFAMILY_OPTIONS;
+        """
+
+    def help_create_columnfamily_types(self):
+        print """
+        CREATE COLUMNFAMILY: Specifying column types
+
+          CREATE ... (KEY <type> PRIMARY KEY,
+                      othercol <type>) ...
+
+        It is possible to assign columns a type during column family creation.
+        Columns configured with a type are validated accordingly when a write
+        occurs, and intelligent CQL drivers and interfaces will be able to
+        decode the column values correctly when receiving them. Column types
+        are specified as a parenthesized, comma-separated list of column term
+        and type pairs. See HELP TYPES; for the list of recognized types.
+        """
+
+    def help_create_columnfamily_options(self):
+        print """
+        CREATE COLUMNFAMILY: Specifying columnfamily options
+
+          CREATE COLUMNFAMILY blah (...)
+             WITH optionname = val AND otheroption = val2;
+
+        A number of optional keyword arguments can be supplied to control the
+        configuration of a new column family, such as the size of the
+        associated row and key caches. Consult your CQL reference for the
+        complete list of options and possible values.
+        """
+
+    def help_create_index(self):
+        print """
+        CREATE INDEX [<indexname>] ON <cfname> ( <colname> );
+
+        A CREATE INDEX statement is used to create a new, automatic secondary
+        index on the given column family, for the named column. A name for the
+        index itself can be specified before the ON keyword, if desired. A
+        single column name must be specified inside the parentheses. It is not
+        necessary for the column to exist on any current rows (Cassandra is
+        schemaless), but the column must already have a type (specified during
+        the CREATE COLUMNFAMILY, or added afterwards with ALTER COLUMNFAMILY.
+        """
+
+    def help_drop(self):
+        print """
+        There are different variants of DROP. For more information, see
+        one of the following:
+
+          HELP DROP_KEYSPACE;
+          HELP DROP_COLUMNFAMILY;
+          HELP DROP_INDEX;
+        """
+
+    def help_drop_keyspace(self):
+        print """
+        DROP KEYSPACE <keyspacename>;
+
+        A DROP KEYSPACE statement results in the immediate, irreversible
+        removal of a keyspace, including all column families in it, and all
+        data contained in those column families.
+        """
+
+    def help_drop_columnfamily(self):
+        print """
+        DROP COLUMNFAMILY <columnfamilyname>;
+
+        A DROP COLUMNFAMILY statement results in the immediate, irreversible
+        removal of a column family, including all data contained in it.
+        """
+
+    def help_drop_index(self):
+        print """
+        DROP INDEX <indexname>;
+
+        A DROP INDEX statement is used to drop an existing secondary index.
+        """
+
+    def help_truncate(self):
+        print """
+        TRUNCATE <columnfamilyname>;
+
+        TRUNCATE accepts a single argument for the column family name, and
+        permanently removes all data from said column family.
+        """
+
+    def help_begin(self):
+        print """
+        BEGIN BATCH [USING CONSISTENCY <level>
+                       [AND TIMESTAMP <timestamp>]]
+          <insert or update or delete statement> ;
+          [ <another insert or update or delete statement ;
+            [...]]
+        APPLY BATCH;
+
+        BATCH supports setting a client-supplied optional global timestamp
+        which will be used for each of the operations included in the batch.
+
+        A single consistency level is used for the entire batch. It appears
+        after the BEGIN BATCH statement, and uses the standard "consistency
+        level specification" (see HELP CONSISTENCYLEVEL). Batched statements
+        default to CONSISTENCY.ONE when left unspecified.
+
+        Only data modification statements (specifically, UPDATE, INSERT,
+        and DELETE) are allowed in a BATCH statement. BATCH is _not_ an
+        analogue for SQL transactions.
+
+        _NOTE: While there are no isolation guarantees, UPDATE queries are
+        atomic within a given record._
+        """
+    help_apply = help_begin
+
+    def help_alter(self):
+        print """
+        ALTER COLUMNFAMILY <cfname> ALTER <columnname> TYPE <type>;
+        ALTER COLUMNFAMILY <cfname> ADD <columnname> <type>;
+        ALTER COLUMNFAMILY <cfname> DROP <columnname>;
+
+        An ALTER statement is used to manipulate column family column
+        metadata. It allows you to add new columns, drop existing columns, or
+        change the data storage type of existing columns. No results are
+        returned.
+
+        See one of the following for more information:
+
+          HELP ALTER_ALTER;
+          HELP ALTER_ADD;
+          HELP ALTER_DROP;
+        """
+
+    def help_alter_alter(self):
+        print """
+        ALTER COLUMNFAMILY: altering existing typed columns
+
+          ALTER COLUMNFAMILY addamsFamily ALTER lastKnownLocation TYPE uuid;
+
+        ALTER COLUMNFAMILY ... ALTER changes the expected storage type for a
+        column. The column must already have a type in the column family
+        metadata. The column may or may not already exist in current rows-- but
+        be aware that no validation of existing data is done. The bytes stored
+        in values for that column will remain unchanged, and if existing data
+        is not deserializable according to the new type, this may cause your
+        CQL driver or interface to report errors.
+        """
+
+    def help_alter_add(self):
+        print """
+        ALTER COLUMNFAMILY: adding a typed column
+
+          ALTER COLUMNFAMILY addamsFamily ADD gravesite varchar;
+
+        The ALTER COLUMNFAMILY ... ADD variant adds a typed column to a column
+        family. The column must not already have a type in the column family
+        metadata. See the warnings on HELP ALTER_ALTER regarding the lack of
+        validation of existing data; they apply here as well.
+        """
+
+    def help_alter_drop(self):
+        print """
+        ALTER COLUMNFAMILY: dropping a typed column
+
+          ALTER COLUMNFAMILY addamsFamily DROP gender;
+
+        An ALTER COLUMNFAMILY ... DROP statement removes the type of a column
+        from the column family metadata. Note that this does _not_ remove the
+        column from current rows; it just removes the metadata saying that the
+        bytes stored under that column are expected to be deserializable
+        according to a certain type.
+        """
+
+    def printout(self, text, color=None, newline=True, out=sys.stdout):
+        if not color or not self.color:
+            out.write(text)
+        else:
+            out.write(color % text)
+
+        if newline:
+            out.write("\n")
+
+    def printerr(self, text, color=RED, newline=True):
+        self.printout(text, color, newline, sys.stderr)
+
+    def add_assumption(self, ksname, cfname, colname, valtype, valclass):
+        try:
+            v_info = self.schema_overrides[(ksname, cfname)]
+        except KeyError:
+            v_info = self.schema_overrides[(ksname, cfname)] = FakeCqlMetadata()
+        if valtype == 'names':
+            v_info.default_name_type = valclass
+        elif valtype == 'values':
+            v_info.default_value_type = valclass
+        elif valtype == 'colvalues':
+            v_info.value_types[colname] = valclass
+
+
+class FakeCqlMetadata:
+    def __init__(self):
+        self.name_types = {}
+        self.value_types = {}
+        self.default_name_type = None
+        self.default_value_type = None
+
+    def join(self, realschema):
+        f = self.__class__()
+        f.default_name_type = self.default_name_type or realschema.default_name_type
+        f.default_value_types = self.default_value_type or realschema.default_value_type
+        f.name_types = realschema.name_types.copy()
+        f.name_types.update(self.name_types)
+        f.value_types = realschema.value_types.copy()
+        f.value_types.update(self.value_types)
+        return f
+
+
+def option_with_default(cparser_getter, section, option, default=None):
+    try:
+        return cparser_getter(section, option)
+    except ConfigParser.Error:
+        return default
+
+def read_options(cmdlineargs, environment):
+    configs = ConfigParser.SafeConfigParser()
+    configs.read(CONFIG_FILE)
+
+    optvalues = optparse.Values()
+    optvalues.username = option_with_default(configs.get, 'authentication', 'username')
+    optvalues.password = option_with_default(configs.get, 'authentication', 'password')
+    optvalues.completekey = option_with_default(configs.get, 'ui', 'completekey', 'tab')
+    optvalues.color = option_with_default(configs.getboolean, 'ui', 'color')
+    if optvalues.color is None:
+        # default yes if tty
+        optvalues.color = bool(os.isatty(0))
+    optvalues.debug = False
+
+    (options, arguments) = parser.parse_args(cmdlineargs, values=optvalues)
+
+    hostname = option_with_default(configs.get, 'connection', 'hostname', DEFAULT_HOST)
+    port = option_with_default(configs.get, 'connection', 'port', DEFAULT_PORT)
+
+    hostname = environment.get('CQLSH_HOST', hostname)
+    port = environment.get('CQLSH_PORT', port)
+
+    if len(arguments) > 0:
+        hostname = arguments[0]
+    if len(arguments) > 1:
+        port = arguments[1]
+
+    try:
+        port = int(port)
+    except ValueError:
+        parser.error('%r is not a valid port number.' % port)
+
+    return options, hostname, port
+
+def main(options, hostname, port):
+    if os.path.exists(HISTORY) and readline is not None:
+        readline.read_history_file(HISTORY)
+        delims = readline.get_completer_delims()
+        delims.replace("'", "")
+        delims += '.'
+        readline.set_completer_delims(delims)
+
+    try:
+        shell = Shell(hostname,
+                      port,
+                      color=options.color,
+                      username=options.username,
+                      password=options.password,
+                      completekey=options.completekey)
+    except KeyboardInterrupt:
+        sys.exit('Connection aborted.')
+    except CQL_ERRORS, e:
+        sys.exit('Connection error: %s' % (e,))
+    if options.debug:
+        shell.debug = True
+
+    while True:
+        try:
+            shell.cmdloop()
+        except SystemExit:
+            if readline is not None:
+                readline.write_history_file(HISTORY)
+            break
+        except cql.Error, cqlerr:
+            shell.printerr(str(cqlerr))
+        except KeyboardInterrupt:
+            shell.reset_statement()
+            print
+        except Exception, err:
+            if options.debug:
+                import traceback
+                traceback.print_exc()
+            else:
+                shell.printerr("Exception: %s" % err)
+
+if __name__ == '__main__':
+    main(*read_options(sys.argv[1:], os.environ))

Added: cassandra/branches/cassandra-1.0/bin/cqlshrc.sample
URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-1.0/bin/cqlshrc.sample?rev=1198994&view=auto
==============================================================================
--- cassandra/branches/cassandra-1.0/bin/cqlshrc.sample (added)
+++ cassandra/branches/cassandra-1.0/bin/cqlshrc.sample Mon Nov  7 23:05:54 2011
@@ -0,0 +1,15 @@
+; Sample ~/.cqlshrc file.
+
+[authentication]
+username = fred
+password = !!bang!!$
+
+[ui]
+color = on
+completekey = tab
+
+[connection]
+hostname = 127.0.0.1
+port = 9160
+
+; vim: set ft=dosini :

Modified: cassandra/branches/cassandra-1.0/build.xml
URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-1.0/build.xml?rev=1198994&r1=1198993&r2=1198994&view=diff
==============================================================================
--- cassandra/branches/cassandra-1.0/build.xml (original)
+++ cassandra/branches/cassandra-1.0/build.xml Mon Nov  7 23:05:54 2011
@@ -853,6 +853,12 @@ url=${svn.entry.url}?pathrev=${svn.entry
           <include name="**/*.thrift" />
         </fileset>
       </copy>
+      <copy todir="${dist.dir}/pylib">
+        <fileset dir="pylib">
+          <include name="**" />
+          <exclude name="**/*.pyc" />
+        </fileset>
+      </copy>
       <copy todir="${dist.dir}/">
         <fileset dir="${basedir}">
           <include name="*.txt" />

Added: cassandra/branches/cassandra-1.0/doc/cql/CQL.css
URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-1.0/doc/cql/CQL.css?rev=1198994&view=auto
==============================================================================
--- cassandra/branches/cassandra-1.0/doc/cql/CQL.css (added)
+++ cassandra/branches/cassandra-1.0/doc/cql/CQL.css Mon Nov  7 23:05:54 2011
@@ -0,0 +1,27 @@
+/* Just a sample of some things that might be helpful in rendering CQL.html */
+
+pre.sample {
+  padding: 1ex;
+  border: 1px solid black;
+  background-color: #ffe0e0;
+}
+pre.syntax {
+  padding: 1ex;
+  border: 1px solid black;
+  background-color: #e0e0ff;
+}
+table {
+  width: 80%;
+  border-collapse: collapse;
+  border: 1px solid black;
+}
+td {
+  padding: 2px 1ex;
+  border: 1px solid black;
+}
+body {
+  background-color: white;
+}
+h2 {
+  margin-top: 3em;
+}