You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by al...@apache.org on 2013/05/21 23:48:26 UTC

[2/2] git commit: cqlsh: drop CQL2/CQL3-beta support

cqlsh: drop CQL2/CQL3-beta support

patch by Aleksey Yeschenko; reviewed by Brandon Williams for
CASSANDRA-5585


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/7f6ac19e
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/7f6ac19e
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/7f6ac19e

Branch: refs/heads/trunk
Commit: 7f6ac19efb9a9d51a3ebdb58197c8fe35476034f
Parents: 2ee9030
Author: Aleksey Yeschenko <al...@apache.org>
Authored: Wed May 22 00:47:44 2013 +0300
Committer: Aleksey Yeschenko <al...@apache.org>
Committed: Wed May 22 00:47:44 2013 +0300

----------------------------------------------------------------------
 CHANGES.txt                                  |    1 +
 bin/cqlsh                                    |  272 ++--------
 pylib/cqlshlib/cql3handling.py               |  362 +++----------
 pylib/cqlshlib/cqlhandling.py                |  612 +--------------------
 pylib/cqlshlib/helptopics.py                 |  251 +---------
 pylib/cqlshlib/test/cassconnect.py           |   28 +-
 pylib/cqlshlib/test/table_arrangements.cql   |  114 ----
 pylib/cqlshlib/test/test_cqlsh_completion.py |   96 +---
 pylib/cqlshlib/test/test_cqlsh_output.py     |  235 +-------
 pylib/cqlshlib/test/test_keyspace_init.cql   |  198 +++++++
 pylib/cqlshlib/test/test_keyspace_init2.cql  |  180 ------
 pylib/cqlshlib/test/test_keyspace_init3.cql  |   36 --
 12 files changed, 394 insertions(+), 1991 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/7f6ac19e/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 87cf9d4..570c867 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -48,6 +48,7 @@
  * Add alias support to SELECT statement (CASSANDRA-5075)
  * Don't create empty RowMutations in CommitLogReplayer (CASSANDRA-5541)
  * Use range tombstones when dropping cfs/columns from schema (CASSANDRA-5579)
+ * cqlsh: drop CQL2/CQL3-beta support (CASSANDRA-5585)
 
 
 1.2.6

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7f6ac19e/bin/cqlsh
----------------------------------------------------------------------
diff --git a/bin/cqlsh b/bin/cqlsh
index c2a3151..8dcc69f 100755
--- a/bin/cqlsh
+++ b/bin/cqlsh
@@ -32,7 +32,7 @@ exit 1
 from __future__ import with_statement
 
 description = "CQL Shell for Apache Cassandra"
-version = "3.0.2"
+version = "4.0.0"
 
 from StringIO import StringIO
 from itertools import groupby
@@ -121,7 +121,7 @@ CONFIG_FILE = os.path.expanduser(os.path.join('~', '.cqlshrc'))
 HISTORY = os.path.expanduser(os.path.join('~', '.cqlsh_history'))
 DEFAULT_HOST = 'localhost'
 DEFAULT_PORT = 9160
-DEFAULT_CQLVER = '3'
+DEFAULT_CQLVER = '3.1.0'
 DEFAULT_TRANSPORT_FACTORY = 'cqlshlib.tfactory.regular_transport_factory'
 
 DEFAULT_TIME_FORMAT = '%Y-%m-%d %H:%M:%S%z'
@@ -155,12 +155,7 @@ parser.add_option('--debug', action='store_true',
                   help='Show additional debugging information')
 parser.add_option('--cqlversion', default=DEFAULT_CQLVER,
                   help='Specify a particular CQL version (default: %default).'
-                       ' Examples: "2", "3.0.0-beta1"')
-parser.add_option("-2", "--cql2", action="store_const", dest='cqlversion', const='2',
-                  help="Shortcut notation for --cqlversion=2")
-parser.add_option("-3", "--cql3", action="store_const", dest='cqlversion', const='3',
-                  help="Shortcut notation for --cqlversion=3")
-
+                       ' Examples: "3.0.3", "3.1.0"')
 
 CQL_ERRORS = (cql.Error,)
 try:
@@ -469,14 +464,6 @@ class Shell(cmd.Cmd):
         self.cursor = self.conn.cursor()
         self.get_connection_versions()
 
-        # use 3.0.0-beta1 syntax if explicitly requested, or if using
-        # cassandra < 1.2. this only affects use of cql3; cql2 syntax
-        # in either case is the same.
-        if self.cassandraver_atleast(1, 2) and not self.is_cql3_beta():
-            cql3handling.use_post_3_0_0_syntax()
-        else:
-            cql3handling.use_pre_3_0_0_syntax()
-
         self.current_keyspace = keyspace
 
         self.color = color
@@ -510,9 +497,6 @@ class Shell(cmd.Cmd):
         self.cql_version = ver
         self.cql_ver_tuple = vertuple
 
-    def is_cql3_beta(self):
-        return self.cql_ver_tuple == (3, 0, 0, 'beta1')
-
     def cqlver_atleast(self, major, minor=0, patch=0):
         return self.cql_ver_tuple[:3] >= (major, minor, patch)
 
@@ -553,34 +537,16 @@ class Shell(cmd.Cmd):
         print "[cqlsh %(shver)s | Cassandra %(build)s | CQL spec %(cql)s | Thrift protocol %(thrift)s]" % vers
 
     def get_connection_versions(self):
-        try:
-            self.cursor.execute("select * from system.local where key = 'local'")
-        except cql.ProgrammingError:
-            vers = self.get_connection_versions_fallback()
-        else:
-            result = self.fetchdict()
-            vers = {
-                'build': result['release_version'],
-                'thrift': result['thrift_version'],
-                'cql': result['cql_version'],
-            }
+        self.cursor.execute("select * from system.local where key = 'local'")
+        result = self.fetchdict()
+        vers = {
+            'build': result['release_version'],
+            'thrift': result['thrift_version'],
+            'cql': result['cql_version'],
+        }
         self.connection_versions = vers
         self.cass_ver_tuple = tuple(map(int, vers['build'].split('-', 1)[0].split('.', 2)))
 
-    def get_connection_versions_fallback(self):
-        if self.cqlver_atleast(3):
-            query = 'select component, version from system."Versions"'
-        else:
-            query = 'select component, version from system.Versions'
-        try:
-            self.cursor.execute(query)
-            vers = dict(self.cursor)
-        except cql.ProgrammingError:
-            # older Cassandra; doesn't have system.Versions
-            thrift_ver = self.get_thrift_version()
-            return {'build': '0.0.0', 'cql': 'unknown', 'thrift': thrift_ver}
-        return vers
-
     def fetchdict(self):
         row = self.cursor.fetchone()
         if row is None:
@@ -615,10 +581,16 @@ class Shell(cmd.Cmd):
         raise ColumnFamilyNotFound("Unconfigured column family %r" % (cfname,))
 
     def get_columnfamily_names(self, ksname=None):
-        if self.cqlver_atleast(3) and not self.is_cql3_beta():
-            return self.get_columnfamily_names_cql3(ksname=ksname)
-        return [c.name for c in self.get_columnfamilies(ksname)]
+        if ksname is None:
+            ksname = self.current_keyspace
+        cf_q = """select columnfamily_name from system.schema_columnfamilies
+                   where keyspace_name=:ks"""
+        self.cursor.execute(cf_q,
+                            {'ks': self.cql_unprotect_name(ksname)},
+                            consistency_level='ONE')
+        return [str(row[0]) for row in self.cursor.fetchall()]
 
+    # TODO: FIXME
     def get_index_names(self, ksname=None):
         indnames = []
         for c in self.get_columnfamilies(ksname):
@@ -627,36 +599,12 @@ class Shell(cmd.Cmd):
                     indnames.append(md.index_name)
         return indnames
 
-    def filterable_column_names(self, cfdef):
-        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
-
     def get_column_names(self, ksname, cfname):
         if ksname is None:
             ksname = self.current_keyspace
-        if self.cqlver_atleast(3) and not (self.is_cql3_beta() and ksname in SYSTEM_KEYSPACES):
-            return self.get_column_names_from_layout(ksname, cfname)
-        else:
-            return self.get_column_names_from_cfdef(ksname, cfname)
-
-    def get_column_names_from_layout(self, ksname, cfname):
         layout = self.get_columnfamily_layout(ksname, cfname)
         return [col.name for col in layout.columns]
 
-    def get_column_names_from_cfdef(self, ksname, cfname):
-        cfdef = self.get_columnfamily(cfname, ksname=ksname)
-        key_alias = cfdef.key_alias
-        if key_alias is None:
-            key_alias = 'KEY'
-        return [key_alias] + sorted([cm.name for cm in cfdef.column_metadata])
-
     # ===== thrift-dependent parts =====
 
     def get_cluster_name(self):
@@ -705,33 +653,13 @@ class Shell(cmd.Cmd):
 
     # ===== cql3-dependent parts =====
 
-    def get_columnfamily_names_cql3(self, ksname=None):
-        if ksname is None:
-            ksname = self.current_keyspace
-        if self.cassandraver_atleast(1, 2):
-            cf_q = """select columnfamily_name from system.schema_columnfamilies
-                       where keyspace_name=:ks"""
-        else:
-            cf_q = """select "columnfamily" from system.schema_columnfamilies
-                       where "keyspace"=:ks"""
-        self.cursor.execute(cf_q,
-                            {'ks': self.cql_unprotect_name(ksname)},
-                            consistency_level='ONE')
-        return [str(row[0]) for row in self.cursor.fetchall()]
-
     def get_columnfamily_layout(self, ksname, cfname):
         if ksname is None:
             ksname = self.current_keyspace
-        if self.cassandraver_atleast(1, 2):
-            cf_q = """select * from system.schema_columnfamilies
-                       where keyspace_name=:ks and columnfamily_name=:cf"""
-            col_q = """select * from system.schema_columns
-                        where keyspace_name=:ks and columnfamily_name=:cf"""
-        else:
-            cf_q = """select * from system.schema_columnfamilies
-                       where "keyspace"=:ks and "columnfamily"=:cf"""
-            col_q = """select * from system.schema_columns
-                        where "keyspace"=:ks and "columnfamily"=:cf"""
+        cf_q = """select * from system.schema_columnfamilies
+                   where keyspace_name=:ks and columnfamily_name=:cf"""
+        col_q = """select * from system.schema_columns
+                    where keyspace_name=:ks and columnfamily_name=:cf"""
         self.cursor.execute(cf_q,
                             {'ks': ksname, 'cf': cfname},
                             consistency_level='ONE')
@@ -975,17 +903,6 @@ class Shell(cmd.Cmd):
         self.flush_output()
         return True
 
-    # these next two functions are not guaranteed perfect; just checks if the
-    # statement parses fully according to cqlsh's own understanding of the
-    # grammar. Changes to the language in Cassandra frequently don't get
-    # updated in cqlsh right away.
-
-    def parseable_as_cql3(self, statement):
-        return cql3handling.CqlRuleSet.lex_and_whole_match(statement) is not None
-
-    def parseable_as_cql2(self, statement):
-        return cqlhandling.CqlRuleSet.lex_and_whole_match(statement) is not None
-
     def get_nametype(self, cursor, num):
         """
         Determine the Cassandra type of a column name from the current row of
@@ -997,26 +914,11 @@ class Shell(cmd.Cmd):
 
         return cursor.name_info[num][1]
 
-    def has_static_result_set(self, cursor):
-        if self.cqlver_atleast(3):
-            return True  # all cql3 resultsets are static, don't bother scanning
-        last_description = None
-        for row in cursor:
-            if last_description is not None and cursor.description != last_description:
-                cursor._reset()
-                return False
-            last_description = cursor.description
-        cursor._reset()
-        return True
-
     def print_result(self, cursor, with_default_limit):
         self.decoding_errors = []
 
         self.writeresult("")
-        if self.has_static_result_set(cursor):
-            self.print_static_result(cursor)
-        else:
-            self.print_dynamic_result(cursor)
+        self.print_static_result(cursor)
         self.writeresult("")
 
         if self.decoding_errors:
@@ -1064,15 +966,6 @@ class Shell(cmd.Cmd):
             line = ' | '.join(col.rjust(w, color=self.color) for (col, w) in zip(row, widths))
             self.writeresult(' ' + line)
 
-    def print_dynamic_result(self, cursor):
-        for row in cursor:
-            colnames = [d[0] for d in cursor.description]
-            colnames_t = [(name, self.get_nametype(cursor, n)) for (n, name) in enumerate(colnames)]
-            colnames = [self.myformat_colname(name, nametype) for (name, nametype) in colnames_t]
-            colvals = map(self.myformat_value, row, cursor.column_types)
-            line = ' | '.join('%s,%s' % (n.coloredval, v.coloredval) for (n, v) in zip(colnames, colvals))
-            self.writeresult(' ' + line)
-
     def emptyline(self):
         pass
 
@@ -1133,20 +1026,14 @@ class Shell(cmd.Cmd):
     def print_recreate_keyspace(self, ksdef, out):
         stratclass = trim_if_present(ksdef.strategy_class, 'org.apache.cassandra.locator.')
         ksname = self.cql_protect_name(ksdef.name)
-        if self.cqlver_atleast(3) and not self.is_cql3_beta():
-            out.write("CREATE KEYSPACE %s WITH replication = {\n" % ksname)
-            out.write("  'class': %s" % self.cql_protect_value(stratclass))
-            for opname, opval in ksdef.strategy_options.iteritems():
-                out.write(",\n  %s: %s" % (self.cql_protect_value(opname),
-                                           self.cql_protect_value(opval)))
-            out.write("\n}")
-            if not ksdef.durable_writes:
-                out.write(" AND durable_writes = 'false'")
-        else:
-            out.write("CREATE KEYSPACE %s WITH strategy_class = %s"
-                       % (ksname, self.cql_protect_value(stratclass)))
-            for opname, opval in ksdef.strategy_options.iteritems():
-                out.write("\n  AND strategy_options:%s = %s" % (opname, self.cql_protect_value(opval)))
+        out.write("CREATE KEYSPACE %s WITH replication = {\n" % ksname)
+        out.write("  'class': %s" % self.cql_protect_value(stratclass))
+        for opname, opval in ksdef.strategy_options.iteritems():
+            out.write(",\n  %s: %s" % (self.cql_protect_value(opname),
+                                       self.cql_protect_value(opval)))
+        out.write("\n}")
+        if not ksdef.durable_writes:
+            out.write(" AND durable_writes = 'false'")
         out.write(';\n')
 
         cfs = self.get_columnfamily_names(ksname)
@@ -1160,78 +1047,11 @@ class Shell(cmd.Cmd):
     def print_recreate_columnfamily(self, ksname, cfname, out):
         """
         Output CQL commands which should be pasteable back into a CQL session
-        to recreate the given table. Can change based on CQL version in use;
-        CQL 3 syntax will not be output when in CQL 2 mode, and properties
-        which are deprecated with CQL 3 use (like default_validation) will not
-        be output when in CQL 3 mode.
+        to recreate the given table.
 
         Writes output to the given out stream.
         """
-
-        # no metainfo available from system.schema_* for system CFs, so we have
-        # to use cfdef-based description for those.
-        if self.cqlver_atleast(3) and not (self.is_cql3_beta() and ksname in SYSTEM_KEYSPACES):
-            try:
-                layout = self.get_columnfamily_layout(ksname, cfname)
-            except CQL_ERRORS:
-                # most likely a 1.1 beta where cql3 is supported, but not system.schema_*
-                if self.debug:
-                    print 'warning: failed to use system.schema_* tables to describe cf'
-                    import traceback
-                    traceback.print_exc()
-            else:
-                return self.print_recreate_columnfamily_from_layout(layout, out)
-
-        cfdef = self.get_columnfamily(cfname, ksname=ksname)
-        return self.print_recreate_columnfamily_from_cfdef(cfdef, out)
-
-    def print_recreate_columnfamily_from_cfdef(self, cfdef, out):
-        cfname = self.cql_protect_name(cfdef.name)
-        out.write("CREATE TABLE %s (\n" % cfname)
-        alias = self.cql_protect_name(cfdef.key_alias) if cfdef.key_alias else 'KEY'
-        keytype = cql_typename(cfdef.key_validation_class)
-        out.write("  %s %s PRIMARY KEY" % (alias, keytype))
-        indexed_columns = []
-        for col in cfdef.column_metadata:
-            colname = self.cql_protect_name(col.name)
-            out.write(",\n  %s %s" % (colname, cql_typename(col.validation_class)))
-            if col.index_name is not None:
-                indexed_columns.append(col)
-        cf_opts = []
-        for (option, thriftname) in cqlruleset.columnfamily_options:
-            optval = getattr(cfdef, thriftname or option, None)
-            if optval is None:
-                continue
-            if option in ('comparator', 'default_validation'):
-                optval = cql_typename(optval)
-            else:
-                if option == 'row_cache_provider':
-                    optval = trim_if_present(optval, 'org.apache.cassandra.cache.')
-                elif option == 'compaction_strategy_class':
-                    optval = trim_if_present(optval, 'org.apache.cassandra.db.compaction.')
-                optval = self.cql_protect_value(optval)
-            cf_opts.append((option, optval))
-        for option, thriftname, _ in cqlruleset.columnfamily_map_options:
-            optmap = getattr(cfdef, thriftname or option, {})
-            for k, v in optmap.items():
-                if option == 'compression_parameters' and k == 'sstable_compression':
-                    v = trim_if_present(v, 'org.apache.cassandra.io.compress.')
-                cf_opts.append(('%s:%s' % (option, k), self.cql_protect_value(v)))
-        out.write('\n)')
-        if cf_opts:
-            joiner = 'WITH'
-            for optname, optval in cf_opts:
-                out.write(" %s\n  %s=%s" % (joiner, optname, optval))
-                joiner = 'AND'
-        out.write(";\n")
-
-        for col in indexed_columns:
-            out.write('\n')
-            # guess CQL can't represent index_type or index_options
-            out.write('CREATE INDEX %s ON %s (%s);\n'
-                         % (col.index_name, cfname, self.cql_protect_name(col.name)))
-
-    def print_recreate_columnfamily_from_layout(self, layout, out):
+        layout = self.get_columnfamily_layout(ksname, cfname)
         cfname = self.cql_protect_name(layout.name)
         out.write("CREATE TABLE %s (\n" % cfname)
         keycol = layout.columns[0]
@@ -1327,12 +1147,8 @@ class Shell(cmd.Cmd):
             if layoutoption == 'compaction_strategy_options':
                 optmap['class'] = compaction_strategy
 
-            if self.cqlver_atleast(3) and not self.is_cql3_beta():
-                cf_opts.append((cql3option, optmap))
-            else:
-                for k, v in optmap.items():
-                    cf_opts.append(('%s:%s' % (cql3option, k.encode('ascii')),
-                                    self.cql_protect_value(v)))
+            cf_opts.append((cql3option, optmap))
+
         if cf_opts:
             for optname, optval in cf_opts:
                 if isinstance(optval, dict):
@@ -1498,10 +1314,6 @@ class Shell(cmd.Cmd):
         When entering CSV data on STDIN, you can use the sequence "\."
         on a line by itself to end the data input.
         """
-        if not self.cqlver_atleast(3):
-            self.printerr('COPY requires CQL version 3.0.0 or higher.')
-            return
-
         ks = self.cql_unprotect_name(parsed.get_binding('ksname', None))
         if ks is None:
             ks = self.current_keyspace
@@ -1823,10 +1635,6 @@ class Shell(cmd.Cmd):
 
           TRACING with no arguments shows the current tracing status.
         """
-        if not self.cqlver_atleast(3):
-            self.printerr('Tracing requires CQL version 3.0.0 or higher.')
-            return
-
         switch = parsed.get_binding('switch')
         if switch is None:
             if self.tracing_enabled:
@@ -1853,7 +1661,7 @@ class Shell(cmd.Cmd):
 
     def do_consistency(self, parsed):
         """
-        CONSISTENCY [cqlsh with CQL3 only]
+        CONSISTENCY [cqlsh only]
 
            Overrides default consistency level (default level is ONE).
 
@@ -1869,10 +1677,6 @@ class Shell(cmd.Cmd):
 
            CONSISTENCY with no arguments shows the current consistency level.
         """
-        if not self.cqlver_atleast(3):
-            self.printerr('CONSISTENCY requires CQL version 3.0.0 or higher.')
-            return
-
         level = parsed.get_binding('level')
         if level is None:
             print 'Current consistency level is %s.' % (self.cursor.consistency_level,)
@@ -2052,7 +1856,7 @@ def read_options(cmdlineargs, environment):
 
     options.cqlversion, cqlvertup = full_cql_version(options.cqlversion)
     if cqlvertup[0] < 3:
-        options.cqlmodule = cqlhandling
+        parser.error('%r is not a supported CQL version.' % options.cqlversion)
     else:
         options.cqlmodule = cql3handling
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7f6ac19e/pylib/cqlshlib/cql3handling.py
----------------------------------------------------------------------
diff --git a/pylib/cqlshlib/cql3handling.py b/pylib/cqlshlib/cql3handling.py
index 8e913dc..1bd9299 100644
--- a/pylib/cqlshlib/cql3handling.py
+++ b/pylib/cqlshlib/cql3handling.py
@@ -56,34 +56,7 @@ class Cql3ParsingRuleSet(CqlParsingRuleSet):
         'key', 'clustering', 'ttl', 'compact', 'storage', 'type', 'values', 'custom'
     ))
 
-    columnfamily_options = (
-        # (CQL option name, Thrift option name (or None if same))
-        ('comment', None),
-        ('compaction_strategy_class', 'compaction_strategy'),
-        ('comparator', 'comparator_type'),
-        ('default_validation', 'default_validation_class'),
-        ('gc_grace_seconds', None),
-        ('index_interval', None),
-        ('read_repair_chance', None),
-        ('replicate_on_write', None),
-        ('populate_io_cache_on_flush', None),
-    )
-
-    old_columnfamily_layout_options = (
-        # (CQL3 option name, schema_columnfamilies column name (or None if same))
-        ('bloom_filter_fp_chance', None),
-        ('caching', None),
-        ('comment', None),
-        ('compaction_strategy_class', None),
-        ('dclocal_read_repair_chance', 'local_read_repair_chance'),
-        ('gc_grace_seconds', None),
-        ('index_interval', None),
-        ('read_repair_chance', None),
-        ('replicate_on_write', None),
-        ('populate_io_cache_on_flush', None),
-    )
-
-    new_columnfamily_layout_options = (
+    columnfamily_layout_options = (
         ('bloom_filter_fp_chance', None),
         ('caching', None),
         ('comment', None),
@@ -95,16 +68,7 @@ class Cql3ParsingRuleSet(CqlParsingRuleSet):
         ('populate_io_cache_on_flush', None),
     )
 
-    old_columnfamily_layout_map_options = (
-        # (CQL3 option prefix, schema_columnfamilies column name (or None if same),
-        #  list of known suboptions)
-        ('compaction_strategy_options', None,
-            ('min_compaction_threshold', 'max_compaction_threshold')),
-        ('compression_parameters', None,
-            ('sstable_compression', 'chunk_length_kb', 'crc_check_chance')),
-    )
-
-    new_columnfamily_layout_map_options = (
+    columnfamily_layout_map_options = (
         # (CQL3 option name, schema_columnfamilies column name (or None if same),
         #  list of known map keys)
         ('compaction', 'compaction_strategy_options',
@@ -113,42 +77,49 @@ class Cql3ParsingRuleSet(CqlParsingRuleSet):
             ('sstable_compression', 'chunk_length_kb', 'crc_check_chance')),
     )
 
-    new_obsolete_cf_options = (
+    obsolete_cf_options = (
         'compaction_parameters',
         'compaction_strategy_class',
         'compaction_strategy_options',
         'compression_parameters',
+        'key_cache_save_period_in_seconds',
+        'key_cache_size',
         'max_compaction_threshold',
+        'memtable_flush_after_mins',
+        'memtable_operations_in_millions',
+        'memtable_throughput_in_mb',
         'min_compaction_threshold',
+        'row_cache_provider',
+        'row_cache_save_period_in_seconds',
+        'row_cache_size',
     )
 
-    @staticmethod
-    def token_dequote(tok):
-        if tok[0] == 'unclosedName':
-            # strip one quote
-            return tok[1][1:].replace('""', '"')
-        # cql2 version knows how to do everything else
-        return CqlParsingRuleSet.token_dequote(tok)
-
-    @classmethod
-    def cql3_dequote_value(cls, value):
-        return cls.cql2_dequote_value(value)
-
-    @staticmethod
-    def cql3_dequote_name(name):
-        name = name.strip()
-        if name == '':
-            return name
-        if name[0] == '"' and name[-1] == '"':
-            name = name[1:-1].replace('""', '"')
-        return name
+    consistency_levels = (
+        'ANY',
+        'ONE',
+        'TWO',
+        'THREE',
+        'QUORUM',
+        'ALL',
+        'LOCAL_QUORUM',
+        'EACH_QUORUM',
+        'SERIAL'
+    )
 
     @classmethod
-    def cql3_escape_value(cls, value):
-        return cls.cql2_escape_value(value)
+    def escape_value(cls, value):
+        if value is None:
+            return 'NULL' # this totally won't work
+        if isinstance(value, bool):
+            value = str(value).lower()
+        elif isinstance(value, float):
+            return '%f' % value
+        elif isinstance(value, int):
+            return str(value)
+        return "'%s'" % value.replace("'", "''")
 
     @staticmethod
-    def cql3_escape_name(name):
+    def escape_name(name):
         return '"%s"' % name.replace('"', '""')
 
     valid_cql3_word_re = re.compile(r'^[a-z][0-9a-z_]*$')
@@ -162,22 +133,28 @@ class Cql3ParsingRuleSet(CqlParsingRuleSet):
         return cls.valid_cql3_word_re.match(s) is not None
 
     @classmethod
-    def cql3_maybe_escape_name(cls, name):
+    def maybe_escape_name(cls, name):
         if cls.is_valid_cql3_name(name):
             return name
-        return cls.cql3_escape_name(name)
+        return cls.escape_name(name)
 
-    @classmethod
-    def dequote_any(cls, t):
-        if t[0] == '"':
-            return cls.cql3_dequote_name(t)
-        return CqlParsingRuleSet.dequote_any(t)
+    @staticmethod
+    def dequote_name(name):
+        name = name.strip()
+        if name == '':
+            return name
+        if name[0] == '"' and name[-1] == '"':
+            name = name[1:-1].replace('""', '"')
+        return name
 
-    dequote_value = cql3_dequote_value
-    dequote_name = cql3_dequote_name
-    escape_value = cql3_escape_value
-    escape_name = cql3_escape_name
-    maybe_escape_name = cql3_maybe_escape_name
+    @staticmethod
+    def dequote_value(cqlword):
+        cqlword = cqlword.strip()
+        if cqlword == '':
+            return cqlword
+        if cqlword[0] == "'" and cqlword[-1] == "'":
+            cqlword = cqlword[1:-1].replace("''", "'")
+        return cqlword
 
 CqlRuleSet = Cql3ParsingRuleSet()
 
@@ -191,7 +168,6 @@ for shorthand in shorthands:
     globals()[shorthand] = getattr(CqlRuleSet, shorthand)
 
 
-
 # BEGIN SYNTAX/COMPLETION RULE DEFINITIONS
 
 syntax_rules = r'''
@@ -327,10 +303,7 @@ JUNK ::= /([ \t\r\f\v]+|(--|[/][/])[^\n\r]*([\n\r]|$)|[/][*].*?[*][/])/ ;
                         | <K_VALUES> )
                       ;
 
-# <property> will be defined once cqlsh determines whether we're using
-# 3.0.0-beta1 or later. :/
-
-<newPropSpec> ::= [propname]=<cident> propeq="=" [propval]=<propertyValue>
+<property> ::= [propname]=<cident> propeq="=" [propval]=<propertyValue>
                 ;
 <propertyValue> ::= propsimpleval=( <stringLiteral>
                                   | <identifier>
@@ -344,37 +317,8 @@ JUNK ::= /([ \t\r\f\v]+|(--|[/][/])[^\n\r]*([\n\r]|$)|[/][*].*?[*][/])/ ;
                       ender="}"
                     ;
 
-<oldPropSpec> ::= [propname]=<optionName> propeq="=" [optval]=<optionVal>
-                ;
-<optionName> ::= optname=<cident> ( optsep=":" subopt=( <cident> | <wholenumber> ) )?
-               ;
-<optionVal> ::= <identifier>
-              | <stringLiteral>
-              | <integer>
-              | <float>
-              ;
 '''
 
-def use_pre_3_0_0_syntax():
-    # cassandra-1.1 support
-    CqlRuleSet.append_rules('''
-        <property> ::= <oldPropSpec> ;
-    ''')
-    CqlRuleSet.columnfamily_layout_map_options = \
-            CqlRuleSet.old_columnfamily_layout_map_options
-    CqlRuleSet.columnfamily_layout_options = \
-            CqlRuleSet.old_columnfamily_layout_options
-
-def use_post_3_0_0_syntax():
-    CqlRuleSet.append_rules('''
-        <property> ::= <newPropSpec> ;
-    ''')
-    CqlRuleSet.columnfamily_layout_map_options = \
-            CqlRuleSet.new_columnfamily_layout_map_options
-    CqlRuleSet.columnfamily_layout_options = \
-            CqlRuleSet.new_columnfamily_layout_options
-    CqlRuleSet.obsolete_cf_options += CqlRuleSet.new_obsolete_cf_options
-
 def prop_equals_completer(ctxt, cass):
     if not working_on_keyspace(ctxt):
         # we know if the thing in the property name position is "compact" or
@@ -387,51 +331,50 @@ def prop_equals_completer(ctxt, cass):
             return ()
     return ['=']
 
-completer_for('oldPropSpec', 'propeq')(prop_equals_completer)
-completer_for('newPropSpec', 'propeq')(prop_equals_completer)
+completer_for('property', 'propeq')(prop_equals_completer)
 
-@completer_for('newPropSpec', 'propname')
-def new_prop_name_completer(ctxt, cass):
+@completer_for('property', 'propname')
+def prop_name_completer(ctxt, cass):
     if working_on_keyspace(ctxt):
-        return ks_new_prop_name_completer(ctxt, cass)
+        return ks_prop_name_completer(ctxt, cass)
     else:
-        return cf_new_prop_name_completer(ctxt, cass)
+        return cf_prop_name_completer(ctxt, cass)
 
 @completer_for('propertyValue', 'propsimpleval')
-def new_prop_val_completer(ctxt, cass):
+def prop_val_completer(ctxt, cass):
     if working_on_keyspace(ctxt):
-        return ks_new_prop_val_completer(ctxt, cass)
+        return ks_prop_val_completer(ctxt, cass)
     else:
-        return cf_new_prop_val_completer(ctxt, cass)
+        return cf_prop_val_completer(ctxt, cass)
 
 @completer_for('propertyValue', 'propmapkey')
-def new_prop_val_mapkey_completer(ctxt, cass):
+def prop_val_mapkey_completer(ctxt, cass):
     if working_on_keyspace(ctxt):
-        return ks_new_prop_val_mapkey_completer(ctxt, cass)
+        return ks_prop_val_mapkey_completer(ctxt, cass)
     else:
-        return cf_new_prop_val_mapkey_completer(ctxt, cass)
+        return cf_prop_val_mapkey_completer(ctxt, cass)
 
 @completer_for('propertyValue', 'propmapval')
-def new_prop_val_mapval_completer(ctxt, cass):
+def prop_val_mapval_completer(ctxt, cass):
     if working_on_keyspace(ctxt):
-        return ks_new_prop_val_mapval_completer(ctxt, cass)
+        return ks_prop_val_mapval_completer(ctxt, cass)
     else:
-        return cf_new_prop_val_mapval_completer(ctxt, cass)
+        return cf_prop_val_mapval_completer(ctxt, cass)
 
 @completer_for('propertyValue', 'ender')
-def new_prop_val_mapender_completer(ctxt, cass):
+def prop_val_mapender_completer(ctxt, cass):
     if working_on_keyspace(ctxt):
-        return ks_new_prop_val_mapender_completer(ctxt, cass)
+        return ks_prop_val_mapender_completer(ctxt, cass)
     else:
-        return cf_new_prop_val_mapender_completer(ctxt, cass)
+        return cf_prop_val_mapender_completer(ctxt, cass)
 
-def ks_new_prop_name_completer(ctxt, cass):
+def ks_prop_name_completer(ctxt, cass):
     optsseen = ctxt.get_binding('propname', ())
     if 'replication' not in optsseen:
         return ['replication']
     return ["durable_writes"]
 
-def ks_new_prop_val_completer(ctxt, cass):
+def ks_prop_val_completer(ctxt, cass):
     optname = ctxt.get_binding('propname')[-1]
     if optname == 'durable_writes':
         return ["'true'", "'false'"]
@@ -439,7 +382,7 @@ def ks_new_prop_val_completer(ctxt, cass):
         return ["{'class': '"]
     return ()
 
-def ks_new_prop_val_mapkey_completer(ctxt, cass):
+def ks_prop_val_mapkey_completer(ctxt, cass):
     optname = ctxt.get_binding('propname')[-1]
     if optname != 'replication':
         return ()
@@ -457,7 +400,7 @@ def ks_new_prop_val_mapkey_completer(ctxt, cass):
         return [Hint('<dc_name>')]
     return map(escape_value, opts.difference(keysseen))
 
-def ks_new_prop_val_mapval_completer(ctxt, cass):
+def ks_prop_val_mapval_completer(ctxt, cass):
     optname = ctxt.get_binding('propname')[-1]
     if optname != 'replication':
         return ()
@@ -466,7 +409,7 @@ def ks_new_prop_val_mapval_completer(ctxt, cass):
         return map(escape_value, CqlRuleSet.replication_strategies)
     return [Hint('<value>')]
 
-def ks_new_prop_val_mapender_completer(ctxt, cass):
+def ks_prop_val_mapender_completer(ctxt, cass):
     optname = ctxt.get_binding('propname')[-1]
     if optname != 'replication':
         return [',']
@@ -485,11 +428,11 @@ def ks_new_prop_val_mapender_completer(ctxt, cass):
         return [',']
     return ['}']
 
-def cf_new_prop_name_completer(ctxt, cass):
+def cf_prop_name_completer(ctxt, cass):
     return [c[0] for c in (CqlRuleSet.columnfamily_layout_options +
                            CqlRuleSet.columnfamily_layout_map_options)]
 
-def cf_new_prop_val_completer(ctxt, cass):
+def cf_prop_val_completer(ctxt, cass):
     exist_opts = ctxt.get_binding('propname')
     this_opt = exist_opts[-1]
     if this_opt == 'compression':
@@ -512,7 +455,7 @@ def cf_new_prop_val_completer(ctxt, cass):
         return CqlRuleSet.consistency_levels
     return [Hint('<option_value>')]
 
-def cf_new_prop_val_mapkey_completer(ctxt, cass):
+def cf_prop_val_mapkey_completer(ctxt, cass):
     optname = ctxt.get_binding('propname')[-1]
     for cql3option, _, subopts in CqlRuleSet.columnfamily_layout_map_options:
         if optname == cql3option:
@@ -538,7 +481,7 @@ def cf_new_prop_val_mapkey_completer(ctxt, cass):
         return map(escape_value, opts)
     return ()
 
-def cf_new_prop_val_mapval_completer(ctxt, cass):
+def cf_prop_val_mapval_completer(ctxt, cass):
     opt = ctxt.get_binding('propname')[-1]
     key = dequote_value(ctxt.get_binding('propmapkey')[-1])
     if opt == 'compaction':
@@ -551,133 +494,12 @@ def cf_new_prop_val_mapval_completer(ctxt, cass):
         return [Hint('<option_value>')]
     return ()
 
-def cf_new_prop_val_mapender_completer(ctxt, cass):
+def cf_prop_val_mapender_completer(ctxt, cass):
     return [',', '}']
 
-@completer_for('optionName', 'optname')
-def old_prop_name_completer(ctxt, cass):
-    if working_on_keyspace(ctxt):
-        return ks_old_prop_name_completer(ctxt, cass)
-    else:
-        return cf_old_prop_name_completer(ctxt, cass)
-
-@completer_for('oldPropSpec', 'optval')
-def old_prop_val_completer(ctxt, cass):
-    if working_on_keyspace(ctxt):
-        return ks_old_prop_val_completer(ctxt, cass)
-    else:
-        return cf_old_prop_val_completer(ctxt, cass)
-
-@completer_for('optionName', 'optsep')
-def old_prop_separator_completer(ctxt, cass):
-    if working_on_keyspace(ctxt):
-        return ks_old_prop_separator_completer(ctxt, cass)
-    else:
-        return cf_old_prop_separator_completer(ctxt, cass)
-
-@completer_for('optionName', 'subopt')
-def old_prop_suboption_completer(ctxt, cass):
-    if working_on_keyspace(ctxt):
-        return ks_old_prop_suboption_completer(ctxt, cass)
-    else:
-        return cf_old_prop_suboption_completer(ctxt, cass)
-
-def ks_old_prop_name_completer(ctxt, cass):
-    exist_opts = ctxt.get_binding('optname', ())
-    try:
-        stratopt = exist_opts.index('strategy_class')
-    except ValueError:
-        return ['strategy_class =']
-    vals = ctxt.get_binding('optval')
-    stratclass = dequote_value(vals[stratopt])
-    if stratclass in CqlRuleSet.replication_factor_strategies:
-        return ['strategy_options:replication_factor =']
-    return [Hint('<strategy_option_name>')]
-
-def ks_old_prop_val_completer(ctxt, cass):
-    exist_opts = ctxt.get_binding('optname', (None,))
-    if exist_opts[-1] == 'strategy_class':
-        return map(escape_value, CqlRuleSet.replication_strategies)
-    return [Hint('<option_value>')]
-
-def ks_old_prop_separator_completer(ctxt, cass):
-    curopt = ctxt.get_binding('optname')[-1]
-    if curopt == 'strategy_options':
-        return [':']
-    return ()
-
-def ks_old_prop_suboption_completer(ctxt, cass):
-    exist_opts = ctxt.get_binding('optname')
-    if exist_opts[-1] != 'strategy_options':
-        return ()
-    try:
-        stratopt = exist_opts.index('strategy_class')
-    except ValueError:
-        return ()
-    vals = ctxt.get_binding('optval')
-    stratclass = dequote_value(vals[stratopt])
-    if stratclass in CqlRuleSet.replication_factor_strategies:
-        return ['replication_factor =']
-    return [Hint('<dc_name>')]
-
-def cf_old_prop_name_completer(ctxt, cass):
-    return list(CqlRuleSet.columnfamily_layout_options) + \
-           [c[0] + ':' for c in CqlRuleSet.columnfamily_layout_map_options]
-
-def cf_old_prop_val_completer(ctxt, cass):
-    exist_opts = ctxt.get_binding('propname')
-    this_opt = exist_opts[-1]
-    if this_opt == 'compression_parameters:sstable_compression':
-        return map(escape_value, CqlRuleSet.available_compression_classes)
-    if this_opt == 'compaction_strategy_class':
-        return map(escape_value, CqlRuleSet.available_compaction_classes)
-    if any(this_opt == opt[0] for opt in CqlRuleSet.obsolete_cf_options):
-        return ["'<obsolete_option>'"]
-    if this_opt in ('comparator', 'default_validation'):
-        return simple_cql_types
-    if this_opt in ('read_repair_chance', 'bloom_filter_fp_chance'):
-        return [Hint('<float_between_0_and_1>')]
-    if this_opt in ('replicate_on_write', 'populate_io_cache_on_flush'):
-        return [Hint('<yes_or_no>')]
-    if this_opt in ('min_compaction_threshold', 'max_compaction_threshold', 'gc_grace_seconds'):
-        return [Hint('<integer>')]
-    return [Hint('<option_value>')]
-
-def cf_old_prop_separator_completer(ctxt, cass):
-    opt = ctxt.get_binding('optname')
-    if any(opt == c[0] for c in CqlRuleSet.columnfamily_layout_map_options):
-        return [':']
-    return ()
-
-def cf_old_prop_suboption_completer(ctxt, cass):
-    opt = ctxt.get_binding('optname')
-    if opt == 'compaction_strategy_options':
-        # try to determine the strategy class in use
-        prevopts = ctxt.get_binding('propname', ())
-        prevvals = ctxt.get_binding('optval', ())
-        for prevopt, prevval in zip(prevopts, prevvals):
-            if prevopt == 'compaction_strategy_class':
-                csc = dequote_value(prevval)
-                break
-        else:
-            layout = get_cf_layout(ctxt, cass)
-            try:
-                csc = layout.compaction_strategy
-            except Exception:
-                csc = ''
-        csc = csc.split('.')[-1]
-        if csc == 'SizeTieredCompactionStrategy':
-            return ['min_sstable_size']
-        elif csc == 'LeveledCompactionStrategy':
-            return ['sstable_size_in_mb']
-    for optname, _, subopts in CqlRuleSet.columnfamily_layout_map_options:
-        if opt == optname:
-            return subopts
-    return ()
-
 @completer_for('tokenDefinition', 'token')
 def token_word_completer(ctxt, cass):
-    return ['TOKEN(']
+    return ['token(']
 
 @completer_for('simpleStorageType', 'typename')
 def storagetype_completer(ctxt, cass):
@@ -1024,27 +846,7 @@ def create_ks_wat_completer(ctxt, cass):
         return ['KEYSPACE']
     return ['KEYSPACE', 'SCHEMA']
 
-@completer_for('oldPropSpec', 'optname')
-def create_ks_opt_completer(ctxt, cass):
-    exist_opts = ctxt.get_binding('optname', ())
-    try:
-        stratopt = exist_opts.index('strategy_class')
-    except ValueError:
-        return ['strategy_class =']
-    vals = ctxt.get_binding('optval')
-    stratclass = dequote_value(vals[stratopt])
-    if stratclass in CqlRuleSet.replication_factor_strategies:
-        return ['strategy_options:replication_factor =']
-    return [Hint('<strategy_option_name>')]
-
-@completer_for('oldPropSpec', 'optval')
-def create_ks_optval_completer(ctxt, cass):
-    exist_opts = ctxt.get_binding('optname', (None,))
-    if exist_opts[-1] == 'strategy_class':
-        return map(escape_value, CqlRuleSet.replication_strategies)
-    return [Hint('<option_value>')]
-
-@completer_for('newPropSpec', 'propname')
+@completer_for('property', 'propname')
 def keyspace_properties_option_name_completer(ctxt, cass):
     optsseen = ctxt.get_binding('propname', ())
     if 'replication' not in optsseen:
@@ -1255,7 +1057,7 @@ explain_completion('alterInstructions', 'newcol', '<new_column_name>')
 
 syntax_rules += r'''
 <alterKeyspaceStatement> ::= "ALTER" ( "KEYSPACE" | "SCHEMA" ) ks=<alterableKeyspaceName>
-                                 "WITH" <newPropSpec> ( "AND" <newPropSpec> )*
+                                 "WITH" <property> ( "AND" <property> )*
                            ;
 '''
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7f6ac19e/pylib/cqlshlib/cqlhandling.py
----------------------------------------------------------------------
diff --git a/pylib/cqlshlib/cqlhandling.py b/pylib/cqlshlib/cqlhandling.py
index 4650084..660c42a 100644
--- a/pylib/cqlshlib/cqlhandling.py
+++ b/pylib/cqlshlib/cqlhandling.py
@@ -19,64 +19,18 @@
 
 import re
 import traceback
-from . import pylexotron, util, helptopics
+from . import pylexotron, util
 from cql import cqltypes
 
 Hint = pylexotron.Hint
 
-SYSTEM_KEYSPACES = ('system',)
-
-cqldocs = helptopics.CQL2HelpTopics()
-
 class CqlParsingRuleSet(pylexotron.ParsingRuleSet):
-    keywords = set((
-        'select', 'from', 'where', 'and', 'key', 'insert', 'update', 'with',
-        'limit', 'using', 'consistency', 'one', 'quorum', 'all', 'any',
-        'local_quorum', 'each_quorum', 'two', 'three', 'use', 'count', 'set',
-        'begin', 'apply', 'batch', 'truncate', 'delete', 'in', 'create',
-        'keyspace', 'schema', 'columnfamily', 'table', 'index', 'on', 'drop',
-        'primary', 'into', 'values', 'timestamp', 'ttl', 'alter', 'add', 'type',
-        'first', 'reversed'
-    ))
-
-    columnfamily_options = (
-        # (CQL option name, Thrift option name (or None if same))
-        ('comment', None),
-        ('comparator', 'comparator_type'),
-        ('read_repair_chance', None),
-        ('gc_grace_seconds', None),
-        ('default_validation', 'default_validation_class'),
-        ('min_compaction_threshold', None),
-        ('max_compaction_threshold', None),
-        ('replicate_on_write', None),
-        ('compaction_strategy_class', 'compaction_strategy'),
-        ('default_time_to_live', None),
-        ('populate_io_cache_on_flush', None),
-    )
-
-    obsolete_cf_options = (
-        ('key_cache_size', None),
-        ('row_cache_size', None),
-        ('row_cache_save_period_in_seconds', None),
-        ('key_cache_save_period_in_seconds', None),
-        ('memtable_throughput_in_mb', None),
-        ('memtable_operations_in_millions', None),
-        ('memtable_flush_after_mins', None),
-        ('row_cache_provider', None),
-    )
-
-    all_columnfamily_options = columnfamily_options + obsolete_cf_options
-
-    columnfamily_map_options = (
-        ('compaction_strategy_options', None,
-            ()),
-        ('compression_parameters', 'compression_options',
-            ('sstable_compression', 'chunk_length_kb', 'crc_check_chance')),
-    )
+    keywords = set()
 
     available_compression_classes = (
         'DeflateCompressor',
         'SnappyCompressor',
+        'LZ4Compressor',
     )
 
     available_compaction_classes = (
@@ -97,20 +51,6 @@ class CqlParsingRuleSet(pylexotron.ParsingRuleSet):
         'org.apache.cassandra.locator.OldNetworkTopologyStrategy'
     )
 
-    consistency_levels = (
-        'ANY',
-        'ONE',
-        'TWO',
-        'THREE',
-        'QUORUM',
-        'ALL',
-        'LOCAL_QUORUM',
-        'EACH_QUORUM'
-    )
-
-    # if a term matches this, it shouldn't need to be quoted to be valid cql
-    valid_cql_word_re = re.compile(r"^(?:[a-z][a-z0-9_]*|-?[0-9][0-9.]*)$", re.I)
-
     def __init__(self, *args, **kwargs):
         pylexotron.ParsingRuleSet.__init__(self, *args, **kwargs)
 
@@ -356,10 +296,6 @@ class CqlParsingRuleSet(pylexotron.ParsingRuleSet):
             print "** New total completion: %r. Checking for further matches...\n" % (first,)
         return self.cql_complete_multiple(text, first, init_bindings, startsymbol=startsymbol)
 
-    @classmethod
-    def is_valid_cql_word(cls, s):
-        return cls.valid_cql_word_re.match(s) is not None and s.lower() not in cls.keywords
-
     @staticmethod
     def cql_extract_orig(toklist, srcstr):
         # low end of span for first token, to high end of span for last token
@@ -367,6 +303,9 @@ class CqlParsingRuleSet(pylexotron.ParsingRuleSet):
 
     @staticmethod
     def token_dequote(tok):
+        if tok[0] == 'unclosedName':
+            # strip one quote
+            return tok[1][1:].replace('""', '"')
         if tok[0] == 'stringLiteral':
             # strip quotes
             return tok[1][1:-1].replace("''", "'")
@@ -380,542 +319,3 @@ class CqlParsingRuleSet(pylexotron.ParsingRuleSet):
     @staticmethod
     def token_is_word(tok):
         return tok[0] == 'identifier'
-
-    @classmethod
-    def cql2_maybe_escape_name(cls, name):
-        if cls.is_valid_cql_word(name):
-            return name
-        return cls.cql2_escape_name(name)
-
-    # XXX: this doesn't really belong here.
-    @classmethod
-    def is_counter_col(cls, cfdef, colname):
-        col_info = [cm for cm in cfdef.column_metadata if cm.name == colname]
-        return bool(col_info and cqltypes.is_counter_type(col_info[0].validation_class))
-
-    @staticmethod
-    def cql2_dequote_value(cqlword):
-        cqlword = cqlword.strip()
-        if cqlword == '':
-            return cqlword
-        if cqlword[0] == "'" and cqlword[-1] == "'":
-            cqlword = cqlword[1:-1].replace("''", "'")
-        return cqlword
-
-    @staticmethod
-    def cql2_escape_value(value):
-        if value is None:
-            return 'NULL' # this totally won't work
-        if isinstance(value, bool):
-            value = str(value).lower()
-        elif isinstance(value, float):
-            return '%f' % value
-        elif isinstance(value, int):
-            return str(value)
-        return "'%s'" % value.replace("'", "''")
-
-    # use _name for keyspace, cf, and column names, and _value otherwise.
-    # also use the cql2_ prefix when dealing with cql2, or leave it off to
-    # get whatever behavior is default for this CqlParsingRuleSet.
-    cql2_dequote_name = dequote_name = dequote_value = cql2_dequote_value
-    cql2_escape_name = escape_name = escape_value = cql2_escape_value
-    maybe_escape_name = cql2_maybe_escape_name
-    dequote_any = cql2_dequote_value
-
-CqlRuleSet = CqlParsingRuleSet()
-
-# convenience for remainder of module
-shorthands = ('completer_for', 'explain_completion',
-              'dequote_value', 'dequote_name',
-              'escape_value', 'escape_name',
-              'maybe_escape_name')
-
-for shorthand in shorthands:
-    globals()[shorthand] = getattr(CqlRuleSet, shorthand)
-
-
-
-# BEGIN SYNTAX/COMPLETION RULE DEFINITIONS
-
-syntax_rules = r'''
-<Start> ::= <CQL_Statement>*
-          ;
-
-<CQL_Statement> ::= [statements]=<statementBody> ";"
-                  ;
-
-# the order of these terminal productions is significant:
-<endline> ::= /\n/ ;
-
-JUNK ::= /([ \t\r\f\v]+|(--|[/][/])[^\n\r]*([\n\r]|$)|[/][*].*?[*][/])/ ;
-
-<stringLiteral> ::= /'([^']|'')*'/ ;
-<float> ::=         /-?[0-9]+\.[0-9]+/ ;
-<integer> ::=       /-?[0-9]+/ ;
-<uuid> ::=          /[0-9a-f]{8}-[0-9a-f]{4}-[0-9a-f]{4}-[0-9a-f]{4}-[0-9a-f]{12}/ ;
-<identifier> ::=    /[a-z][a-z0-9_]*/ ;
-<colon> ::=         ":" ;
-<star> ::=          "*" ;
-<range> ::=         ".." ;
-<endtoken> ::=      ";" ;
-<op> ::=            /[-+=,().]/ ;
-<cmp> ::=           /[<>]=?/ ;
-
-<unclosedString>  ::= /'([^']|'')*/ ;
-<unclosedComment> ::= /[/][*][^\n]*$/ ;
-
-<symbol> ::= <star>
-           | <range>
-           | <op>
-           | <cmp>
-           ;
-<name> ::= <identifier>
-         | <stringLiteral>
-         | <integer>
-         ;
-<term> ::= <stringLiteral>
-         | <integer>
-         | <float>
-         | <uuid>
-         ;
-<colname> ::= <term>
-            | <identifier>
-            | nocomplete=<K_KEY>
-            ;
-
-<statementBody> ::= <useStatement>
-                  | <selectStatement>
-                  | <dataChangeStatement>
-                  | <schemaChangeStatement>
-                  ;
-
-<dataChangeStatement> ::= <insertStatement>
-                        | <updateStatement>
-                        | <deleteStatement>
-                        | <truncateStatement>
-                        | <batchStatement>
-                        ;
-
-<schemaChangeStatement> ::= <createKeyspaceStatement>
-                          | <createColumnFamilyStatement>
-                          | <createIndexStatement>
-                          | <dropKeyspaceStatement>
-                          | <dropColumnFamilyStatement>
-                          | <dropIndexStatement>
-                          | <alterTableStatement>
-                          ;
-
-<consistencylevel> ::= cl=<identifier> ;
-
-<storageType> ::= typename=( <identifier> | <stringLiteral> ) ;
-
-<keyspaceName> ::= ksname=<name> ;
-
-<columnFamilyName> ::= ( ksname=<name> "." )? cfname=<name> ;
-'''
-
-@completer_for('colname', 'nocomplete')
-def nocomplete(ctxt, cass):
-    return ()
-
-@completer_for('consistencylevel', 'cl')
-def cl_completer(ctxt, cass):
-    return CqlRuleSet.consistency_levels
-
-@completer_for('storageType', 'typename')
-def storagetype_completer(ctxt, cass):
-    return cqltypes.cql_types
-
-@completer_for('keyspaceName', 'ksname')
-def ks_name_completer(ctxt, cass):
-    return map(maybe_escape_name, cass.get_keyspace_names())
-
-@completer_for('columnFamilyName', 'ksname')
-def cf_ks_name_completer(ctxt, cass):
-    return [maybe_escape_name(ks) + '.' for ks in cass.get_keyspace_names()]
-
-@completer_for('columnFamilyName', 'cfname')
-def cf_name_completer(ctxt, cass):
-    ks = ctxt.get_binding('ksname', None)
-    if ks is not None:
-        ks = dequote_name(ks)
-    try:
-        cfnames = cass.get_columnfamily_names(ks)
-    except Exception:
-        if ks is None:
-            return ()
-        raise
-    return map(maybe_escape_name, cfnames)
-
-def get_cfdef(ctxt, cass):
-    ks = ctxt.get_binding('ksname', None)
-    cf = ctxt.get_binding('cfname')
-    return cass.get_columnfamily(cf, ksname=ks)
-
-syntax_rules += r'''
-<useStatement> ::= "USE" ksname=<keyspaceName>
-                 ;
-<selectStatement> ::= "SELECT" <whatToSelect>
-                        "FROM" cf=<columnFamilyName>
-                          ("USING" "CONSISTENCY" selcl=<consistencylevel>)?
-                          ("WHERE" <selectWhereClause>)?
-                          ("LIMIT" limit=<integer>)?
-                    ;
-<selectWhereClause> ::= <relation> ("AND" <relation>)*
-                      | keyname=<colname> "IN" "(" <term> ("," <term>)* ")"
-                      ;
-<relation> ::= [rel_lhs]=<colname> ("=" | "<" | ">" | "<=" | ">=") <colname>
-             ;
-<whatToSelect> ::= colname=<colname> ("," colname=<colname>)*
-                 | ("FIRST" <integer>)? "REVERSED"? (rangestart=<colname> ".." rangeend=<colname>
-                                                     | "*")
-                 | "COUNT" countparens="(" "*" ")"
-                 ;
-'''
-
-@completer_for('selectStatement', 'selcl')
-def select_statement_consistencylevel(ctxt, cass):
-    return [cl for cl in CqlRuleSet.consistency_levels if cl != 'ANY']
-
-@completer_for('selectWhereClause', 'keyname')
-def select_where_keyname_completer(ctxt, cass):
-    cfdef = get_cfdef(ctxt, cass)
-    return [cfdef.key_alias if cfdef.key_alias is not None else 'KEY']
-
-@completer_for('relation', 'rel_lhs')
-def select_relation_lhs_completer(ctxt, cass):
-    cfdef = get_cfdef(ctxt, cass)
-    return map(maybe_escape_name, cass.filterable_column_names(cfdef))
-
-@completer_for('whatToSelect', 'countparens')
-def select_count_parens_completer(ctxt, cass):
-    return ['(*)']
-
-explain_completion('whatToSelect', 'colname')
-explain_completion('whatToSelect', 'rangestart', '<range_start>')
-explain_completion('whatToSelect', 'rangeend', '<range_end>')
-
-syntax_rules += r'''
-<insertStatement> ::= "INSERT" "INTO" cf=<columnFamilyName>
-                               "(" keyname=<colname> ","
-                                   [colname]=<colname> ( "," [colname]=<colname> )* ")"
-                      "VALUES" "(" <term> "," <term> ( "," <term> )* ")"
-                      ( "USING" [insertopt]=<usingOption>
-                                ( "AND" [insertopt]=<usingOption> )* )?
-                    ;
-<usingOption> ::= "CONSISTENCY" <consistencylevel>
-                | "TIMESTAMP" <integer>
-                | "TTL" <integer>
-                ;
-'''
-
-@completer_for('insertStatement', 'keyname')
-def insert_keyname_completer(ctxt, cass):
-    cfdef = get_cfdef(ctxt, cass)
-    return [cfdef.key_alias if cfdef.key_alias is not None else 'KEY']
-
-explain_completion('insertStatement', 'colname')
-
-@completer_for('insertStatement', 'insertopt')
-def insert_option_completer(ctxt, cass):
-    opts = set('CONSISTENCY TIMESTAMP TTL'.split())
-    for opt in ctxt.get_binding('insertopt', ()):
-        opts.discard(opt.split()[0])
-    return opts
-
-syntax_rules += r'''
-<updateStatement> ::= "UPDATE" cf=<columnFamilyName>
-                        ( "USING" [updateopt]=<usingOption>
-                                  ( "AND" [updateopt]=<usingOption> )* )?
-                        "SET" <assignment> ( "," <assignment> )*
-                        "WHERE" <updateWhereClause>
-                    ;
-<assignment> ::= updatecol=<colname> "=" update_rhs=<colname>
-                                         ( counterop=( "+" | "-"? ) <integer> )?
-               ;
-<updateWhereClause> ::= updatefiltercol=<colname> "=" <term>
-                      | updatefilterkey=<colname> filter_in="IN" "(" <term> ( "," <term> )* ")"
-                      ;
-'''
-
-@completer_for('updateStatement', 'updateopt')
-def insert_option_completer(ctxt, cass):
-    opts = set('CONSISTENCY TIMESTAMP TTL'.split())
-    for opt in ctxt.get_binding('updateopt', ()):
-        opts.discard(opt.split()[0])
-    return opts
-
-@completer_for('assignment', 'updatecol')
-def update_col_completer(ctxt, cass):
-    cfdef = get_cfdef(ctxt, cass)
-    colnames = map(maybe_escape_name, [cm.name for cm in cfdef.column_metadata])
-    return colnames + [Hint('<colname>')]
-
-@completer_for('assignment', 'update_rhs')
-def update_countername_completer(ctxt, cass):
-    cfdef = get_cfdef(ctxt, cass)
-    curcol = dequote_name(ctxt.get_binding('updatecol', ''))
-    return [maybe_escape_name(curcol)] if CqlRuleSet.is_counter_col(cfdef, curcol) else [Hint('<term>')]
-
-@completer_for('assignment', 'counterop')
-def update_counterop_completer(ctxt, cass):
-    cfdef = get_cfdef(ctxt, cass)
-    curcol = dequote_name(ctxt.get_binding('updatecol', ''))
-    return ['+', '-'] if CqlRuleSet.is_counter_col(cfdef, curcol) else []
-
-@completer_for('updateWhereClause', 'updatefiltercol')
-def update_filtercol_completer(ctxt, cass):
-    cfdef = get_cfdef(ctxt, cass)
-    return map(maybe_escape_name, cass.filterable_column_names(cfdef))
-
-@completer_for('updateWhereClause', 'updatefilterkey')
-def update_filterkey_completer(ctxt, cass):
-    cfdef = get_cfdef(ctxt, cass)
-    return [cfdef.key_alias if cfdef.key_alias is not None else 'KEY']
-
-@completer_for('updateWhereClause', 'filter_in')
-def update_filter_in_completer(ctxt, cass):
-    cfdef = get_cfdef(ctxt, cass)
-    fk = ctxt.get_binding('updatefilterkey')
-    return ['IN'] if fk in ('KEY', cfdef.key_alias) else []
-
-syntax_rules += r'''
-<deleteStatement> ::= "DELETE" ( [delcol]=<colname> ( "," [delcol]=<colname> )* )?
-                        "FROM" cf=<columnFamilyName>
-                        ( "USING" [delopt]=<deleteOption> ( "AND" [delopt]=<deleteOption> )* )?
-                        "WHERE" <updateWhereClause>
-                    ;
-<deleteOption> ::= "CONSISTENCY" <consistencylevel>
-                 | "TIMESTAMP" <integer>
-                 ;
-'''
-
-@completer_for('deleteStatement', 'delopt')
-def delete_opt_completer(ctxt, cass):
-    opts = set('CONSISTENCY TIMESTAMP'.split())
-    for opt in ctxt.get_binding('delopt', ()):
-        opts.discard(opt.split()[0])
-    return opts
-
-explain_completion('deleteStatement', 'delcol', '<column_to_delete>')
-
-syntax_rules += r'''
-<batchStatement> ::= "BEGIN" "BATCH"
-                        ( "USING" [batchopt]=<usingOption>
-                                  ( "AND" [batchopt]=<usingOption> )* )?
-                        [batchstmt]=<batchStatementMember> ";"
-                            ( [batchstmt]=<batchStatementMember> ";" )*
-                     "APPLY" "BATCH"
-                   ;
-<batchStatementMember> ::= <insertStatement>
-                         | <updateStatement>
-                         | <deleteStatement>
-                         ;
-'''
-
-@completer_for('batchStatement', 'batchopt')
-def batch_opt_completer(ctxt, cass):
-    opts = set('CONSISTENCY TIMESTAMP'.split())
-    for opt in ctxt.get_binding('batchopt', ()):
-        opts.discard(opt.split()[0])
-    return opts
-
-syntax_rules += r'''
-<truncateStatement> ::= "TRUNCATE" cf=<columnFamilyName>
-                      ;
-'''
-
-syntax_rules += r'''
-<createKeyspaceStatement> ::= "CREATE" "KEYSPACE" ksname=<name>
-                                 "WITH" [optname]=<optionName> "=" [optval]=<optionVal>
-                                 ( "AND" [optname]=<optionName> "=" [optval]=<optionVal> )*
-                            ;
-<optionName> ::= <identifier> ( ":" ( <identifier> | <integer> ) )?
-               ;
-<optionVal> ::= <stringLiteral>
-              | <identifier>
-              | <integer>
-              ;
-'''
-
-explain_completion('createKeyspaceStatement', 'ksname', '<new_keyspace_name>')
-
-@completer_for('createKeyspaceStatement', 'optname')
-def create_ks_opt_completer(ctxt, cass):
-    exist_opts = ctxt.get_binding('optname', ())
-    try:
-        stratopt = exist_opts.index('strategy_class')
-    except ValueError:
-        return ['strategy_class =']
-    vals = ctxt.get_binding('optval')
-    stratclass = dequote_value(vals[stratopt])
-    if stratclass in CqlRuleSet.replication_factor_strategies:
-        return ['strategy_options:replication_factor =']
-    return [Hint('<strategy_option_name>')]
-
-@completer_for('createKeyspaceStatement', 'optval')
-def create_ks_optval_completer(ctxt, cass):
-    exist_opts = ctxt.get_binding('optname', (None,))
-    if exist_opts[-1] == 'strategy_class':
-        return map(escape_value, CqlRuleSet.replication_strategies)
-    return [Hint('<option_value>')]
-
-syntax_rules += r'''
-<createColumnFamilyStatement> ::= "CREATE" ( "COLUMNFAMILY" | "TABLE" ) cf=<name>
-                                    "(" keyalias=<colname> <storageType> "PRIMARY" "KEY"
-                                        ( "," colname=<colname> <storageType> )* ")"
-                                   ( "WITH" [cfopt]=<cfOptionName> "=" [optval]=<cfOptionVal>
-                                     ( "AND" [cfopt]=<cfOptionName> "=" [optval]=<cfOptionVal> )* )?
-                                ;
-
-<cfOptionName> ::= cfoptname=<identifier> ( cfoptsep=":" cfsubopt=( <identifier> | <integer> ) )?
-                 ;
-
-<cfOptionVal> ::= <identifier>
-                | <stringLiteral>
-                | <integer>
-                | <float>
-                ;
-'''
-
-explain_completion('createColumnFamilyStatement', 'keyalias', '<new_key_name>')
-explain_completion('createColumnFamilyStatement', 'cf', '<new_table_name>')
-explain_completion('createColumnFamilyStatement', 'colname', '<new_column_name>')
-
-@completer_for('cfOptionName', 'cfoptname')
-def create_cf_option_completer(ctxt, cass):
-    return [c[0] for c in CqlRuleSet.columnfamily_options] + \
-           [c[0] + ':' for c in CqlRuleSet.columnfamily_map_options]
-
-@completer_for('cfOptionName', 'cfoptsep')
-def create_cf_suboption_separator(ctxt, cass):
-    opt = ctxt.get_binding('cfoptname')
-    if any(opt == c[0] for c in CqlRuleSet.columnfamily_map_options):
-        return [':']
-    return ()
-
-@completer_for('cfOptionName', 'cfsubopt')
-def create_cf_suboption_completer(ctxt, cass):
-    opt = ctxt.get_binding('cfoptname')
-    if opt == 'compaction_strategy_options':
-        # try to determine the strategy class in use
-        prevopts = ctxt.get_binding('cfopt', ())
-        prevvals = ctxt.get_binding('optval', ())
-        for prevopt, prevval in zip(prevopts, prevvals):
-            if prevopt == 'compaction_strategy_class':
-                csc = dequote_value(prevval)
-                break
-        else:
-            cf = ctxt.get_binding('cf')
-            try:
-                csc = cass.get_columnfamily(cf).compaction_strategy
-            except Exception:
-                csc = ''
-        csc = csc.split('.')[-1]
-        if csc == 'SizeTieredCompactionStrategy':
-            return ['min_sstable_size']
-        elif csc == 'LeveledCompactionStrategy':
-            return ['sstable_size_in_mb']
-    for optname, _, subopts in CqlRuleSet.columnfamily_map_options:
-        if opt == optname:
-            return subopts
-    return ()
-
-def create_cf_option_val_completer(ctxt, cass):
-    exist_opts = ctxt.get_binding('cfopt')
-    this_opt = exist_opts[-1]
-    if this_opt == 'compression_parameters:sstable_compression':
-        return map(escape_value, CqlRuleSet.available_compression_classes)
-    if this_opt == 'compaction_strategy_class':
-        return map(escape_value, CqlRuleSet.available_compaction_classes)
-    if any(this_opt == opt[0] for opt in CqlRuleSet.obsolete_cf_options):
-        return ["'<obsolete_option>'"]
-    if this_opt in ('comparator', 'default_validation'):
-        return cqltypes.cql_types
-    if this_opt == 'read_repair_chance':
-        return [Hint('<float_between_0_and_1>')]
-    if this_opt in ('replicate_on_write', 'populate_io_cache_on_flush'):
-        return [Hint('<yes_or_no>')]
-    if this_opt in ('min_compaction_threshold', 'max_compaction_threshold', 'gc_grace_seconds', 'default_time_to_live'):
-        return [Hint('<integer>')]
-    return [Hint('<option_value>')]
-
-completer_for('createColumnFamilyStatement', 'optval') \
-    (create_cf_option_val_completer)
-
-syntax_rules += r'''
-<createIndexStatement> ::= "CREATE" "INDEX" indexname=<identifier>? "ON"
-                               cf=<name> "(" col=<colname> ")"
-                         ;
-'''
-
-explain_completion('createIndexStatement', 'indexname', '<new_index_name>')
-
-@completer_for('createIndexStatement', 'cf')
-def create_index_cf_completer(ctxt, cass):
-    return map(maybe_escape_name, cass.get_columnfamily_names())
-
-@completer_for('createIndexStatement', 'col')
-def create_index_col_completer(ctxt, cass):
-    cfdef = cass.get_columnfamily(dequote_name(ctxt.get_binding('cf')))
-    colnames = [md.name for md in cfdef.column_metadata if md.index_name is None]
-    return map(maybe_escape_name, colnames)
-
-syntax_rules += r'''
-<dropKeyspaceStatement> ::= "DROP" "KEYSPACE" ksname=<keyspaceName>
-                          ;
-'''
-
-@completer_for('dropKeyspaceStatement', 'ksname')
-def drop_ks_completer(ctxt, cass):
-    return map(maybe_escape_name, cass.get_keyspace_names())
-
-syntax_rules += r'''
-<dropColumnFamilyStatement> ::= "DROP" ( "COLUMNFAMILY" | "TABLE" ) cf=<name>
-                              ;
-'''
-
-@completer_for('dropColumnFamilyStatement', 'cf')
-def drop_cf_completer(ctxt, cass):
-    return map(maybe_escape_name, cass.get_columnfamily_names())
-
-syntax_rules += r'''
-<dropIndexStatement> ::= "DROP" "INDEX" indexname=<name>
-                       ;
-'''
-
-@completer_for('dropIndexStatement', 'indexname')
-def drop_index_completer(ctxt, cass):
-    return map(maybe_escape_name, cass.get_index_names())
-
-syntax_rules += r'''
-<alterTableStatement> ::= "ALTER" ( "COLUMNFAMILY" | "TABLE" ) cf=<name> <alterInstructions>
-                        ;
-<alterInstructions> ::= "ALTER" existcol=<name> "TYPE" <storageType>
-                      | "ADD" newcol=<name> <storageType>
-                      | "DROP" existcol=<name>
-                      | "WITH" [cfopt]=<cfOptionName> "=" [optval]=<cfOptionVal>
-                        ( "AND" [cfopt]=<cfOptionName> "=" [optval]=<cfOptionVal> )*
-                      ;
-'''
-
-@completer_for('alterTableStatement', 'cf')
-def alter_table_cf_completer(ctxt, cass):
-    return map(maybe_escape_name, cass.get_columnfamily_names())
-
-@completer_for('alterInstructions', 'existcol')
-def alter_table_col_completer(ctxt, cass):
-    cfdef = cass.get_columnfamily(dequote_name(ctxt.get_binding('cf')))
-    cols = [md.name for md in cfdef.column_metadata]
-    if cfdef.key_alias is not None:
-        cols.append(cfdef.key_alias)
-    return map(maybe_escape_name, cols)
-
-explain_completion('alterInstructions', 'newcol', '<new_column_name>')
-
-completer_for('alterInstructions', 'optval') \
-    (create_cf_option_val_completer)
-
-# END SYNTAX/COMPLETION RULE DEFINITIONS
-
-CqlRuleSet.append_rules(syntax_rules)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7f6ac19e/pylib/cqlshlib/helptopics.py
----------------------------------------------------------------------
diff --git a/pylib/cqlshlib/helptopics.py b/pylib/cqlshlib/helptopics.py
index a5b9e48..fb769f3 100644
--- a/pylib/cqlshlib/helptopics.py
+++ b/pylib/cqlshlib/helptopics.py
@@ -458,251 +458,6 @@ class CQLHelpTopics(object):
         unset.
         """
 
-class CQL2HelpTopics(CQLHelpTopics):
-    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_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_select(self):
-        print """
-        SELECT [FIRST n] [REVERSED] <selectExpr>
-          FROM [<keyspace>.]<table>
-            [USING CONSISTENCY <consistencylevel>]
-            [WHERE <clause>]
-            [ORDER BY <colname> [DESC]]
-            [LIMIT m];
-
-        SELECT is used to read one or more records from a CQL table. It returns
-        a set of rows matching the selection criteria specified.
-
-        Note that FIRST and REVERSED are only supported in CQL 2, and ORDER BY
-        is only supported in CQL 3 and higher.
-
-        For more information, see one of the following:
-
-          HELP SELECT_EXPR
-          HELP SELECT_TABLE
-          HELP SELECT_WHERE
-          HELP SELECT_LIMIT
-          HELP CONSISTENCYLEVEL
-        """
-
-    def help_delete(self):
-        print """
-        DELETE [<col1> [, <col2>, ...] FROM [<keyspace>.]<tablename>
-               [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):
-        print """
-        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. Cassandra timestamps are generally
-        specified using milliseconds since the Unix epoch (1970-01-01 00:00:00
-        UTC).
-        """
-
-    def help_update(self):
-        print """
-        UPDATE [<keyspace>.]<columnFamily>
-                              [USING CONSISTENCY <consistencylevel>
-                                [AND TIMESTAMP <timestamp>]
-                                [AND TTL <timeToLive>]]
-               SET name1 = value1, name2 = value2 WHERE <keycol> = keyval;
-
-        An UPDATE is used to write one or more columns to a record in a table.
-        No results are returned. The record's primary key must be completely
-        and uniquely specified; that is, if the primary key includes multiple
-        columns, all must be explicitly given in the WHERE clause.
-
-        Statements begin with the UPDATE keyword followed by the name of the
-        table to be updated.
-
-        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. Cassandra timestamps are generally specified
-        using milliseconds since the Unix epoch (1970-01-01 00:00:00 UTC).
-
-        <timeToLive> defines the optional time to live (TTL) in seconds for the
-        new column value(s). It must be an integer.
-        """
-
-    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)
-         * TWO
-         * THREE
-         * QUORUM
-         * ALL
-         * LOCAL_QUORUM
-         * EACH_QUORUM
-
-        For more information on how consistency levels work, consult your
-        Cassandra documentation.
-        """
-
-    def help_insert(self):
-        print """
-        INSERT INTO [<keyspace>.]<tablename>
-                    ( <colname1>, <colname2> [, <colname3> [, ...]] )
-               VALUES ( <colval1>, <colval2> [, <colval3> [, ...]] )
-               [USING CONSISTENCY <consistencylevel>
-                 [AND TIMESTAMP <timestamp>]
-                 [AND TTL <timeToLive>]];
-
-        An INSERT is used to write one or more columns to a record in a
-        CQL table. No results are returned.
-
-        Values for all component columns in the table's primary key must
-        be given. Also, there must be at least one non-primary-key column
-        specified (Cassandra rows are not considered to exist with only
-        a key and no associated columns).
-
-        Unlike in SQL, the semantics of INSERT and UPDATE are identical.
-        In either case a record is created if none existed before, and
-        udpated when it does. For more information, see one of the
-        following:
-
-          HELP UPDATE
-          HELP UPDATE_USING
-          HELP CONSISTENCYLEVEL
-        """
-
-    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.
-
-         ** [FIRST n] and [REVERSED] are no longer supported in CQL 3.
-        """
-
 class CQL3HelpTopics(CQLHelpTopics):
     def help_create_keyspace(self):
         print """
@@ -916,15 +671,15 @@ class CQL3HelpTopics(CQLHelpTopics):
 
     def help_create(self):
         super(CQL3HelpTopics, self).help_create()
-        print "          HELP CREATE_USER;"
+        print "          HELP CREATE_USER;\n"
 
     def help_alter(self):
         super(CQL3HelpTopics, self).help_alter()
-        print "          HELP ALTER_USER;"
+        print "          HELP ALTER_USER;\n"
 
     def help_drop(self):
         super(CQL3HelpTopics, self).help_drop()
-        print "          HELP DROP_USER;"
+        print "          HELP DROP_USER;\n"
 
     def help_list(self):
         print """

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7f6ac19e/pylib/cqlshlib/test/cassconnect.py
----------------------------------------------------------------------
diff --git a/pylib/cqlshlib/test/cassconnect.py b/pylib/cqlshlib/test/cassconnect.py
index 2883dbc..63d8c10 100644
--- a/pylib/cqlshlib/test/cassconnect.py
+++ b/pylib/cqlshlib/test/cassconnect.py
@@ -22,12 +22,11 @@ import os.path
 from .basecase import cql, cqlsh, cqlshlog, TEST_HOST, TEST_PORT, rundir
 from .run_cqlsh import run_cqlsh, call_cqlsh
 
-test_keyspace_init2 = os.path.join(rundir, 'test_keyspace_init2.cql')
-test_keyspace_init3 = os.path.join(rundir, 'test_keyspace_init3.cql')
+test_keyspace_init = os.path.join(rundir, 'test_keyspace_init.cql')
 
 def get_cassandra_connection(cql_version=None):
     if cql_version is None:
-        cql_version = '2.0.0'
+        cql_version = '3.1.0'
     conn = cql.connect(TEST_HOST, TEST_PORT, cql_version=cql_version)
     # until the cql lib does this for us
     conn.cql_version = cql_version
@@ -49,14 +48,14 @@ def create_test_keyspace(cursor):
     ksname = make_test_ks_name()
     qksname = quote_name(cursor, ksname)
     cursor.execute('''
-        CREATE KEYSPACE %s WITH strategy_class = 'SimpleStrategy'
-                           AND strategy_options:replication_factor = 1;
+        CREATE KEYSPACE %s WITH replication =
+            {'class': 'SimpleStrategy', 'replication_factor': 1};
     ''' % quote_name(cursor, ksname))
     cursor.execute('USE %s;' % qksname)
     TEST_KEYSPACES_CREATED.append(ksname)
     return ksname
 
-def split_cql_commands(source, cqlver='2.0.0'):
+def split_cql_commands(source, cqlver='3.1.0'):
     ruleset = cql_rule_set(cqlver)
     statements, in_batch = ruleset.cql_split_statements(source)
     if in_batch:
@@ -73,18 +72,10 @@ def execute_cql_file(cursor, fname):
     with open(fname) as f:
         return execute_cql_commands(cursor, f.read())
 
-def populate_test_db_cql3(cursor):
-    execute_cql_file(cursor, test_keyspace_init3)
-
-def populate_test_db_cql2(cursor):
-    execute_cql_file(cursor, test_keyspace_init2)
-
 def create_test_db():
-    with cassandra_cursor(ks=None) as c:
+    with cassandra_cursor(ks=None, cql_version='3.1.0') as c:
         k = create_test_keyspace(c)
-        populate_test_db_cql2(c)
-    with cassandra_cursor(ks=k, cql_version='3.0.0') as c:
-        populate_test_db_cql3(c)
+        execute_cql_file(c, test_keyspace_init)
     return k
 
 def remove_test_db():
@@ -132,10 +123,7 @@ def cassandra_cursor(cql_version=None, ks=''):
         conn.close()
 
 def cql_rule_set(cqlver):
-    if str(cqlver).startswith('2'):
-        return cqlsh.cqlhandling.CqlRuleSet
-    else:
-        return cqlsh.cql3handling.CqlRuleSet
+    return cqlsh.cql3handling.CqlRuleSet
 
 def quote_name(cqlver, name):
     if isinstance(cqlver, cql.cursor.Cursor):

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7f6ac19e/pylib/cqlshlib/test/table_arrangements.cql
----------------------------------------------------------------------
diff --git a/pylib/cqlshlib/test/table_arrangements.cql b/pylib/cqlshlib/test/table_arrangements.cql
deleted file mode 100644
index c3ccc41..0000000
--- a/pylib/cqlshlib/test/table_arrangements.cql
+++ /dev/null
@@ -1,114 +0,0 @@
--- type A: single-column PK, compact storage
--- FAILS: CREATE TABLE type_a_1 (a int PRIMARY KEY) WITH COMPACT STORAGE;
--- Bad Request: No definition found that is not part of the PRIMARY KEY
-CREATE TABLE type_a_2 (a int PRIMARY KEY, b int) WITH COMPACT STORAGE;
-CREATE TABLE type_a_3 (a int PRIMARY KEY, b int, c int) WITH COMPACT STORAGE;
-
--- type B: single-column PK, dynamic storage
-CREATE TABLE type_b_1 (a int PRIMARY KEY);
-CREATE TABLE type_b_2 (a int PRIMARY KEY, b int);
-CREATE TABLE type_b_3 (a int PRIMARY KEY, b int, c int);
-
--- type C: compound PK, plain partition key, compact storage
-CREATE TABLE type_c_2_2 (a int, b int, PRIMARY KEY (a, b)) WITH COMPACT STORAGE;
-CREATE TABLE type_c_3_3 (a int, b int, c int, PRIMARY KEY (a, b, c)) WITH COMPACT STORAGE;
-CREATE TABLE type_c_3_2 (a int, b int, c int, PRIMARY KEY (a, b)) WITH COMPACT STORAGE;
--- FAILS: CREATE TABLE type_c_4_2 (a int, b int, c int, d int, PRIMARY KEY (a, b)) WITH COMPACT STORAGE;
--- Bad Request: COMPACT STORAGE WITH composite PRIMARY KEY allows no more than one column not part of the PRIMARY KEY (got: d, c)
-CREATE TABLE type_c_4_3 (a int, b int, c int, d int, PRIMARY KEY (a, b, c)) WITH COMPACT STORAGE;
-
--- type D: compound PK, plain partition key, dynamic storage
-CREATE TABLE type_d_2_2 (a int, b int, PRIMARY KEY (a, b));
-CREATE TABLE type_d_3_2 (a int, b int, c int, PRIMARY KEY (a, b));
-CREATE TABLE type_d_3_3 (a int, b int, c int, PRIMARY KEY (a, b, c));
-CREATE TABLE type_d_4_2 (a int, b int, c int, d int, PRIMARY KEY (a, b));
-
--- type E: compound PK, multipart partition key, all key components used in partitioning,
--- compact storage
--- FAILS: CREATE TABLE type_e_2_2 (a int, b int, PRIMARY KEY ((a, b))) WITH COMPACT STORAGE;
--- Bad Request: No definition found that is not part of the PRIMARY KEY
-CREATE TABLE type_e_3_2 (a int, b int, c int, PRIMARY KEY ((a, b))) WITH COMPACT STORAGE;
-CREATE TABLE type_e_4_2 (a int, b int, c int, d int, PRIMARY KEY ((a, b))) WITH COMPACT STORAGE;
-CREATE TABLE type_e_4_3 (a int, b int, c int, d int, PRIMARY KEY ((a, b, c))) WITH COMPACT STORAGE;
-
--- type F: compound PK, multipart partition key, all key components used in partitioning,
--- dynamic storage
-CREATE TABLE type_f_2_2 (a int, b int, PRIMARY KEY ((a, b)));
-CREATE TABLE type_f_3_2 (a int, b int, c int, PRIMARY KEY ((a, b)));
-CREATE TABLE type_f_4_2 (a int, b int, c int, d int, PRIMARY KEY ((a, b)));
-CREATE TABLE type_f_4_3 (a int, b int, c int, d int, PRIMARY KEY ((a, b, c)));
-
--- type G: compound PK, multipart partition key, not all key components used in partitioning,
--- compact storage
-CREATE TABLE type_g_3_3_2 (a int, b int, c int, PRIMARY KEY ((a, b), c)) WITH COMPACT STORAGE;
-CREATE TABLE type_g_4_3_2 (a int, b int, c int, d int, PRIMARY KEY ((a, b), c)) WITH COMPACT STORAGE;
-CREATE TABLE type_g_4_4_2 (a int, b int, c int, d int, PRIMARY KEY ((a, b), c, d)) WITH COMPACT STORAGE;
-CREATE TABLE type_g_4_4_3 (a int, b int, c int, d int, PRIMARY KEY ((a, b, c), d)) WITH COMPACT STORAGE;
--- FAILS: CREATE TABLE type_g_5_3_2 (a int, b int, c int, d int, e int, PRIMARY KEY ((a, b), c)) WITH COMPACT STORAGE;
--- Bad Request: COMPACT STORAGE with composite PRIMARY KEY allows no more than one column not part of the PRIMARY KEY (got: d, e)
-CREATE TABLE type_g_5_4_2 (a int, b int, c int, d int, e int, PRIMARY KEY ((a, b), c, d)) WITH COMPACT STORAGE;
-
--- type H: compound PK, multipart partition key, not all key components used in partitioning,
--- dynamic storage
-CREATE TABLE type_h_3_3_2 (a int, b int, c int, PRIMARY KEY ((a, b), c)); 
-CREATE TABLE type_h_4_3_2 (a int, b int, c int, d int, PRIMARY KEY ((a, b), c));
-CREATE TABLE type_h_4_4_2 (a int, b int, c int, d int, PRIMARY KEY ((a, b), c, d));
-CREATE TABLE type_h_4_4_3 (a int, b int, c int, d int, PRIMARY KEY ((a, b, c), d));
-CREATE TABLE type_h_5_3_2 (a int, b int, c int, d int, e int, PRIMARY KEY ((a, b), c));
-CREATE TABLE type_h_5_4_2 (a int, b int, c int, d int, e int, PRIMARY KEY ((a, b), c, d));
-
--- type A with collections (these should fail, but don't)
-CREATE TABLE type_aa_2_2 (a int PRIMARY KEY, b map<float, text>) WITH COMPACT STORAGE;
-CREATE TABLE type_aa_3_2 (a int PRIMARY KEY, b map<float, text>, c int) WITH COMPACT STORAGE;
-
--- type B with collections
-CREATE TABLE type_bb_2_2 (a int PRIMARY KEY, b map<float, text>);
-CREATE TABLE type_bb_3_2 (a int PRIMARY KEY, b map<float, text>, c int);
-CREATE TABLE type_bb_3_23 (a int PRIMARY KEY, b map<float, text>, c set<int>);
-
--- type C with collections
--- FAILS: CREATE TABLE type_cc_4_3_2 (a int, b map<float, text>, c int, d int, PRIMARY KEY (a, b, c)) WITH COMPACT STORAGE;
--- Bad Request: Invalid collection type for PRIMARY KEY component b
--- FAILS: CREATE TABLE type_cc_4_3_4 (a int, b int, c int, d map<float, text>, PRIMARY KEY (a, b, c)) WITH COMPACT STORAGE;
--- Bad Request: Collection types are not supported with COMPACT STORAGE
-
--- type D with collections
--- FAILS: CREATE TABLE type_dd_3_2_1 (a map<float, text>, b int, c int, PRIMARY KEY (a, b));
--- Bad Request: Invalid collection type for PRIMARY KEY component a
--- FAILS: CREATE TABLE type_dd_3_2_2 (a int, b map<float, text>, c int, PRIMARY KEY (a, b));
--- Bad Request: Invalid collection type for PRIMARY KEY component b
-CREATE TABLE type_dd_3_2_3 (a int, b int, c map<float, text>, PRIMARY KEY (a, b));
-CREATE TABLE type_dd_4_3_4 (a int, b int, c int, d map<float, text>, PRIMARY KEY (a, b, c));
-CREATE TABLE type_dd_5_3_4 (a int, b int, c int, d map<float, text>, e int, PRIMARY KEY (a, b, c));
-CREATE TABLE type_dd_5_3_45 (a int, b int, c int, d map<float, text>, e list<int>, PRIMARY KEY (a, b, c));
-
--- type E with collections (these should all fail, but some don't)
--- FAILS: CREATE TABLE type_ee_3_2_2 (a int, b map<float, text>, c int, PRIMARY KEY ((a, b))) WITH COMPACT STORAGE;
--- Bad Request: Invalid collection type for PRIMARY KEY component b
-CREATE TABLE type_ee_3_2_3 (a int, b int, c map<float, text>, PRIMARY KEY ((a, b))) WITH COMPACT STORAGE;
-CREATE TABLE type_ee_4_3_4 (a int, b int, c int, d map<float, text>, PRIMARY KEY ((a, b, c))) WITH COMPACT STORAGE;
-CREATE TABLE type_ee_5_3_45 (a int, b int, c int, d map<float, text>, e list<int>, PRIMARY KEY ((a, b, c))) WITH COMPACT STORAGE;
-
--- type F with collections
--- FAILS: CREATE TABLE type_ff_3_2_1 (a list<int>, b int, c int, PRIMARY KEY ((a, b)));
--- Bad Request: Invalid collection type for PRIMARY KEY component a
-CREATE TABLE type_ff_3_2_3 (a int, b int, c map<float, text>, PRIMARY KEY ((a, b)));
-CREATE TABLE type_ff_4_3_4 (a int, b int, c int, d map<float, text>, PRIMARY KEY ((a, b, c)));
-CREATE TABLE type_ff_5_2_45 (a int, b int, c int, d map<float, text>, e list<int>, PRIMARY KEY ((a, b)));
-CREATE TABLE type_ff_5_3_45 (a int, b int, c int, d map<float, text>, e list<int>, PRIMARY KEY ((a, b, c)));
-
--- type G with collections
--- FAILS: CREATE TABLE type_gg_4_3_2_1 (a set<int>, b int, c int, d int, PRIMARY KEY ((a, b), c)) WITH COMPACT STORAGE;
--- Bad Request: Invalid collection type for PRIMARY KEY component a
--- FAILS: CREATE TABLE type_gg_4_3_2_4 (a int, b int, c int, d list<int>, PRIMARY KEY ((a, b), c)) WITH COMPACT STORAGE;
--- Bad Request: Collection types are not supported with COMPACT STORAGE
--- FAILS: CREATE TABLE type_gg_5_3_2_4 (a int, b int, c int, d map<float, text>, e list<int>, PRIMARY KEY ((a, b), c)) WITH COMPACT STORAGE;
--- Bad Request: Collection types are not supported with COMPACT STORAGE
-
--- type H with collections
--- FAILS: CREATE TABLE type_hh_4_3_2_1 (a set<int>, b int, c int, d int, PRIMARY KEY ((a, b), c));
--- Bad Request: Invalid collection type for PRIMARY KEY component a
--- FAILS: CREATE TABLE type_hh_4_3_2_3 (a int, b int, c list<int>, d int, PRIMARY KEY ((a, b), c));
--- Bad Request: Invalid collection type for PRIMARY KEY component c
-CREATE TABLE type_hh_4_3_2_4 (a int, b int, c int, d list<int>, PRIMARY KEY ((a, b), c));
-CREATE TABLE type_hh_5_3_2_45 (a int, b int, c int, d map<float, text>, e list<int>, PRIMARY KEY ((a, b), c));