You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by mi...@apache.org on 2014/02/14 19:57:25 UTC

git commit: User types support for cqlsh.

Updated Branches:
  refs/heads/trunk 4b08be49a -> e5199eeac


User types support for cqlsh.

Patch by Mikhail Stepura; reviewed by Aleksey Yeschenko for CASSANDRA-6305


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

Branch: refs/heads/trunk
Commit: e5199eeac1e5a5cb1c402e146955ae26cf595f96
Parents: 4b08be4
Author: Mikhail Stepura <mi...@apache.org>
Authored: Mon Jan 6 18:51:50 2014 -0800
Committer: Mikhail Stepura <mi...@apache.org>
Committed: Fri Feb 14 10:55:32 2014 -0800

----------------------------------------------------------------------
 bin/cqlsh                      | 101 ++++++++++++++++++++++++++++++++--
 pylib/cqlshlib/cql3handling.py | 105 +++++++++++++++++++++++++++++++++---
 pylib/cqlshlib/formatting.py   |  20 ++++++-
 pylib/cqlshlib/usertypes.py    |  63 ++++++++++++++++++++++
 4 files changed, 279 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5199eea/bin/cqlsh
----------------------------------------------------------------------
diff --git a/bin/cqlsh b/bin/cqlsh
index 0add46c..427a0a2 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 = "4.1.1"
+version = "4.1.2"
 
 from StringIO import StringIO
 from itertools import groupby
@@ -112,7 +112,7 @@ cqlshlibdir = os.path.join(CASSANDRA_PATH, 'pylib')
 if os.path.isdir(cqlshlibdir):
     sys.path.insert(0, cqlshlibdir)
 
-from cqlshlib import cqlhandling, cql3handling, pylexotron
+from cqlshlib import cqlhandling, cql3handling, pylexotron, usertypes
 from cqlshlib.displaying import (RED, BLUE, ANSI_RESET, COLUMN_NAME_COLORS,
                                  FormattedValue, colorme)
 from cqlshlib.formatting import format_by_type
@@ -230,7 +230,9 @@ cqlsh_extra_syntax_rules = r'''
                                   | ( "COLUMNFAMILY" | "TABLE" ) cf=<columnFamilyName>
                                   | ( "COLUMNFAMILIES" | "TABLES" )
                                   | "FULL"? "SCHEMA"
-                                  | "CLUSTER" )
+                                  | "CLUSTER"
+                                  | "TYPES"
+                                  | "TYPE" ut=<userTypeName>)
                     ;
 
 <consistencyCommand> ::= "CONSISTENCY" ( level=<consistencyLevel> )?
@@ -366,6 +368,9 @@ class ColumnFamilyNotFound(Exception):
 class VersionNotSupported(Exception):
     pass
 
+class UserTypeNotFound(Exception):
+    pass
+
 class DecodeError(Exception):
     verb = 'decode'
 
@@ -455,6 +460,7 @@ class Shell(cmd.Cmd):
     shunted_query_out = None
     csv_dialect_defaults = dict(delimiter=',', doublequote=False,
                                 escapechar='\\', quotechar='"')
+    utypes_meta = None
 
     def __init__(self, hostname, port, transport_factory, color=False,
                  username=None, password=None, encoding=None, stdin=None, tty=True,
@@ -514,6 +520,9 @@ class Shell(cmd.Cmd):
             self.show_line_nums = True
         self.stdin = stdin
         self.query_out = sys.stdout
+        self.refresh_utypes_meta()
+        #UserTypes will use it to get theirs field names
+        usertypes.get_field_names = lambda k, t: self.utypes_meta.get_field_names(k, t)
 
     def set_expanded_cql_version(self, ver):
         ver, vertuple = full_cql_version(ver)
@@ -626,6 +635,21 @@ class Shell(cmd.Cmd):
         layout = self.get_columnfamily_layout(ksname, cfname)
         return [col.name for col in layout.columns]
 
+    def get_usertype_names(self, ksname=None):
+        if ksname is None:
+            ksname = self.current_keyspace
+
+        return self.utypes_meta.get_usertypes_names(ksname)
+
+    def get_usertype_layout(self, ksname, typename):
+        if ksname is None:
+            ksname = self.current_keyspace
+        layout = self.utypes_meta.get_fields_with_types(ksname, typename)
+
+        if not layout:
+            raise UserTypeNotFound("User type %r not found" % typename)
+        return layout
+
     # ===== thrift-dependent parts =====
 
     def get_cluster_name(self):
@@ -693,6 +717,15 @@ class Shell(cmd.Cmd):
         cols = self.fetchdict_all()
         return cql3handling.CqlTableDef.from_layout(layout, cols)
 
+    def build_usertypes_meta(self):
+        ut_q = """select * from system.schema_usertypes"""
+        self.cursor.execute(ut_q, consistency_level='ONE')
+        data = self.fetchdict_all()
+        if not data:
+            return None
+
+        return cql3handling.UserTypesMeta.from_layout(data)
+
     # ===== end cql3-dependent parts =====
 
     def reset_statement(self):
@@ -755,6 +788,15 @@ class Shell(cmd.Cmd):
                 return
             yield newline
 
+    def refresh_utypes_meta(self):
+        try:
+            self.utypes_meta = self.build_usertypes_meta()
+        except Exception, e:
+            if self.debug:
+                import traceback
+                traceback.print_exc()
+
+
     def cmdloop(self):
         """
         Adapted from cmd.Cmd's version, because there is literally no way with
@@ -926,6 +968,8 @@ class Shell(cmd.Cmd):
             self.writeresult("")
             self.print_static_result(self.cursor)
         self.flush_output()
+        if self.could_change_schema(statement):
+            self.refresh_utypes_meta()
         return True
 
     def get_nametype(self, cursor, num):
@@ -1082,6 +1126,14 @@ class Shell(cmd.Cmd):
             out.write(" AND durable_writes = 'false'")
         out.write(';\n')
 
+        #print usertypes first
+        uts = self.get_usertype_names(ksname)
+        if uts:
+            out.write('\nUSE %s;\n' % ksname)
+            for ut in uts:
+                out.write('\n')
+                self.print_recreate_usertype(ksdef.name, ut, out)
+
         cfs = self.get_columnfamily_names(ksname)
         if cfs:
             out.write('\nUSE %s;\n' % ksname)
@@ -1090,6 +1142,16 @@ class Shell(cmd.Cmd):
                 # yes, cf might be looked up again. oh well.
                 self.print_recreate_columnfamily(ksdef.name, cf, out)
 
+    def print_recreate_usertype(self, ksname, utname, out):
+        layout = self.get_usertype_layout(ksname, utname)
+        out.write("CREATE TYPE %s (\n" % utname)
+        for (index,(tname,ttype)) in enumerate(layout, 1):
+            out.write("  %s %s" % (tname, ttype))
+            if index < len(layout):
+                out.write(",\n")
+            else:
+                out.write("\n); \n")
+
     def print_recreate_columnfamily(self, ksname, cfname, out):
         """
         Output CQL commands which should be pasteable back into a CQL session
@@ -1099,6 +1161,7 @@ class Shell(cmd.Cmd):
         """
         layout = self.get_columnfamily_layout(ksname, cfname)
         cfname = self.cql_protect_name(layout.name)
+
         out.write("CREATE TABLE %s (\n" % cfname)
 
         for col in layout.columns:
@@ -1238,6 +1301,26 @@ class Shell(cmd.Cmd):
             cmd.Cmd.columnize(self, self.get_columnfamily_names(ksname))
             print
 
+    def describe_usertypes(self, ksname):
+        print
+        if ksname is None:
+            for k in self.get_keyspaces():
+                name = self.cql_protect_name(k.name)
+                print 'Keyspace %s' % (name,)
+                print '---------%s' % ('-' * len(name))
+                cmd.Cmd.columnize(self, self.get_usertype_names(k.name))
+                print
+        else:
+            cmd.Cmd.columnize(self, self.get_usertype_names(ksname))
+            print
+
+    def describe_usertype(self, ksname, typename):
+        if ksname is None:
+            ksname = self.current_keyspace
+        print
+        self.print_recreate_usertype(ksname, typename, sys.stdout)
+        print
+
     def describe_cluster(self):
         print '\nCluster: %s' % self.get_cluster_name()
         p = trim_if_present(self.get_partitioner(), 'org.apache.cassandra.dht.')
@@ -1323,6 +1406,12 @@ class Shell(cmd.Cmd):
             self.describe_columnfamily(ks, cf)
         elif what in ('columnfamilies', 'tables'):
             self.describe_columnfamilies(self.current_keyspace)
+        elif what == 'types':
+            self.describe_usertypes(self.current_keyspace)
+        elif what == 'type':
+            ks = self.cql_unprotect_name(parsed.get_binding('ksname', None))
+            ut = self.cql_unprotect_name(parsed.get_binding('utname'))
+            self.describe_usertype(ks, ut)
         elif what == 'cluster':
             self.describe_cluster()
         elif what == 'schema':
@@ -1850,6 +1939,12 @@ class Shell(cmd.Cmd):
             text = '%s:%d:%s' % (self.stdin.name, self.lineno, text)
         self.writeresult(text, color, newline=newline, out=sys.stderr)
 
+    @staticmethod
+    def could_change_schema(statement):
+        import re
+        return re.match(r"(alter|create|drop)\s+type", statement.lower())
+
+
 class ErrorHandlingSchemaDecoder(cql.decoders.SchemaDecoder):
     def name_decode_error(self, err, namebytes, expectedtype):
         return DecodeError(namebytes, err, expectedtype)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5199eea/pylib/cqlshlib/cql3handling.py
----------------------------------------------------------------------
diff --git a/pylib/cqlshlib/cql3handling.py b/pylib/cqlshlib/cql3handling.py
index cf26e6e..093354a 100644
--- a/pylib/cqlshlib/cql3handling.py
+++ b/pylib/cqlshlib/cql3handling.py
@@ -245,11 +245,14 @@ JUNK ::= /([ \t\r\f\v]+|(--|[/][/])[^\n\r]*([\n\r]|$)|[/][*].*?[*][/])/ ;
 <schemaChangeStatement> ::= <createKeyspaceStatement>
                           | <createColumnFamilyStatement>
                           | <createIndexStatement>
+                          | <createUserTypeStatement>
                           | <dropKeyspaceStatement>
                           | <dropColumnFamilyStatement>
                           | <dropIndexStatement>
+                          | <dropUserTypeStatement>
                           | <alterTableStatement>
                           | <alterKeyspaceStatement>
+                          | <alterUserTypeStatement>
                           ;
 
 <authenticationStatement> ::= <createUserStatement>
@@ -266,15 +269,19 @@ JUNK ::= /([ \t\r\f\v]+|(--|[/][/])[^\n\r]*([\n\r]|$)|[/][*].*?[*][/])/ ;
 # timestamp is included here, since it's also a keyword
 <simpleStorageType> ::= typename=( <identifier> | <stringLiteral> | <K_TIMESTAMP> ) ;
 
-<storageType> ::= <simpleStorageType> | <collectionType> ;
+<userType> ::= utname=<cfOrKsName> ;
 
-<collectionType> ::= "map" "<" <simpleStorageType> "," <simpleStorageType> ">"
-                   | "list" "<" <simpleStorageType> ">"
-                   | "set" "<" <simpleStorageType> ">"
+<storageType> ::= <simpleStorageType> | <collectionType> | <userType> ;
+
+<collectionType> ::= "map" "<" <simpleStorageType> "," ( <simpleStorageType> | <userType> ) ">"
+                   | "list" "<" ( <simpleStorageType> | <userType> ) ">"
+                   | "set" "<" ( <simpleStorageType> | <userType> ) ">"
                    ;
 
 <columnFamilyName> ::= ( ksname=<cfOrKsName> dot="." )? cfname=<cfOrKsName> ;
 
+<userTypeName> ::= ( ksname=<cfOrKsName> dot="." )? utname=<cfOrKsName> ;
+
 <keyspaceName> ::= ksname=<cfOrKsName> ;
 
 <nonSystemKeyspaceName> ::= ksname=<cfOrKsName> ;
@@ -508,17 +515,19 @@ def ks_name_completer(ctxt, cass):
     ksnames = [n for n in cass.get_keyspace_names() if n not in NONALTERBALE_KEYSPACES]
     return map(maybe_escape_name, ksnames)
 
-@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', 'dot')
+completer_for('columnFamilyName', 'ksname')(cf_ks_name_completer)
+
 def cf_ks_dot_completer(ctxt, cass):
     name = dequote_name(ctxt.get_binding('ksname'))
     if name in cass.get_keyspace_names():
         return ['.']
     return []
 
+completer_for('columnFamilyName', 'dot')(cf_ks_dot_completer)
+
 @completer_for('columnFamilyName', 'cfname')
 def cf_name_completer(ctxt, cass):
     ks = ctxt.get_binding('ksname', None)
@@ -532,6 +541,26 @@ def cf_name_completer(ctxt, cass):
         raise
     return map(maybe_escape_name, cfnames)
 
+completer_for('userTypeName', 'ksname')(cf_ks_name_completer)
+
+completer_for('userTypeName', 'dot')(cf_ks_dot_completer)
+
+def ut_name_completer(ctxt, cass):
+    ks = ctxt.get_binding('ksname', None)
+    if ks is not None:
+        ks = dequote_name(ks)
+    try:
+        utnames = cass.get_usertype_names(ks)
+    except Exception:
+        if ks is None:
+            return ()
+        raise
+    return map(maybe_escape_name, utnames)
+
+
+completer_for('userTypeName', 'utname')(ut_name_completer)
+completer_for('userType', 'utname')(ut_name_completer)
+
 @completer_for('unreservedKeyword', 'nocomplete')
 def unreserved_keyword_completer(ctxt, cass):
     # we never want to provide completions through this production;
@@ -546,6 +575,13 @@ def get_cf_layout(ctxt, cass):
     cf = dequote_name(ctxt.get_binding('cfname'))
     return cass.get_columnfamily_layout(ks, cf)
 
+def get_ut_layout(ctxt, cass):
+    ks = ctxt.get_binding('ksname', None)
+    if ks is not None:
+        ks = dequote_name(ks)
+    ut = dequote_name(ctxt.get_binding('utname'))
+    return cass.get_usertype_layout(ks, ut)
+
 def working_on_keyspace(ctxt):
     wat = ctxt.get_binding('wat').upper()
     if wat in ('KEYSPACE', 'SCHEMA'):
@@ -930,9 +966,16 @@ syntax_rules += r'''
                                cf=<columnFamilyName> "(" col=<cident> ")"
                                ( "USING" <stringLiteral> ( "WITH" "OPTIONS" "=" <mapLiteral> )? )?
                          ;
+
+<createUserTypeStatement> ::= "CREATE" "TYPE" ( ks=<nonSystemKeyspaceName> dot="." )? typename=<cfOrKsName> "(" newcol=<cident> <storageType>
+                                ( "," [newcolname]=<cident> <storageType> )*
+                            ")"
+                         ;
 '''
 
 explain_completion('createIndexStatement', 'indexname', '<new_index_name>')
+explain_completion('createUserTypeStatement', 'typename', '<new_type_name>')
+explain_completion('createUserTypeStatement', 'newcol', '<new_field_name>')
 
 @completer_for('createIndexStatement', 'col')
 def create_index_col_completer(ctxt, cass):
@@ -949,6 +992,10 @@ syntax_rules += r'''
 
 <dropIndexStatement> ::= "DROP" "INDEX" ("IF" "EXISTS")? indexname=<identifier>
                        ;
+
+<dropUserTypeStatement> ::= "DROP" "TYPE" ut=<userTypeName>
+                              ;
+
 '''
 
 @completer_for('dropIndexStatement', 'indexname')
@@ -966,6 +1013,15 @@ syntax_rules += r'''
                       | "RENAME" existcol=<cident> "TO" newcol=<cident>
                          ( "AND" existcol=<cident> "TO" newcol=<cident> )*
                       ;
+
+<alterUserTypeStatement> ::= "ALTER" "TYPE" ut=<userTypeName>
+                               <alterTypeInstructions>
+                             ;
+<alterTypeInstructions> ::= "RENAME" "TO" typename=<cfOrKsName>
+                           | "ALTER" existcol=<cident> "TYPE" <storageType>
+                           | "ADD" newcol=<cident> <storageType>
+                           | "RENAME" existcol=<cident> "TO" newcol=<cident>
+                           ;
 '''
 
 @completer_for('alterInstructions', 'existcol')
@@ -974,7 +1030,16 @@ def alter_table_col_completer(ctxt, cass):
     cols = [md.name for md in layout.columns]
     return map(maybe_escape_name, cols)
 
+@completer_for('alterTypeInstructions', 'existcol')
+def alter_type_field_completer(ctxt, cass):
+    layout = get_ut_layout(ctxt, cass)
+    fields = [tuple[0] for tuple in layout]
+    return map(maybe_escape_name, fields)
+
 explain_completion('alterInstructions', 'newcol', '<new_column_name>')
+explain_completion('alterTypeInstructions', 'typename', '<new_type_name>')
+explain_completion('alterTypeInstructions', 'newcol', '<new_field_name>')
+
 
 syntax_rules += r'''
 <alterKeyspaceStatement> ::= "ALTER" ( "KEYSPACE" | "SCHEMA" ) ks=<alterableKeyspaceName>
@@ -1170,3 +1235,31 @@ class CqlTableDef:
     def __str__(self):
         return '<%s %s.%s>' % (self.__class__.__name__, self.keyspace, self.name)
     __repr__ = __str__
+
+class UserTypesMeta(object):
+    _meta = {}
+
+    def __init__(self, meta):
+        self._meta = meta
+
+    @classmethod
+    def from_layout(cls, layout):
+        result = {}
+        for row in layout:
+            ksname = row[u'keyspace_name']
+            if ksname not in result:
+                result[ksname] = {}
+            utname = row[u'type_name']
+
+            result[ksname][utname] = zip(row[u'column_names'], row[u'column_types'])
+        return cls(meta=result)
+
+    def get_usertypes_names(self, keyspace):
+        return map(str, self._meta.get(keyspace, {}).keys())
+
+    def get_field_names(self, keyspace, type):
+        return [row[0] for row in self._meta.get(keyspace, {}).get(type, [])]
+
+    def get_fields_with_types(self, ksname, typename):
+        return [(field[0], lookup_casstype(field[1]).cql_parameterized_type()) for field in
+                self._meta.get(ksname, {}).get(typename, [])]
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5199eea/pylib/cqlshlib/formatting.py
----------------------------------------------------------------------
diff --git a/pylib/cqlshlib/formatting.py b/pylib/cqlshlib/formatting.py
index fff75c2..7a0ef17 100644
--- a/pylib/cqlshlib/formatting.py
+++ b/pylib/cqlshlib/formatting.py
@@ -21,7 +21,6 @@ import math
 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]')
@@ -238,3 +237,22 @@ def format_value_map(val, encoding, colormap, time_format, float_precision, subt
                + rb
     displaywidth = 4 * len(subs) + sum(k.displaywidth + v.displaywidth for (k, v) in subs)
     return FormattedValue(bval, coloredval, displaywidth)
+
+def format_value_utype(val, encoding, colormap, time_format, float_precision, subtypes, nullval, **_):
+    def format_field_value(v, subtype):
+        return format_value(subtype, v, encoding=encoding, colormap=colormap,
+                            time_format=time_format, float_precision=float_precision,
+                            nullval=nullval, quote=True)
+
+    def format_field_name(name):
+        return format_value_text(name, encoding=encoding, colormap=colormap, quote=False)
+
+    subs = [(format_field_name(k), format_field_value(v, subtypes[index])) for (index, (k, v)) in enumerate(val)]
+    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)
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5199eea/pylib/cqlshlib/usertypes.py
----------------------------------------------------------------------
diff --git a/pylib/cqlshlib/usertypes.py b/pylib/cqlshlib/usertypes.py
new file mode 100644
index 0000000..078016b
--- /dev/null
+++ b/pylib/cqlshlib/usertypes.py
@@ -0,0 +1,63 @@
+# 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.
+
+from cql.marshal import uint16_unpack
+from cql.cqltypes import CompositeType
+from formatting import formatter_for, format_value_utype
+
+
+def get_field_names(ks_name, ut_name):
+    """
+    UserTypes will use this function to get its fields names from Shell's utypes_dict
+    """
+    raise NotImplementedError("this function shall be overloaded by Shell")
+
+
+class UserType(CompositeType):
+    typename = 'UserType'
+
+    @classmethod
+    def apply_parameters(cls, *subtypes):
+        ksname = subtypes[0].cassname
+        newname = subtypes[1].cassname.decode("hex")
+        field_names = get_field_names(ksname, newname)
+        assert len(field_names) == len(subtypes[2:])
+        formatter_for(newname)(format_value_utype)
+        return type(newname, (cls,), {'subtypes': subtypes[2:],
+                                      'cassname': cls.cassname, 'typename': newname, 'fieldnames': field_names})
+
+    @classmethod
+    def cql_parameterized_type(cls):
+        return cls.typename
+
+    @classmethod
+    def deserialize_safe(cls, byts):
+        p = 0
+        result = []
+        for col_name, col_type in zip(cls.fieldnames, cls.subtypes):
+            if p == len(byts):
+                break
+            itemlen = uint16_unpack(byts[p:p + 2])
+            p += 2
+            item = byts[p:p + itemlen]
+            p += itemlen
+            result.append((str(col_name), col_type.from_binary(item)))
+            p += 1
+
+        return result
+
+
+