You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by br...@apache.org on 2012/08/31 14:43:50 UTC

git commit: cqlsh: update encoding for collections Patch by paul cannon reviewed by brandonwilliams for CASSANDRA-4500

Updated Branches:
  refs/heads/trunk 39fdebfd4 -> bd2752721


cqlsh: update encoding for collections
Patch by paul cannon reviewed by brandonwilliams for CASSANDRA-4500


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

Branch: refs/heads/trunk
Commit: bd27527211c630f886e665bb4d0b76be78769edd
Parents: 39fdebf
Author: Brandon Williams <br...@apache.org>
Authored: Fri Aug 31 07:42:31 2012 -0500
Committer: Brandon Williams <br...@apache.org>
Committed: Fri Aug 31 07:42:31 2012 -0500

----------------------------------------------------------------------
 bin/cqlsh                       |  254 +++++++---------------------------
 lib/cql-internal-only-1.1.0.zip |  Bin 0 -> 78637 bytes
 pylib/cqlshlib/cql3handling.py  |    7 +-
 pylib/cqlshlib/cqlhandling.py   |   42 +-----
 pylib/cqlshlib/displaying.py    |  107 ++++++++++++++
 pylib/cqlshlib/formatting.py    |  192 +++++++++++++++++++++++++
 pylib/cqlshlib/util.py          |    5 +
 7 files changed, 361 insertions(+), 246 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/bd275272/bin/cqlsh
----------------------------------------------------------------------
diff --git a/bin/cqlsh b/bin/cqlsh
index 7ea0128..24217ae 100755
--- a/bin/cqlsh
+++ b/bin/cqlsh
@@ -39,7 +39,6 @@ from itertools import groupby
 from contextlib import contextmanager, closing
 from glob import glob
 from functools import partial
-from collections import defaultdict
 
 import cmd
 import sys
@@ -49,19 +48,10 @@ import optparse
 import ConfigParser
 import codecs
 import locale
-import re
 import platform
 import warnings
 import csv
 
-# 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(os.path.realpath(__file__)), '..', 'pylib')
-if os.path.isdir(cqlshlibdir):
-    sys.path.insert(0, cqlshlibdir)
-
-from cqlshlib import cqlhandling, cql3handling, pylexotron, wcwidth
-
 try:
     import readline
 except ImportError:
@@ -105,7 +95,20 @@ except ImportError, e:
 
 import cql.decoders
 from cql.cursor import _COUNT_DESCRIPTION, _VOID_DESCRIPTION
+from cql.cqltypes import (cql_types, cql_typename, lookup_casstype, lookup_cqltype,
+                          CassandraType)
 
+# 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(os.path.realpath(__file__)), '..', 'pylib')
+if os.path.isdir(cqlshlibdir):
+    sys.path.insert(0, cqlshlibdir)
+
+from cqlshlib import cqlhandling, cql3handling, pylexotron
+from cqlshlib.displaying import (RED, BLUE, ANSI_RESET, COLUMN_NAME_COLORS,
+                                 FormattedValue, colorme)
+from cqlshlib.formatting import format_by_type
+from cqlshlib.util import trim_if_present
 
 CONFIG_FILE = os.path.expanduser(os.path.join('~', '.cqlshrc'))
 HISTORY = os.path.expanduser(os.path.join('~', '.cqlsh_history'))
@@ -139,16 +142,6 @@ parser.add_option("-3", "--cql3", action="store_const", dest='cqlversion', const
                   help="Shortcut notation for --cqlversion=3")
 
 
-RED = '\033[0;1;31m'
-GREEN = '\033[0;1;32m'
-YELLOW = '\033[0;1;33m'
-BLUE = '\033[0;1;34m'
-MAGENTA = '\033[0;1;35m'
-CYAN = '\033[0;1;36m'
-WHITE = '\033[0;1;37m'
-DARK_MAGENTA = '\033[0;35m'
-ANSI_RESET = '\033[0m'
-
 CQL_ERRORS = (cql.Error,)
 try:
     from thrift.Thrift import TException
@@ -361,159 +354,18 @@ def full_cql_version(ver):
     vertuple = tuple(map(int, ver_parts[0].split('.')) + [ver_parts[1]])
     return ver, vertuple
 
-def trim_if_present(s, prefix):
-    if s.startswith(prefix):
-        return s[len(prefix):]
-    return s
-
-class FormattedValue:
-    def __init__(self, strval, coloredval, displaywidth):
-        self.strval = strval
-        self.coloredval = coloredval
-        self.displaywidth = displaywidth
-
-    def __len__(self):
-        return len(self.strval)
-
-    def _pad(self, width, fill=' '):
-        if width > self.displaywidth:
-            return fill * (width - self.displaywidth)
-        else:
-            return ''
-
-    def ljust(self, width, fill=' '):
-        """
-        Similar to self.strval.ljust(width), but takes expected terminal
-        display width into account for special characters, and does not
-        take color escape codes into account.
-        """
-        return self.strval + self._pad(width, fill)
-
-    def rjust(self, width, fill=' '):
-        """
-        Similar to self.strval.rjust(width), but takes expected terminal
-        display width into account for special characters, and does not
-        take color escape codes into account.
-        """
-        return self._pad(width, fill) + self.strval
-
-    def color_rjust(self, width, fill=' '):
-        """
-        Similar to self.rjust(width), but uses this value's colored
-        representation, and does not take color escape codes into account
-        in determining width.
-        """
-        return self._pad(width, fill) + self.coloredval
-
-    def color_ljust(self, width, fill=' '):
-        """
-        Similar to self.ljust(width), but uses this value's colored
-        representation, and does not take color escape codes into account
-        in determining width.
-        """
-        return self.coloredval + self._pad(width, fill)
-
-unicode_controlchars_re = re.compile(r'[\x00-\x31\x7f-\xa0]')
-controlchars_re = re.compile(r'[\x00-\x31\x7f-\xff]')
-
-def _show_control_chars(match):
-    txt = repr(match.group(0))
-    if txt.startswith('u'):
-        txt = txt[2:-1]
-    else:
-        txt = txt[1:-1]
-    return txt
-
-bits_to_turn_red_re = re.compile(r'\\([^uUx]|u[0-9a-fA-F]{4}|x[0-9a-fA-F]{2}|U[0-9a-fA-F]{8})')
-
-def _make_turn_bits_red_f(color1, color2):
-    def _turn_bits_red(match):
-        txt = match.group(0)
-        if txt == '\\\\':
-            return '\\'
-        return color1 + txt + color2
-    return _turn_bits_red
-
-DEFAULT_VALUE_COLORS = dict(
-    default=YELLOW,
-    text=YELLOW,
-    error=RED,
-    hex=DARK_MAGENTA,
-    timestamp=GREEN,
-    int=GREEN,
-    float=GREEN,
-    decimal=GREEN,
-    boolean=GREEN,
-    uuid=GREEN,
-)
-
-COLUMN_NAME_COLORS = defaultdict(lambda: MAGENTA,
-    error=RED,
-    hex=DARK_MAGENTA,
-)
-
-def unix_time_from_uuid1(u):
-    return (u.get_time() - 0x01B21DD213814000) / 10000000.0
-
-def format_value(val, casstype, output_encoding, addcolor=False, time_format='',
-                 float_precision=3, colormap=DEFAULT_VALUE_COLORS, nullval='null'):
-    casstype = trim_if_present(casstype, 'org.apache.cassandra.db.marshal.')
-    color = colormap['default']
-    coloredval = None
-    displaywidth = None
-
-    if val is None:
-        bval = nullval
-        color = colormap['error']
-    elif isinstance(val, DecodeError):
-        casstype = 'BytesType'
-        bval = repr(val.thebytes)
-        color = colormap['hex']
-    elif casstype == 'UTF8Type':
-        escapedval = val.replace(u'\\', u'\\\\')
-        escapedval = unicode_controlchars_re.sub(_show_control_chars, escapedval)
-        bval = escapedval.encode(output_encoding, 'backslashreplace')
-        displaywidth = wcwidth.wcswidth(bval.decode(output_encoding))
+def format_value(val, typeclass, output_encoding, addcolor=False, time_format=None,
+                 float_precision=None, colormap=None, nullval=None):
+    if isinstance(val, DecodeError):
         if addcolor:
-            tbr = _make_turn_bits_red_f(colormap['hex'], colormap['text'])
-            coloredval = colormap['text'] + bits_to_turn_red_re.sub(tbr, bval) + ANSI_RESET
-    elif casstype in ('DateType', 'TimeUUIDType'):
-        if casstype == 'TimeUUIDType':
-            val = unix_time_from_uuid1(val)
-        timestamp = time.localtime(val)
-        bval = time.strftime(time_format, timestamp)
-        color = colormap['timestamp']
-    elif casstype in ('LongType', 'Int32Type', 'IntegerType', 'CounterColumnType'):
-        # base-10 only for now; support others?
-        bval = str(val)
-        color = colormap['int']
-    elif casstype in ('FloatType', 'DoubleType'):
-        bval = '%.*g' % (float_precision, val)
-        color = colormap['float']
-    elif casstype in ('DecimalType', 'UUIDType', 'BooleanType'):
-        # let python do these for us
-        bval = str(val)
-        color = colormap[cqlruleset.cql_typename(casstype)]
-    elif casstype == 'BytesType':
-        bval = ''.join('%02x' % ord(c) for c in val)
-        color = colormap['hex']
-    else:
-        # AsciiType is the only other one known right now, but handle others
-        val = str(val)
-        escapedval = val.replace('\\', '\\\\')
-        bval = controlchars_re.sub(_show_control_chars, escapedval)
-        if addcolor:
-            tbr = _make_turn_bits_red_f(colormap['hex'], colormap['text'])
-            coloredval = colormap['text'] + bits_to_turn_red_re.sub(tbr, bval) + ANSI_RESET
-
-    if displaywidth is None:
-        displaywidth = len(bval)
-    if not addcolor:
-        coloredval = bval
-    elif coloredval is None:
-        coloredval = color + bval + ANSI_RESET
-
-    return FormattedValue(bval, coloredval, displaywidth)
+            return colorme(val.thebytes, colormap, 'hex')
+        else:
+            return FormattedValue(val.thebytes)
+    if not issubclass(typeclass, CassandraType):
+        typeclass = lookup_casstype(typeclass)
+    return format_by_type(typeclass, val, output_encoding, colormap=colormap,
+                          addcolor=addcolor, nullval=nullval, time_format=time_format,
+                          float_precision=float_precision)
 
 def show_warning_without_quoting_line(message, category, filename, lineno, file=None, line=None):
     if file is None:
@@ -657,14 +509,14 @@ class Shell(cmd.Cmd):
                 cf = self.cql_protect_name(cf)
                 if override.default_name_type:
                     print 'ASSUME %s NAMES ARE %s;' \
-                          % (cf, cqlruleset.cql_typename(override.default_name_type))
+                          % (cf, cql_typename(override.default_name_type))
                 if override.default_value_type:
                     print 'ASSUME %s VALUES ARE %s;' \
-                          % (cf, cqlruleset.cql_typename(override.default_value_type))
+                          % (cf, cql_typename(override.default_value_type))
                 for colname, vtype in override.value_types.items():
                     colname = self.cql_protect_name(colname)
                     print 'ASSUME %s(%s) VALUES ARE %s;' \
-                          % (cf, colname, cqlruleset.cql_typename(vtype))
+                          % (cf, colname, cql_typename(vtype))
         print
 
     def get_cluster_versions(self):
@@ -1059,15 +911,7 @@ class Shell(cmd.Cmd):
         This is necessary to differentiate some things like ascii vs. blob hex.
         """
 
-        if getattr(cursor, 'supports_name_info', False):
-            return cursor.name_info[num][1]
-
-        # This is a pretty big hack, but necessary until we can rely on
-        # python-cql 1.0.10 being around.
-        row = cursor.result[cursor.rs_idx - 1]
-        col = row.columns[num]
-        schema = cursor.decoder.schema
-        return schema.name_types.get(col.name, schema.default_name_type)
+        return cursor.name_info[num][1]
 
     def print_count_result(self, cursor):
         if not cursor.result:
@@ -1077,21 +921,21 @@ class Shell(cmd.Cmd):
         self.writeresult(cursor.result[0])
         self.writeresult("")
 
-    def print_result(self, cursor):
-        self.decoding_errors = []
-
-        # first pass: see if we have a static column set
+    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:
-                static = False
-                break
+                return False
             last_description = cursor.description
-        else:
-            static = True
         cursor._reset()
+        return True
+
+    def print_result(self, cursor):
+        self.decoding_errors = []
 
-        if static:
+        if self.has_static_result_set(cursor):
             self.print_static_result(cursor)
         else:
             self.print_dynamic_result(cursor)
@@ -1105,10 +949,10 @@ class Shell(cmd.Cmd):
                                  % (len(self.decoding_errors) - 2), color=RED)
 
     def print_static_result(self, cursor):
-        colnames, coltypes = zip(*cursor.description)[:2]
+        colnames = [d[0] for d in cursor.description]
         colnames_t = [(name, self.get_nametype(cursor, n)) for (n, name) in enumerate(colnames)]
         formatted_names = [self.myformat_colname(name, nametype) for (name, nametype) in colnames_t]
-        formatted_data = [map(self.myformat_value, row, coltypes) for row in cursor]
+        formatted_data = [map(self.myformat_value, row, cursor.column_types) for row in cursor]
 
         # determine column widths
         widths = [n.displaywidth for n in formatted_names]
@@ -1128,10 +972,10 @@ class Shell(cmd.Cmd):
 
     def print_dynamic_result(self, cursor):
         for row in cursor:
-            colnames, coltypes = zip(*cursor.description)[:2]
+            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 = [self.myformat_value(val, casstype) for (val, casstype) in zip(row, coltypes)]
+            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)
 
@@ -1234,12 +1078,12 @@ class Shell(cmd.Cmd):
         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 = cqlruleset.cql_typename(cfdef.key_validation_class)
+        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, cqlruleset.cql_typename(col.validation_class)))
+            out.write(",\n  %s %s" % (colname, cql_typename(col.validation_class)))
             if col.index_name is not None:
                 indexed_columns.append(col)
         cf_opts = []
@@ -1248,7 +1092,7 @@ class Shell(cmd.Cmd):
             if optval is None:
                 continue
             if option in ('comparator', 'default_validation'):
-                optval = cqlruleset.cql_typename(optval)
+                optval = cql_typename(optval)
             else:
                 if option == 'row_cache_provider':
                     optval = trim_if_present(optval, 'org.apache.cassandra.cache.')
@@ -1612,11 +1456,11 @@ class Shell(cmd.Cmd):
                 row = self.cursor.fetchone()
                 if row is None:
                     break
-                fmt = lambda v, d: \
-                    format_value(v, d[1], output_encoding=encoding, nullval=nullval,
+                fmt = lambda v, t: \
+                    format_value(v, t, output_encoding=encoding, nullval=nullval,
                                  time_format=self.display_time_format,
                                  float_precision=self.display_float_precision).strval
-                writer.writerow(map(fmt, row, self.cursor.description))
+                writer.writerow(map(fmt, row, self.cursor.column_types))
                 rows += 1
         finally:
             if do_close:
@@ -1726,7 +1570,7 @@ class Shell(cmd.Cmd):
             if cqltype is None:
                 continue
             try:
-                validator_class = cqlruleset.find_validator_class(cqltype)
+                validator_class = lookup_cqltype(cqltype).cass_parameterized_type()
             except KeyError:
                 self.printerr('Error: validator type %s not found.' % cqltype)
             else:
@@ -1868,7 +1712,7 @@ class Shell(cmd.Cmd):
 
     def help_types(self):
         print "\n        CQL types recognized by this version of cqlsh:\n"
-        for t in cqlruleset.cql_types:
+        for t in cql_types:
             print '          ' + t
         print """
         For information on the various recognizable input formats for these

http://git-wip-us.apache.org/repos/asf/cassandra/blob/bd275272/lib/cql-internal-only-1.1.0.zip
----------------------------------------------------------------------
diff --git a/lib/cql-internal-only-1.1.0.zip b/lib/cql-internal-only-1.1.0.zip
new file mode 100644
index 0000000..52b1e61
Binary files /dev/null and b/lib/cql-internal-only-1.1.0.zip differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/bd275272/pylib/cqlshlib/cql3handling.py
----------------------------------------------------------------------
diff --git a/pylib/cqlshlib/cql3handling.py b/pylib/cqlshlib/cql3handling.py
index 63a5d0c..107d443 100644
--- a/pylib/cqlshlib/cql3handling.py
+++ b/pylib/cqlshlib/cql3handling.py
@@ -17,6 +17,7 @@
 import re
 from warnings import warn
 from .cqlhandling import CqlParsingRuleSet, Hint
+from cql.cqltypes import cql_types, cql_typename
 
 try:
     import json
@@ -131,7 +132,7 @@ CqlRuleSet = Cql3ParsingRuleSet()
 shorthands = ('completer_for', 'explain_completion',
               'dequote_value', 'dequote_name',
               'escape_value', 'escape_name',
-              'maybe_escape_name', 'cql_typename')
+              'maybe_escape_name')
 
 for shorthand in shorthands:
     globals()[shorthand] = getattr(CqlRuleSet, shorthand)
@@ -250,7 +251,7 @@ def token_word_completer(ctxt, cass):
 
 @completer_for('storageType', 'typename')
 def storagetype_completer(ctxt, cass):
-    return CqlRuleSet.cql_types
+    return cql_types
 
 @completer_for('keyspaceName', 'ksname')
 def ks_name_completer(ctxt, cass):
@@ -611,7 +612,7 @@ def create_cf_option_val_completer(ctxt, cass):
     if any(this_opt == opt[0] for opt in CqlRuleSet.obsolete_cf_options):
         return ["'<obsolete_option>'"]
     if this_opt in ('comparator', 'default_validation'):
-        return CqlRuleSet.cql_types
+        return cql_types
     if this_opt in ('read_repair_chance', 'bloom_filter_fp_chance'):
         return [Hint('<float_between_0_and_1>')]
     if this_opt == 'replicate_on_write':

http://git-wip-us.apache.org/repos/asf/cassandra/blob/bd275272/pylib/cqlshlib/cqlhandling.py
----------------------------------------------------------------------
diff --git a/pylib/cqlshlib/cqlhandling.py b/pylib/cqlshlib/cqlhandling.py
index 715f6f9..06f029f 100644
--- a/pylib/cqlshlib/cqlhandling.py
+++ b/pylib/cqlshlib/cqlhandling.py
@@ -20,6 +20,7 @@
 import re
 import traceback
 from . import pylexotron, util
+from cql import cqltypes
 
 Hint = pylexotron.Hint
 
@@ -77,27 +78,6 @@ class CqlParsingRuleSet(pylexotron.ParsingRuleSet):
         'SizeTieredCompactionStrategy'
     )
 
-    cql_type_to_apache_class = {
-        'ascii': 'AsciiType',
-        'bigint': 'LongType',
-        'blob': 'BytesType',
-        'boolean': 'BooleanType',
-        'counter': 'CounterColumnType',
-        'decimal': 'DecimalType',
-        'double': 'DoubleType',
-        'float': 'FloatType',
-        'int': 'Int32Type',
-        'text': 'UTF8Type',
-        'timestamp': 'DateType',
-        'uuid': 'UUIDType',
-        'varchar': 'UTF8Type',
-        'varint': 'IntegerType'
-    }
-
-    apache_class_to_cql_type = dict((v,k) for (k,v) in cql_type_to_apache_class.items())
-
-    cql_types = sorted(cql_type_to_apache_class.keys())
-
     replication_strategies = (
         'SimpleStrategy',
         'OldNetworkTopologyStrategy',
@@ -362,20 +342,6 @@ class CqlParsingRuleSet(pylexotron.ParsingRuleSet):
         return self.cql_complete_multiple(text, first, init_bindings, startsymbol=startsymbol)
 
     @classmethod
-    def cql_typename(cls, classname):
-        fq_classname = 'org.apache.cassandra.db.marshal.'
-        if classname.startswith(fq_classname):
-            classname = classname[len(fq_classname):]
-        try:
-            return cls.apache_class_to_cql_type[classname]
-        except KeyError:
-            return cls.escape_value(classname)
-
-    @classmethod
-    def find_validator_class(cls, cqlname):
-        return cls.cql_type_to_apache_class[cqlname]
-
-    @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
 
@@ -410,7 +376,7 @@ class CqlParsingRuleSet(pylexotron.ParsingRuleSet):
     @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 cls.cql_typename(col_info[0].validation_class) == 'counter')
+        return bool(col_info and cqltypes.is_counter_type(col_info[0].validation_class))
 
     @staticmethod
     def cql2_dequote_value(cqlword):
@@ -543,7 +509,7 @@ def cl_completer(ctxt, cass):
 
 @completer_for('storageType', 'typename')
 def storagetype_completer(ctxt, cass):
-    return CqlRuleSet.cql_types
+    return cqltypes.cql_types
 
 @completer_for('keyspaceName', 'ksname')
 def ks_name_completer(ctxt, cass):
@@ -850,7 +816,7 @@ def create_cf_option_val_completer(ctxt, cass):
     if any(this_opt == opt[0] for opt in CqlRuleSet.obsolete_cf_options):
         return ["'<obsolete_option>'"]
     if this_opt in ('comparator', 'default_validation'):
-        return CqlRuleSet.cql_types
+        return cqltypes.cql_types
     if this_opt == 'read_repair_chance':
         return [Hint('<float_between_0_and_1>')]
     if this_opt == 'replicate_on_write':

http://git-wip-us.apache.org/repos/asf/cassandra/blob/bd275272/pylib/cqlshlib/displaying.py
----------------------------------------------------------------------
diff --git a/pylib/cqlshlib/displaying.py b/pylib/cqlshlib/displaying.py
new file mode 100644
index 0000000..adeaefd
--- /dev/null
+++ b/pylib/cqlshlib/displaying.py
@@ -0,0 +1,107 @@
+# 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.
+
+import re
+from collections import defaultdict
+
+RED = '\033[0;1;31m'
+GREEN = '\033[0;1;32m'
+YELLOW = '\033[0;1;33m'
+BLUE = '\033[0;1;34m'
+MAGENTA = '\033[0;1;35m'
+CYAN = '\033[0;1;36m'
+WHITE = '\033[0;1;37m'
+DARK_MAGENTA = '\033[0;35m'
+ANSI_RESET = '\033[0m'
+
+def colorme(bval, colormap, colorkey):
+    if colormap is None:
+        colormap = DEFAULT_VALUE_COLORS
+    return FormattedValue(bval, colormap[colorkey] + bval + colormap['reset'])
+
+class FormattedValue:
+    def __init__(self, strval, coloredval=None, displaywidth=None):
+        self.strval = strval
+        if coloredval is None:
+            coloredval = strval
+        self.coloredval = coloredval
+        if displaywidth is None:
+            displaywidth = len(strval)
+        # displaywidth is useful for display of special unicode characters
+        # with
+        self.displaywidth = displaywidth
+
+    def __len__(self):
+        return len(self.strval)
+
+    def _pad(self, width, fill=' '):
+        if width > self.displaywidth:
+            return fill * (width - self.displaywidth)
+        else:
+            return ''
+
+    def ljust(self, width, fill=' '):
+        """
+        Similar to self.strval.ljust(width), but takes expected terminal
+        display width into account for special characters, and does not
+        take color escape codes into account.
+        """
+        return self.strval + self._pad(width, fill)
+
+    def rjust(self, width, fill=' '):
+        """
+        Similar to self.strval.rjust(width), but takes expected terminal
+        display width into account for special characters, and does not
+        take color escape codes into account.
+        """
+        return self._pad(width, fill) + self.strval
+
+    def color_rjust(self, width, fill=' '):
+        """
+        Similar to self.rjust(width), but uses this value's colored
+        representation, and does not take color escape codes into account
+        in determining width.
+        """
+        return self._pad(width, fill) + self.coloredval
+
+    def color_ljust(self, width, fill=' '):
+        """
+        Similar to self.ljust(width), but uses this value's colored
+        representation, and does not take color escape codes into account
+        in determining width.
+        """
+        return self.coloredval + self._pad(width, fill)
+
+DEFAULT_VALUE_COLORS = dict(
+    default=YELLOW,
+    text=YELLOW,
+    error=RED,
+    hex=DARK_MAGENTA,
+    timestamp=GREEN,
+    int=GREEN,
+    float=GREEN,
+    decimal=GREEN,
+    boolean=GREEN,
+    uuid=GREEN,
+    collection=BLUE,
+    reset=ANSI_RESET,
+)
+
+COLUMN_NAME_COLORS = defaultdict(lambda: MAGENTA,
+    error=RED,
+    hex=DARK_MAGENTA,
+    reset=ANSI_RESET,
+)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/bd275272/pylib/cqlshlib/formatting.py
----------------------------------------------------------------------
diff --git a/pylib/cqlshlib/formatting.py b/pylib/cqlshlib/formatting.py
new file mode 100644
index 0000000..a5a0fd2
--- /dev/null
+++ b/pylib/cqlshlib/formatting.py
@@ -0,0 +1,192 @@
+# 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.
+
+import re
+import time
+from collections import defaultdict
+from . import wcwidth
+from .displaying import colorme, FormattedValue, DEFAULT_VALUE_COLORS
+from cql import cqltypes
+
+unicode_controlchars_re = re.compile(r'[\x00-\x31\x7f-\xa0]')
+controlchars_re = re.compile(r'[\x00-\x31\x7f-\xff]')
+
+def _show_control_chars(match):
+    txt = repr(match.group(0))
+    if txt.startswith('u'):
+        txt = txt[2:-1]
+    else:
+        txt = txt[1:-1]
+    return txt
+
+bits_to_turn_red_re = re.compile(r'\\([^uUx]|u[0-9a-fA-F]{4}|x[0-9a-fA-F]{2}|U[0-9a-fA-F]{8})')
+
+def _make_turn_bits_red_f(color1, color2):
+    def _turn_bits_red(match):
+        txt = match.group(0)
+        if txt == '\\\\':
+            return '\\'
+        return color1 + txt + color2
+    return _turn_bits_red
+
+default_null_placeholder = 'null'
+default_time_format = ''
+default_float_precision = 3
+default_colormap = DEFAULT_VALUE_COLORS
+empty_colormap = defaultdict(lambda: '')
+
+def format_by_type(cqltype, val, encoding, colormap=None, addcolor=False,
+                   nullval=None, time_format=None, float_precision=None):
+    if nullval is None:
+        nullval = default_null_placeholder
+    if val is None:
+        return colorme(nullval, colormap, 'error')
+    if addcolor is False:
+        colormap = empty_colormap
+    elif colormap is None:
+        colormap = default_colormap
+    if time_format is None:
+        time_format = default_time_format
+    if float_precision is None:
+        float_precision = default_float_precision
+    return format_value(cqltype, val, encoding=encoding, colormap=colormap,
+                        time_format=time_format, float_precision=float_precision,
+                        nullval=nullval)
+
+def format_value_default(val, colormap, **_):
+    val = str(val)
+    escapedval = val.replace('\\', '\\\\')
+    bval = controlchars_re.sub(_show_control_chars, escapedval)
+    tbr = _make_turn_bits_red_f(colormap['hex'], colormap['text'])
+    coloredval = colormap['text'] + bits_to_turn_red_re.sub(tbr, bval) + colormap['reset']
+    return FormattedValue(bval, coloredval)
+
+# Mapping cql type base names ("int", "map", etc) to formatter functions,
+# making format_value a generic function
+_formatters = {}
+
+def format_value(cqltype, val, **kwargs):
+    formatter = _formatters.get(cqltype.typename, format_value_default)
+    return formatter(val, subtypes=cqltype.subtypes, **kwargs)
+
+def formatter_for(typname):
+    def registrator(f):
+        _formatters[typname] = f
+        return f
+    return registrator
+
+@formatter_for('bytes')
+def format_value_bytes(val, colormap, **_):
+    bval = ''.join('%02x' % ord(c) for c in val)
+    return colorme(bval, colormap, 'hex')
+
+def format_python_formatted_type(val, colormap, color):
+    bval = str(val)
+    return colorme(bval, colormap, color)
+
+@formatter_for('decimal')
+def format_value_decimal(val, colormap, **_):
+    return format_python_formatted_type(val, colormap, 'decimal')
+
+@formatter_for('uuid')
+def format_value_uuid(val, colormap, **_):
+    return format_python_formatted_type(val, colormap, 'uuid')
+
+@formatter_for('boolean')
+def format_value_boolean(val, colormap, **_):
+    return format_python_formatted_type(val, colormap, 'boolean')
+
+def format_floating_point_type(val, colormap, float_precision, **_):
+    bval = '%.*g' % (float_precision, val)
+    return colorme(bval, colormap, 'float')
+
+formatter_for('float')(format_floating_point_type)
+formatter_for('double')(format_floating_point_type)
+
+def format_integer_type(val, colormap, **_):
+    # base-10 only for now; support others?
+    bval = str(val)
+    return colorme(bval, colormap, 'int')
+
+formatter_for('bigint')(format_integer_type)
+formatter_for('int')(format_integer_type)
+formatter_for('varint')(format_integer_type)
+formatter_for('counter')(format_integer_type)
+
+@formatter_for('timestamp')
+def format_value_timestamp(val, colormap, time_format, **_):
+    bval = time.strftime(time_format, time.localtime(val))
+    return colorme(bval, colormap, 'timestamp')
+
+@formatter_for('timeuuid')
+def format_value_timeuuid(val, colormap, time_format, **_):
+    utime = unix_time_from_uuid1(val)
+    bval = time.strftime(time_format, time.localtime(utime))
+    return colorme(bval, colormap, 'timestamp')
+
+@formatter_for('text')
+def format_value_text(val, encoding, colormap, **_):
+    escapedval = val.replace(u'\\', u'\\\\')
+    escapedval = unicode_controlchars_re.sub(_show_control_chars, escapedval)
+    bval = escapedval.encode(encoding, 'backslashreplace')
+    displaywidth = wcwidth.wcswidth(bval.decode(encoding))
+    tbr = _make_turn_bits_red_f(colormap['hex'], colormap['text'])
+    coloredval = colormap['text'] + bits_to_turn_red_re.sub(tbr, bval) + colormap['reset']
+    return FormattedValue(bval, coloredval)
+
+# name alias
+formatter_for('varchar')(format_value_text)
+
+def format_simple_collection(subtype, val, lbracket, rbracket, encoding,
+                             colormap, time_format, float_precision, nullval):
+    subs = [format_value(subtype, sval, encoding=encoding, colormap=colormap,
+                         time_format=time_format, float_precision=float_precision,
+                         nullval=nullval)
+            for sval in val]
+    bval = lbracket + ', '.join(sval.strval for sval in subs) + rbracket
+    lb, sep, rb = [colormap['collection'] + s + colormap['reset']
+                   for s in (lbracket, ', ', rbracket)]
+    coloredval = lb + sep.join(sval.coloredval for sval in subs) + rb
+    displaywidth = 2 * len(subs) + sum(sval.displaywidth for sval in subs)
+    return FormattedValue(bval, coloredval, displaywidth)
+
+@formatter_for('list')
+def format_value_list(val, encoding, colormap, time_format, float_precision, subtypes, nullval, **_):
+    return format_simple_collection(subtypes[0], val, '[', ']', encoding, colormap,
+                                    time_format, float_precision, nullval)
+
+@formatter_for('set')
+def format_value_set(val, encoding, colormap, time_format, float_precision, subtypes, nullval, **_):
+    return format_simple_collection(subtypes[0], val, '{', '}', encoding, colormap,
+                                    time_format, float_precision, nullval)
+
+@formatter_for('map')
+def format_value_map(val, encoding, colormap, time_format, float_precision, subtypes, nullval, **_):
+    def subformat(v, subtype):
+        return format_value(subtype, v, encoding=encoding, colormap=colormap,
+                            time_format=time_format, float_precision=float_precision,
+                            nullval=nullval)
+
+    subkeytype, subvaltype = subtypes
+    subs = [(subformat(k, subkeytype), subformat(v, subvaltype)) for (k, v) in val.items()]
+    bval = '{' + ', '.join(k.strval + ': ' + v.strval for (k, v) in subs) + '}'
+    lb, comma, colon, rb = [colormap['collection'] + s + colormap['reset']
+                            for s in ('{', ', ', ': ', '}')]
+    coloredval = lb \
+               + comma.join(k.coloredval + colon + v.coloredval for (k, v) in subs) \
+               + rb
+    displaywidth = 4 * len(subs) + sum(k.displaywidth + v.displaywidth for (k, v) in subs)
+    return FormattedValue(bval, coloredval, displaywidth)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/bd275272/pylib/cqlshlib/util.py
----------------------------------------------------------------------
diff --git a/pylib/cqlshlib/util.py b/pylib/cqlshlib/util.py
index ea0fbf4..e62ded4 100644
--- a/pylib/cqlshlib/util.py
+++ b/pylib/cqlshlib/util.py
@@ -72,3 +72,8 @@ def list_bifilter(pred, iterable):
 
 def identity(x):
     return x
+
+def trim_if_present(s, prefix):
+    if s.startswith(prefix):
+        return s[len(prefix):]
+    return s