Return-Path: X-Original-To: apmail-cassandra-commits-archive@www.apache.org Delivered-To: apmail-cassandra-commits-archive@www.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id C332D18401 for ; Fri, 11 Sep 2015 14:48:56 +0000 (UTC) Received: (qmail 76704 invoked by uid 500); 11 Sep 2015 14:48:53 -0000 Delivered-To: apmail-cassandra-commits-archive@cassandra.apache.org Received: (qmail 76609 invoked by uid 500); 11 Sep 2015 14:48:53 -0000 Mailing-List: contact commits-help@cassandra.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@cassandra.apache.org Delivered-To: mailing list commits@cassandra.apache.org Received: (qmail 76302 invoked by uid 99); 11 Sep 2015 14:48:53 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Fri, 11 Sep 2015 14:48:53 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 32965E00C5; Fri, 11 Sep 2015 14:48:53 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: slebresne@apache.org To: commits@cassandra.apache.org Date: Fri, 11 Sep 2015 14:48:55 -0000 Message-Id: <86ab16ffe67f4e5e9670cf9b9a316687@git.apache.org> In-Reply-To: <17ce2e4686384ab09992a4f7b0b3be4c@git.apache.org> References: <17ce2e4686384ab09992a4f7b0b3be4c@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: [3/4] cassandra git commit: Make cqlsh PEP8 compliant (and other cleanups) - 2.2 patch Make cqlsh PEP8 compliant (and other cleanups) - 2.2 patch patch by philipthompson; reviewed by Stefania for CASSANDRA-10066 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/72f2cc4b Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/72f2cc4b Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/72f2cc4b Branch: refs/heads/cassandra-3.0 Commit: 72f2cc4b175978548756919b4d3f95cfa2885435 Parents: 765e3bb Author: Philip Thompson Authored: Mon Aug 31 14:48:50 2015 -0400 Committer: Sylvain Lebresne Committed: Fri Sep 11 16:47:18 2015 +0200 ---------------------------------------------------------------------- CHANGES.txt | 1 + bin/cqlsh.py | 128 +++++++------- pylib/cqlshlib/__init__.py | 1 - pylib/cqlshlib/cql3handling.py | 83 ++++++++- pylib/cqlshlib/cqlhandling.py | 10 +- pylib/cqlshlib/displaying.py | 12 +- pylib/cqlshlib/formatting.py | 42 +++-- pylib/cqlshlib/helptopics.py | 6 +- pylib/cqlshlib/pylexotron.py | 55 +++++- pylib/cqlshlib/saferscanner.py | 8 +- pylib/cqlshlib/sslhandling.py | 4 +- pylib/cqlshlib/util.py | 1 + pylib/cqlshlib/wcwidth.py | 326 ++++++++++++++++++------------------ 13 files changed, 414 insertions(+), 263 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/cassandra/blob/72f2cc4b/CHANGES.txt ---------------------------------------------------------------------- diff --git a/CHANGES.txt b/CHANGES.txt index b927f95..cd29592 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -9,6 +9,7 @@ * Fall back to 1/4 commitlog volume for commitlog_total_space on small disks (CASSANDRA-10199) Merged from 2.1: + * (cqlsh) Make cqlsh PEP8 Compliant (CASSANDRA-10066) * (cqlsh) Fix error when starting cqlsh with --debug (CASSANDRA-10282) * Scrub, Cleanup and Upgrade do not unmark compacting until all operations have completed, regardless of the occurence of exceptions (CASSANDRA-10274) http://git-wip-us.apache.org/repos/asf/cassandra/blob/72f2cc4b/bin/cqlsh.py ---------------------------------------------------------------------- diff --git a/bin/cqlsh.py b/bin/cqlsh.py index 4fce833..2636a59 100644 --- a/bin/cqlsh.py +++ b/bin/cqlsh.py @@ -30,30 +30,29 @@ exit 1 ":""" from __future__ import with_statement -from uuid import UUID - -description = "CQL Shell for Apache Cassandra" -version = "5.0.1" - -from StringIO import StringIO -from contextlib import contextmanager -from glob import glob import cmd -import sys -import os -import time -import optparse -import ConfigParser import codecs +import ConfigParser +import csv +import getpass import locale +import multiprocessing +import optparse +import os import platform +import sys +import time +import traceback import warnings -import csv -import getpass +from contextlib import contextmanager from functools import partial -import traceback +from glob import glob +from StringIO import StringIO +from uuid import UUID +description = "CQL Shell for Apache Cassandra" +version = "5.0.1" readline = None try: @@ -108,12 +107,14 @@ except ImportError, e: 'Module load path: %r\n\n' 'Error: %s\n' % (sys.executable, sys.path, e)) +from cassandra.auth import PlainTextAuthProvider from cassandra.cluster import Cluster, PagedResult -from cassandra.query import SimpleStatement, ordered_dict_factory +from cassandra.metadata import (ColumnMetadata, KeyspaceMetadata, + TableMetadata, protect_name, protect_names, + protect_value) from cassandra.policies import WhiteListRoundRobinPolicy from cassandra.protocol import QueryMessage, ResultMessage -from cassandra.metadata import protect_name, protect_names, protect_value, KeyspaceMetadata, TableMetadata, ColumnMetadata -from cassandra.auth import PlainTextAuthProvider +from cassandra.query import SimpleStatement, ordered_dict_factory # cqlsh should run correctly when run out of a Cassandra source tree, # out of an unpacked Cassandra tarball, and after a proper package install. @@ -121,16 +122,15 @@ cqlshlibdir = os.path.join(CASSANDRA_PATH, 'pylib') if os.path.isdir(cqlshlibdir): sys.path.insert(0, cqlshlibdir) -from cqlshlib import cqlhandling, cql3handling, pylexotron, sslhandling -from cqlshlib.displaying import (RED, BLUE, CYAN, ANSI_RESET, COLUMN_NAME_COLORS, - FormattedValue, colorme) -from cqlshlib.formatting import format_by_type, formatter_for, format_value_utype -from cqlshlib.util import trim_if_present, get_file_encoding_bomsize -from cqlshlib.formatting import DateTimeFormat -from cqlshlib.formatting import DEFAULT_TIMESTAMP_FORMAT -from cqlshlib.formatting import DEFAULT_DATE_FORMAT -from cqlshlib.formatting import DEFAULT_NANOTIME_FORMAT -from cqlshlib.tracing import print_trace_session, print_trace +from cqlshlib import cql3handling, cqlhandling, pylexotron, sslhandling +from cqlshlib.displaying import (ANSI_RESET, BLUE, COLUMN_NAME_COLORS, CYAN, + RED, FormattedValue, colorme) +from cqlshlib.formatting import (DEFAULT_DATE_FORMAT, DEFAULT_NANOTIME_FORMAT, + DEFAULT_TIMESTAMP_FORMAT, DateTimeFormat, + format_by_type, format_value_utype, + formatter_for) +from cqlshlib.tracing import print_trace, print_trace_session +from cqlshlib.util import get_file_encoding_bomsize, trim_if_present DEFAULT_HOST = '127.0.0.1' DEFAULT_PORT = 9042 @@ -182,7 +182,7 @@ parser.add_option("--connect-timeout", default=DEFAULT_CONNECT_TIMEOUT_SECONDS, optvalues = optparse.Values() (options, arguments) = parser.parse_args(sys.argv[1:], values=optvalues) -#BEGIN history/config definition +# BEGIN history/config definition HISTORY_DIR = os.path.expanduser(os.path.join('~', '.cassandra')) if hasattr(options, 'cqlshrc'): @@ -213,7 +213,7 @@ if os.path.exists(OLD_CONFIG_FILE): OLD_HISTORY = os.path.expanduser(os.path.join('~', '.cqlsh_history')) if os.path.exists(OLD_HISTORY): os.rename(OLD_HISTORY, HISTORY) -#END history/config definition +# END history/config definition CQL_ERRORS = ( cassandra.AlreadyExists, cassandra.AuthenticationFailed, cassandra.InvalidRequest, @@ -454,12 +454,15 @@ class KeyspaceNotFound(Exception): class ColumnFamilyNotFound(Exception): pass + class IndexNotFound(Exception): pass + class ObjectNotFound(Exception): pass + class VersionNotSupported(Exception): pass @@ -467,9 +470,11 @@ class VersionNotSupported(Exception): class UserTypeNotFound(Exception): pass + class FunctionNotFound(Exception): pass + class AggregateNotFound(Exception): pass @@ -586,6 +591,7 @@ class FrozenType(cassandra.cqltypes._ParameterizedType): subtype, = cls.subtypes return subtype.to_binary(val, protocol_version) + class Shell(cmd.Cmd): custom_prompt = os.getenv('CQLSH_PROMPT', '') if custom_prompt is not '': @@ -695,7 +701,7 @@ class Shell(cmd.Cmd): self.stdin = stdin self.query_out = sys.stdout self.consistency_level = cassandra.ConsistencyLevel.ONE - self.serial_consistency_level = cassandra.ConsistencyLevel.SERIAL; + self.serial_consistency_level = cassandra.ConsistencyLevel.SERIAL # the python driver returns BLOBs as string, but we expect them as bytearrays cassandra.cqltypes.BytesType.deserialize = staticmethod(lambda byts, protocol_version: bytearray(byts)) cassandra.cqltypes.CassandraType.support_empty_values = True @@ -708,7 +714,7 @@ class Shell(cmd.Cmd): def refresh_schema_metadata_best_effort(self): try: - self.conn.refresh_schema_metadata(5) #will throw exception if there is a schema mismatch + self.conn.refresh_schema_metadata(5) # will throw exception if there is a schema mismatch except Exception: self.printerr("Warning: schema version mismatch detected, which might be caused by DOWN nodes; if " "this is not the case, check the schema versions of your nodes in system.local and " @@ -753,9 +759,9 @@ class Shell(cmd.Cmd): def show_host(self): print "Connected to %s at %s:%d." % \ - (self.applycolor(self.get_cluster_name(), BLUE), - self.hostname, - self.port) + (self.applycolor(self.get_cluster_name(), BLUE), + self.hostname, + self.port) def show_version(self): vers = self.connection_versions.copy() @@ -837,7 +843,7 @@ class Shell(cmd.Cmd): return self.conn.metadata.partitioner def get_keyspace_meta(self, ksname): - if not ksname in self.conn.metadata.keyspaces: + if ksname not in self.conn.metadata.keyspaces: raise KeyspaceNotFound('Keyspace %r not found.' % ksname) return self.conn.metadata.keyspaces[ksname] @@ -1283,7 +1289,7 @@ class Shell(cmd.Cmd): def print_warnings(self, warnings): if warnings is None or len(warnings) == 0: - return; + return self.writeresult('') self.writeresult('Warnings :') @@ -1502,10 +1508,9 @@ class Shell(cmd.Cmd): p = trim_if_present(self.get_partitioner(), 'org.apache.cassandra.dht.') print 'Partitioner: %s\n' % p # TODO: snitch? - #snitch = trim_if_present(self.get_snitch(), 'org.apache.cassandra.locator.') - #print 'Snitch: %s\n' % snitch - if self.current_keyspace is not None \ - and self.current_keyspace != 'system': + # snitch = trim_if_present(self.get_snitch(), 'org.apache.cassandra.locator.') + # print 'Snitch: %s\n' % snitch + if self.current_keyspace is not None and self.current_keyspace != 'system': print "Range ownership:" ring = self.get_ring() for entry in ring.items(): @@ -1515,7 +1520,7 @@ class Shell(cmd.Cmd): def describe_schema(self, include_system=False): print for k in self.get_keyspaces(): - if include_system or not k.name in cql3handling.SYSTEM_KEYSPACES: + if include_system or k.name not in cql3handling.SYSTEM_KEYSPACES: self.print_recreate_keyspace(k, sys.stdout) print @@ -1592,7 +1597,7 @@ class Shell(cmd.Cmd): Output CQL commands that could be used to recreate the entire object schema, where object can be either a keyspace or a table or an index (in this order). - """ + """ what = parsed.matched[1][1].lower() if what == 'functions': ksname = self.cql_unprotect_name(parsed.get_binding('ksname', None)) @@ -1747,7 +1752,7 @@ class Shell(cmd.Cmd): linesource.next() reader = csv.reader(linesource, **dialect_options) - from multiprocessing import Process, Pipe, cpu_count + from multiprocessing import Pipe, cpu_count # Pick a resonable number of child processes. We need to leave at # least one core for the parent process. This doesn't necessarily @@ -2185,7 +2190,7 @@ class Shell(cmd.Cmd): Clears the console. """ import subprocess - subprocess.call(['clear','cls'][myplatform == 'Windows'], shell=True) + subprocess.call(['clear', 'cls'][myplatform == 'Windows'], shell=True) do_cls = do_clear def do_debug(self, parsed): @@ -2266,8 +2271,9 @@ class Shell(cmd.Cmd): text = '%s:%d:%s' % (self.stdin.name, self.lineno, text) self.writeresult(text, color, newline=newline, out=sys.stderr) -import multiprocessing + class ImportProcess(multiprocessing.Process): + def __init__(self, parent, pipe, ks, cf, columns, nullval): multiprocessing.Process.__init__(self) self.pipe = pipe @@ -2275,7 +2281,7 @@ class ImportProcess(multiprocessing.Process): self.ks = ks self.cf = cf - #validate we can fetch metdata but don't store it since win32 needs to pickle + # validate we can fetch metdata but don't store it since win32 needs to pickle parent.get_table_meta(ks, cf) self.columns = columns @@ -2290,15 +2296,15 @@ class ImportProcess(multiprocessing.Process): def run(self): new_cluster = Cluster( - contact_points=(self.hostname,), - port=self.port, - cql_version=self.cql_version, - protocol_version=DEFAULT_PROTOCOL_VERSION, - auth_provider=self.auth_provider, - ssl_options=sslhandling.ssl_settings(hostname, CONFIG_FILE) if self.ssl else None, - load_balancing_policy=WhiteListRoundRobinPolicy([self.hostname]), - compression=None, - connect_timeout=self.connect_timeout) + contact_points=(self.hostname,), + port=self.port, + cql_version=self.cql_version, + protocol_version=DEFAULT_PROTOCOL_VERSION, + auth_provider=self.auth_provider, + ssl_options=sslhandling.ssl_settings(self.hostname, CONFIG_FILE) if self.ssl else None, + load_balancing_policy=WhiteListRoundRobinPolicy([self.hostname]), + compression=None, + connect_timeout=self.connect_timeout) session = new_cluster.connect(self.ks) conn = session._pools.values()[0]._connection @@ -2378,8 +2384,8 @@ class ImportProcess(multiprocessing.Process): full_query = query % (','.join(row),) query_message = QueryMessage( - full_query, self.consistency_level, serial_consistency_level=None, - fetch_size=None, paging_state=None, timestamp=insert_timestamp) + full_query, self.consistency_level, serial_consistency_level=None, + fetch_size=None, paging_state=None, timestamp=insert_timestamp) request_id = conn.get_request_id() binary_message = query_message.to_binary( @@ -2413,7 +2419,6 @@ class ImportProcess(multiprocessing.Process): self.terminate() - class RateMeter(object): def __init__(self, log_rate): @@ -2483,6 +2488,7 @@ def option_with_default(cparser_getter, section, option, default=None): except ConfigParser.Error: return default + def raw_option_with_default(configs, section, option, default=None): """ Same (almost) as option_with_default() but won't do any string interpolation. @@ -2529,7 +2535,7 @@ def read_options(cmdlineargs, environment): optvalues.time_format = raw_option_with_default(configs, 'ui', 'time_format', DEFAULT_TIMESTAMP_FORMAT) optvalues.nanotime_format = raw_option_with_default(configs, 'ui', 'nanotime_format', - DEFAULT_NANOTIME_FORMAT) + DEFAULT_NANOTIME_FORMAT) optvalues.date_format = raw_option_with_default(configs, 'ui', 'date_format', DEFAULT_DATE_FORMAT) optvalues.float_precision = option_with_default(configs.getint, 'ui', 'float_precision', http://git-wip-us.apache.org/repos/asf/cassandra/blob/72f2cc4b/pylib/cqlshlib/__init__.py ---------------------------------------------------------------------- diff --git a/pylib/cqlshlib/__init__.py b/pylib/cqlshlib/__init__.py index 588a655..635f0d9 100644 --- a/pylib/cqlshlib/__init__.py +++ b/pylib/cqlshlib/__init__.py @@ -13,4 +13,3 @@ # 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. - http://git-wip-us.apache.org/repos/asf/cassandra/blob/72f2cc4b/pylib/cqlshlib/cql3handling.py ---------------------------------------------------------------------- diff --git a/pylib/cqlshlib/cql3handling.py b/pylib/cqlshlib/cql3handling.py index 7c68001..31b3061 100644 --- a/pylib/cqlshlib/cql3handling.py +++ b/pylib/cqlshlib/cql3handling.py @@ -16,7 +16,6 @@ from .cqlhandling import CqlParsingRuleSet, Hint from cassandra.metadata import maybe_escape_name -from cassandra.metadata import escape_name simple_cql_types = set(('ascii', 'bigint', 'blob', 'boolean', 'counter', 'date', 'decimal', 'double', 'float', 'inet', 'int', @@ -26,7 +25,9 @@ simple_cql_types.difference_update(('set', 'map', 'list')) from . import helptopics cqldocs = helptopics.CQL3HelpTopics() + class UnexpectedTableStructure(UserWarning): + def __init__(self, msg): self.msg = msg @@ -36,6 +37,7 @@ class UnexpectedTableStructure(UserWarning): SYSTEM_KEYSPACES = ('system', 'system_traces', 'system_auth', 'system_distributed') NONALTERBALE_KEYSPACES = ('system') + class Cql3ParsingRuleSet(CqlParsingRuleSet): columnfamily_layout_options = ( @@ -79,7 +81,7 @@ class Cql3ParsingRuleSet(CqlParsingRuleSet): @classmethod def escape_value(cls, value): if value is None: - return 'NULL' # this totally won't work + return 'NULL' # this totally won't work if isinstance(value, bool): value = str(value).lower() elif isinstance(value, float): @@ -318,6 +320,7 @@ JUNK ::= /([ \t\r\f\v]+|(--|[/][/])[^\n\r]*([\n\r]|$)|[/][*].*?[*][/])/ ; ''' + 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 @@ -332,6 +335,7 @@ def prop_equals_completer(ctxt, cass): completer_for('property', 'propeq')(prop_equals_completer) + @completer_for('property', 'propname') def prop_name_completer(ctxt, cass): if working_on_keyspace(ctxt): @@ -339,6 +343,7 @@ def prop_name_completer(ctxt, cass): else: return cf_prop_name_completer(ctxt, cass) + @completer_for('propertyValue', 'propsimpleval') def prop_val_completer(ctxt, cass): if working_on_keyspace(ctxt): @@ -346,6 +351,7 @@ def prop_val_completer(ctxt, cass): else: return cf_prop_val_completer(ctxt, cass) + @completer_for('propertyValue', 'propmapkey') def prop_val_mapkey_completer(ctxt, cass): if working_on_keyspace(ctxt): @@ -353,6 +359,7 @@ def prop_val_mapkey_completer(ctxt, cass): else: return cf_prop_val_mapkey_completer(ctxt, cass) + @completer_for('propertyValue', 'propmapval') def prop_val_mapval_completer(ctxt, cass): if working_on_keyspace(ctxt): @@ -360,6 +367,7 @@ def prop_val_mapval_completer(ctxt, cass): else: return cf_prop_val_mapval_completer(ctxt, cass) + @completer_for('propertyValue', 'ender') def prop_val_mapender_completer(ctxt, cass): if working_on_keyspace(ctxt): @@ -367,12 +375,14 @@ def prop_val_mapender_completer(ctxt, cass): else: return cf_prop_val_mapender_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_prop_val_completer(ctxt, cass): optname = ctxt.get_binding('propname')[-1] if optname == 'durable_writes': @@ -381,6 +391,7 @@ def ks_prop_val_completer(ctxt, cass): return ["{'class': '"] return () + def ks_prop_val_mapkey_completer(ctxt, cass): optname = ctxt.get_binding('propname')[-1] if optname != 'replication': @@ -399,6 +410,7 @@ def ks_prop_val_mapkey_completer(ctxt, cass): return [Hint('')] return map(escape_value, opts.difference(keysseen)) + def ks_prop_val_mapval_completer(ctxt, cass): optname = ctxt.get_binding('propname')[-1] if optname != 'replication': @@ -408,6 +420,7 @@ def ks_prop_val_mapval_completer(ctxt, cass): return map(escape_value, CqlRuleSet.replication_strategies) return [Hint('')] + def ks_prop_val_mapender_completer(ctxt, cass): optname = ctxt.get_binding('propname')[-1] if optname != 'replication': @@ -427,10 +440,12 @@ def ks_prop_val_mapender_completer(ctxt, cass): return [','] return ['}'] + def cf_prop_name_completer(ctxt, cass): return [c[0] for c in (CqlRuleSet.columnfamily_layout_options + CqlRuleSet.columnfamily_layout_map_options)] + def cf_prop_val_completer(ctxt, cass): exist_opts = ctxt.get_binding('propname') this_opt = exist_opts[-1] @@ -450,6 +465,7 @@ def cf_prop_val_completer(ctxt, cass): return [Hint('')] return [Hint('')] + def cf_prop_val_mapkey_completer(ctxt, cass): optname = ctxt.get_binding('propname')[-1] for cql3option, _, subopts in CqlRuleSet.columnfamily_layout_map_options: @@ -486,6 +502,7 @@ def cf_prop_val_mapkey_completer(ctxt, cass): return map(escape_value, opts) return () + def cf_prop_val_mapval_completer(ctxt, cass): opt = ctxt.get_binding('propname')[-1] key = dequote_value(ctxt.get_binding('propmapkey')[-1]) @@ -504,36 +521,44 @@ def cf_prop_val_mapval_completer(ctxt, cass): return ["'ALL'", "'NONE'"] return () + def cf_prop_val_mapender_completer(ctxt, cass): return [',', '}'] + @completer_for('tokenDefinition', 'token') def token_word_completer(ctxt, cass): return ['token('] + @completer_for('simpleStorageType', 'typename') def storagetype_completer(ctxt, cass): return simple_cql_types + @completer_for('keyspaceName', 'ksname') def ks_name_completer(ctxt, cass): return map(maybe_escape_name, cass.get_keyspace_names()) + @completer_for('nonSystemKeyspaceName', 'ksname') def ks_name_completer(ctxt, cass): ksnames = [n for n in cass.get_keyspace_names() if n not in SYSTEM_KEYSPACES] return map(maybe_escape_name, ksnames) + @completer_for('alterableKeyspaceName', 'ksname') 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) + def cf_ks_name_completer(ctxt, cass): return [maybe_escape_name(ks) + '.' for ks in cass.get_keyspace_names()] 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(): @@ -542,6 +567,7 @@ def cf_ks_dot_completer(ctxt, cass): completer_for('columnFamilyName', 'dot')(cf_ks_dot_completer) + @completer_for('columnFamilyName', 'cfname') def cf_name_completer(ctxt, cass): ks = ctxt.get_binding('ksname', None) @@ -559,6 +585,7 @@ 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: @@ -575,6 +602,7 @@ def ut_name_completer(ctxt, cass): 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; @@ -582,6 +610,7 @@ def unreserved_keyword_completer(ctxt, cass): # names, CF names, property values, etc. return () + def get_table_meta(ctxt, cass): ks = ctxt.get_binding('ksname', None) if ks is not None: @@ -589,6 +618,7 @@ def get_table_meta(ctxt, cass): cf = dequote_name(ctxt.get_binding('cfname')) return cass.get_table_meta(ks, cf) + def get_ut_layout(ctxt, cass): ks = ctxt.get_binding('ksname', None) if ks is not None: @@ -596,6 +626,7 @@ def get_ut_layout(ctxt, cass): 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'): @@ -639,7 +670,6 @@ syntax_rules += r''' ''' - def udf_name_completer(ctxt, cass): ks = ctxt.get_binding('ksname', None) if ks is not None: @@ -701,6 +731,7 @@ completer_for('userAggregateName', 'ksname')(cf_ks_dot_completer) completer_for('userAggregateName', 'dot')(cf_ks_dot_completer) completer_for('userAggregateName', 'udaname')(uda_name_completer) + @completer_for('orderByClause', 'ordercol') def select_order_column_completer(ctxt, cass): prev_order_cols = ctxt.get_binding('ordercol', ()) @@ -715,15 +746,18 @@ def select_order_column_completer(ctxt, cass): return [maybe_escape_name(order_by_candidates[len(prev_order_cols)])] return [Hint('No more orderable columns here.')] + @completer_for('relation', 'token') def relation_token_word_completer(ctxt, cass): return ['TOKEN('] + @completer_for('relation', 'rel_tokname') def relation_token_subject_completer(ctxt, cass): layout = get_table_meta(ctxt, cass) return [key.name for key in layout.partition_key] + @completer_for('relation', 'rel_lhs') def select_relation_lhs_completer(ctxt, cass): layout = get_table_meta(ctxt, cass) @@ -760,13 +794,15 @@ syntax_rules += r''' ; ''' + def regular_column_names(table_meta): if not table_meta or not table_meta.columns: return [] - regular_coulmns = list(set(table_meta.columns.keys()) - - set([key.name for key in table_meta.partition_key]) - - set([key.name for key in table_meta.clustering_key])) - return regular_coulmns + regular_columns = list(set(table_meta.columns.keys()) - + set([key.name for key in table_meta.partition_key]) - + set([key.name for key in table_meta.clustering_key])) + return regular_columns + @completer_for('insertStatement', 'colname') def insert_colname_completer(ctxt, cass): @@ -779,6 +815,7 @@ def insert_colname_completer(ctxt, cass): normalcols = set(regular_column_names(layout)) - colnames return map(maybe_escape_name, normalcols) + @completer_for('insertStatement', 'newval') def insert_newval_completer(ctxt, cass): layout = get_table_meta(ctxt, cass) @@ -798,6 +835,7 @@ def insert_newval_completer(ctxt, cass): return [Hint('' % (maybe_escape_name(curcol), cqltype.cql_parameterized_type()))] + @completer_for('insertStatement', 'valcomma') def insert_valcomma_completer(ctxt, cass): layout = get_table_meta(ctxt, cass) @@ -807,6 +845,7 @@ def insert_valcomma_completer(ctxt, cass): return [','] return [')'] + @completer_for('insertStatement', 'insertopt') def insert_option_completer(ctxt, cass): opts = set('TIMESTAMP TTL'.split()) @@ -835,6 +874,7 @@ syntax_rules += r''' ; ''' + @completer_for('updateStatement', 'updateopt') def insert_option_completer(ctxt, cass): opts = set('TIMESTAMP TTL'.split()) @@ -842,11 +882,13 @@ def insert_option_completer(ctxt, cass): opts.discard(opt.split()[0]) return opts + @completer_for('assignment', 'updatecol') def update_col_completer(ctxt, cass): layout = get_table_meta(ctxt, cass) return map(maybe_escape_name, regular_column_names(layout)) + @completer_for('assignment', 'update_rhs') def update_countername_completer(ctxt, cass): layout = get_table_meta(ctxt, cass) @@ -861,12 +903,14 @@ def update_countername_completer(ctxt, cass): return ["["] return [Hint('' % cqltype.cql_parameterized_type())] + @completer_for('assignment', 'counterop') def update_counterop_completer(ctxt, cass): layout = get_table_meta(ctxt, cass) curcol = dequote_name(ctxt.get_binding('updatecol', '')) return ['+', '-'] if layout.columns[curcol].data_type.typename == 'counter' else [] + @completer_for('assignment', 'inc') def update_counter_inc_completer(ctxt, cass): layout = get_table_meta(ctxt, cass) @@ -875,6 +919,7 @@ def update_counter_inc_completer(ctxt, cass): return [Hint('')] return [] + @completer_for('assignment', 'listadder') def update_listadder_completer(ctxt, cass): rhs = ctxt.get_binding('update_rhs') @@ -882,6 +927,7 @@ def update_listadder_completer(ctxt, cass): return ['+'] return [] + @completer_for('assignment', 'listcol') def update_listcol_completer(ctxt, cass): rhs = ctxt.get_binding('update_rhs') @@ -890,6 +936,7 @@ def update_listcol_completer(ctxt, cass): return [maybe_escape_name(colname)] return [] + @completer_for('assignment', 'indexbracket') def update_indexbracket_completer(ctxt, cass): layout = get_table_meta(ctxt, cass) @@ -912,6 +959,7 @@ syntax_rules += r''' ; ''' + @completer_for('deleteStatement', 'delopt') def delete_opt_completer(ctxt, cass): opts = set('TIMESTAMP'.split()) @@ -919,6 +967,7 @@ def delete_opt_completer(ctxt, cass): opts.discard(opt.split()[0]) return opts + @completer_for('deleteSelector', 'delcol') def delete_delcol_completer(ctxt, cass): layout = get_table_meta(ctxt, cass) @@ -938,6 +987,7 @@ syntax_rules += r''' ; ''' + @completer_for('batchStatement', 'batchopt') def batch_opt_completer(ctxt, cass): opts = set('TIMESTAMP'.split()) @@ -956,6 +1006,7 @@ syntax_rules += r''' ; ''' + @completer_for('createKeyspaceStatement', 'wat') def create_ks_wat_completer(ctxt, cass): # would prefer to get rid of the "schema" nomenclature in cql3 @@ -995,6 +1046,7 @@ syntax_rules += r''' ; ''' + @completer_for('cfamOrdering', 'ordercol') def create_cf_clustering_order_colname_completer(ctxt, cass): colnames = map(dequote_name, ctxt.get_binding('newcolname', ())) @@ -1002,6 +1054,7 @@ def create_cf_clustering_order_colname_completer(ctxt, cass): # precisely which are. This is good enough for now return colnames + @completer_for('createColumnFamilyStatement', 'wat') def create_cf_wat_completer(ctxt, cass): # would prefer to get rid of the "columnfamily" nomenclature in cql3 @@ -1012,6 +1065,7 @@ def create_cf_wat_completer(ctxt, cass): explain_completion('createColumnFamilyStatement', 'cf', '') explain_completion('compositeKeyCfSpec', 'newcolname', '') + @completer_for('createColumnFamilyStatement', 'dot') def create_cf_ks_dot_completer(ctxt, cass): ks = dequote_name(ctxt.get_binding('ks')) @@ -1019,6 +1073,7 @@ def create_cf_ks_dot_completer(ctxt, cass): return ['.'] return [] + @completer_for('pkDef', 'ptkey') def create_cf_pkdef_declaration_completer(ctxt, cass): cols_declared = ctxt.get_binding('newcolname') @@ -1030,6 +1085,7 @@ def create_cf_pkdef_declaration_completer(ctxt, cass): return () return [maybe_escape_name(cols_declared[0])] + @completer_for('compositeKeyCfSpec', 'pkey') def create_cf_composite_key_declaration_completer(ctxt, cass): cols_declared = ctxt.get_binding('newcolname') @@ -1041,14 +1097,17 @@ def create_cf_composite_key_declaration_completer(ctxt, cass): return () return [maybe_escape_name(cols_declared[0])] + @completer_for('compositeKeyCfSpec', 'k') def create_cf_composite_primary_key_keyword_completer(ctxt, cass): return ['KEY ('] + @completer_for('compositeKeyCfSpec', 'p') def create_cf_composite_primary_key_paren_completer(ctxt, cass): return ['('] + @completer_for('compositeKeyCfSpec', 'c') def create_cf_composite_primary_key_comma_completer(ctxt, cass): cols_declared = ctxt.get_binding('newcolname') @@ -1106,6 +1165,7 @@ explain_completion('createIndexStatement', 'indexname', '') explain_completion('createUserTypeStatement', 'typename', '') explain_completion('createUserTypeStatement', 'newcol', '') + @completer_for('createIndexStatement', 'col') def create_index_col_completer(ctxt, cass): layout = get_table_meta(ctxt, cass) @@ -1139,10 +1199,12 @@ syntax_rules += r''' ''' + @completer_for('indexName', 'ksname') def idx_ks_name_completer(ctxt, cass): return [maybe_escape_name(ks) + '.' for ks in cass.get_keyspace_names()] + @completer_for('indexName', 'dot') def idx_ks_dot_completer(ctxt, cass): name = dequote_name(ctxt.get_binding('ksname')) @@ -1150,6 +1212,7 @@ def idx_ks_dot_completer(ctxt, cass): return ['.'] return [] + @completer_for('indexName', 'idxname') def idx_ks_idx_name_completer(ctxt, cass): ks = ctxt.get_binding('ksname', None) @@ -1185,12 +1248,14 @@ syntax_rules += r''' ; ''' + @completer_for('alterInstructions', 'existcol') def alter_table_col_completer(ctxt, cass): layout = get_table_meta(ctxt, cass) cols = [str(md) 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) @@ -1310,6 +1375,7 @@ syntax_rules += r''' ; ''' + @completer_for('username', 'name') def username_name_completer(ctxt, cass): def maybe_quote(name): @@ -1324,6 +1390,7 @@ def username_name_completer(ctxt, cass): session = cass.session return [maybe_quote(row.values()[0].replace("'", "''")) for row in session.execute("LIST USERS")] + @completer_for('rolename', 'role') def rolename_completer(ctxt, cass): def maybe_quote(name): @@ -1348,12 +1415,14 @@ syntax_rules += r''' ''' explain_completion('createTriggerStatement', 'class', '\'fully qualified class name\'') + def get_trigger_names(ctxt, cass): ks = ctxt.get_binding('ksname', None) if ks is not None: ks = dequote_name(ks) return cass.get_trigger_names(ks) + @completer_for('dropTriggerStatement', 'triggername') def alter_type_field_completer(ctxt, cass): names = get_trigger_names(ctxt, cass) http://git-wip-us.apache.org/repos/asf/cassandra/blob/72f2cc4b/pylib/cqlshlib/cqlhandling.py ---------------------------------------------------------------------- diff --git a/pylib/cqlshlib/cqlhandling.py b/pylib/cqlshlib/cqlhandling.py index 98f9f7b..f171148 100644 --- a/pylib/cqlshlib/cqlhandling.py +++ b/pylib/cqlshlib/cqlhandling.py @@ -210,17 +210,17 @@ class CqlParsingRuleSet(pylexotron.ParsingRuleSet): newcandidates = [] for c in candidates: if self.want_space_between(tokens[-1], c) \ - and prefix is None \ - and not text[-1].isspace() \ - and not c[0].isspace(): + and prefix is None \ + and not text[-1].isspace() \ + and not c[0].isspace(): c = ' ' + c newcandidates.append(c) candidates = newcandidates # append a space for single, complete identifiers if len(candidates) == 1 and candidates[0][-1].isalnum() \ - and lasttype != 'unclosedString' \ - and lasttype != 'unclosedName': + and lasttype != 'unclosedString' \ + and lasttype != 'unclosedName': candidates[0] += ' ' return candidates, hints http://git-wip-us.apache.org/repos/asf/cassandra/blob/72f2cc4b/pylib/cqlshlib/displaying.py ---------------------------------------------------------------------- diff --git a/pylib/cqlshlib/displaying.py b/pylib/cqlshlib/displaying.py index 0cac309..00f3156 100644 --- a/pylib/cqlshlib/displaying.py +++ b/pylib/cqlshlib/displaying.py @@ -14,7 +14,6 @@ # 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' @@ -27,12 +26,15 @@ 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: @@ -108,7 +110,7 @@ DEFAULT_VALUE_COLORS = dict( ) COLUMN_NAME_COLORS = defaultdict(lambda: MAGENTA, - error=RED, - blob=DARK_MAGENTA, - reset=ANSI_RESET, -) + error=RED, + blob=DARK_MAGENTA, + reset=ANSI_RESET, + ) http://git-wip-us.apache.org/repos/asf/cassandra/blob/72f2cc4b/pylib/cqlshlib/formatting.py ---------------------------------------------------------------------- diff --git a/pylib/cqlshlib/formatting.py b/pylib/cqlshlib/formatting.py index f028424..c26e20c 100644 --- a/pylib/cqlshlib/formatting.py +++ b/pylib/cqlshlib/formatting.py @@ -20,12 +20,10 @@ import platform import re import sys import platform -import time from collections import defaultdict from . import wcwidth from .displaying import colorme, FormattedValue, DEFAULT_VALUE_COLORS -from datetime import datetime, timedelta from cassandra.cqltypes import EMPTY from cassandra.util import datetime_from_timestamp from util import UTC @@ -35,6 +33,7 @@ is_win = platform.system() == 'Windows' 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'): @@ -45,6 +44,7 @@ def _show_control_chars(match): 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) @@ -58,6 +58,7 @@ 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, date_time_format=None, float_precision=None): if nullval is None: @@ -76,6 +77,7 @@ def format_by_type(cqltype, val, encoding, colormap=None, addcolor=False, date_time_format=date_time_format, float_precision=float_precision, nullval=nullval) + def color_text(bval, colormap, displaywidth=None): # note that here, we render natural backslashes as just backslashes, # in the same color as surrounding text, when using color. When not @@ -101,11 +103,14 @@ DEFAULT_TIMESTAMP_FORMAT = '%Y-%m-%d %H:%M:%S%z' if platform.system() == 'Windows': DEFAULT_TIME_FORMAT = '%Y-%m-%d %H:%M:%S %Z' + class DateTimeFormat(): + def __init__(self, timestamp_format=DEFAULT_TIMESTAMP_FORMAT, date_format=DEFAULT_DATE_FORMAT, nanotime_format=DEFAULT_NANOTIME_FORMAT): - self.timestamp_format=timestamp_format - self.date_format=date_format - self.nanotime_format=nanotime_format + self.timestamp_format = timestamp_format + self.date_format = date_format + self.nanotime_format = nanotime_format + def format_value_default(val, colormap, **_): val = str(val) @@ -117,18 +122,21 @@ def format_value_default(val, colormap, **_): # making format_value a generic function _formatters = {} + def format_value(type, val, **kwargs): if val == EMPTY: return format_value_default('', **kwargs) formatter = _formatters.get(type.__name__, format_value_default) return formatter(val, **kwargs) + def formatter_for(typname): def registrator(f): _formatters[typname] = f return f return registrator + @formatter_for('bytearray') def format_value_blob(val, colormap, **_): bval = '0x' + ''.join('%02x' % c for c in val) @@ -142,10 +150,12 @@ def format_python_formatted_type(val, colormap, color, quote=False): bval = "'%s'" % bval 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') @@ -155,17 +165,19 @@ def format_value_uuid(val, colormap, **_): def formatter_value_inet(val, colormap, quote=False, **_): return format_python_formatted_type(val, colormap, 'inet', quote=quote) + @formatter_for('bool') def format_value_boolean(val, colormap, **_): return format_python_formatted_type(val, colormap, 'boolean') + def format_floating_point_type(val, colormap, float_precision, **_): if math.isnan(val): bval = 'NaN' elif math.isinf(val): bval = 'Infinity' else: - exponent = int(math.log10(abs(val))) if abs(val) > sys.float_info.epsilon else -sys.maxint -1 + exponent = int(math.log10(abs(val))) if abs(val) > sys.float_info.epsilon else -sys.maxsize - 1 if -4 <= exponent < float_precision: # when this is true %g will not use scientific notation, # increasing precision should not change this decision @@ -177,6 +189,7 @@ def format_floating_point_type(val, colormap, float_precision, **_): formatter_for('float')(format_floating_point_type) + def format_integer_type(val, colormap, **_): # base-10 only for now; support others? bval = str(val) @@ -185,6 +198,7 @@ def format_integer_type(val, colormap, **_): formatter_for('long')(format_integer_type) formatter_for('int')(format_integer_type) + @formatter_for('datetime') def format_value_timestamp(val, colormap, date_time_format, quote=False, **_): bval = strftime(date_time_format.timestamp_format, calendar.timegm(val.utctimetuple())) @@ -192,18 +206,22 @@ def format_value_timestamp(val, colormap, date_time_format, quote=False, **_): bval = "'%s'" % bval return colorme(bval, colormap, 'timestamp') + def strftime(time_format, seconds): tzless_dt = datetime_from_timestamp(seconds) return tzless_dt.replace(tzinfo=UTC()).strftime(time_format) + @formatter_for('Date') def format_value_date(val, colormap, **_): return format_python_formatted_type(val, colormap, 'date') + @formatter_for('Time') def format_value_time(val, colormap, **_): return format_python_formatted_type(val, colormap, 'time') + @formatter_for('str') def format_value_text(val, encoding, colormap, quote=False, **_): escapedval = val.replace(u'\\', u'\\\\') @@ -219,6 +237,7 @@ def format_value_text(val, encoding, colormap, quote=False, **_): # name alias formatter_for('unicode')(format_value_text) + def format_simple_collection(val, lbracket, rbracket, encoding, colormap, date_time_format, float_precision, nullval): subs = [format_value(type(sval), sval, encoding=encoding, colormap=colormap, @@ -232,16 +251,19 @@ def format_simple_collection(val, lbracket, rbracket, encoding, 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, date_time_format, float_precision, nullval, **_): return format_simple_collection(val, '[', ']', encoding, colormap, date_time_format, float_precision, nullval) + @formatter_for('tuple') def format_value_tuple(val, encoding, colormap, date_time_format, float_precision, nullval, **_): return format_simple_collection(val, '(', ')', encoding, colormap, date_time_format, float_precision, nullval) + @formatter_for('set') def format_value_set(val, encoding, colormap, date_time_format, float_precision, nullval, **_): return format_simple_collection(sorted(val), '{', '}', encoding, colormap, @@ -262,8 +284,8 @@ def format_value_map(val, encoding, colormap, date_time_format, float_precision, 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 + + 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) formatter_for('OrderedDict')(format_value_map) @@ -287,7 +309,7 @@ def format_value_utype(val, encoding, colormap, date_time_format, float_precisio 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 + + 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/72f2cc4b/pylib/cqlshlib/helptopics.py ---------------------------------------------------------------------- diff --git a/pylib/cqlshlib/helptopics.py b/pylib/cqlshlib/helptopics.py index 860a582..8e296e1 100644 --- a/pylib/cqlshlib/helptopics.py +++ b/pylib/cqlshlib/helptopics.py @@ -16,9 +16,11 @@ from .cql3handling import simple_cql_types + class CQLHelpTopics(object): + def get_help_topics(self): - return [ t[5:] for t in dir(self) if t.startswith('help_') ] + return [t[5:] for t in dir(self) if t.startswith('help_')] def print_help_topic(self, topic): getattr(self, 'help_' + topic.lower())() @@ -613,7 +615,9 @@ class CQLHelpTopics(object): unset. """ + class CQL3HelpTopics(CQLHelpTopics): + def help_create_keyspace(self): print """ CREATE KEYSPACE http://git-wip-us.apache.org/repos/asf/cassandra/blob/72f2cc4b/pylib/cqlshlib/pylexotron.py ---------------------------------------------------------------------- diff --git a/pylib/cqlshlib/pylexotron.py b/pylib/cqlshlib/pylexotron.py index b7558f5..9cf0f4e 100644 --- a/pylib/cqlshlib/pylexotron.py +++ b/pylib/cqlshlib/pylexotron.py @@ -17,7 +17,9 @@ import re from .saferscanner import SaferScanner + class LexingError(Exception): + @classmethod def from_text(cls, rulestr, unmatched, msg='Lexing error'): bad_char = len(rulestr) - len(unmatched) @@ -37,7 +39,9 @@ class LexingError(Exception): def __str__(self): return '%s at line %d, char %d' % (self.msg, self.linenum, self.charnum) + class Hint: + def __init__(self, text): self.text = text @@ -50,9 +54,11 @@ class Hint: def __repr__(self): return '%s(%r)' % (self.__class__, self.text) + def is_hint(x): return isinstance(x, Hint) + class ParseContext: """ These are meant to be immutable, although it would be something of a @@ -116,7 +122,9 @@ class ParseContext: return '<%s matched=%r remainder=%r prodname=%r bindings=%r>' \ % (self.__class__.__name__, self.matched, self.remainder, self.productionname, self.bindings) + class matcher: + def __init__(self, arg): self.arg = arg @@ -154,7 +162,9 @@ class matcher: def __repr__(self): return '%s(%r)' % (self.__class__.__name__, self.arg) + class choice(matcher): + def match(self, ctxt, completions): foundctxts = [] for a in self.arg: @@ -162,11 +172,15 @@ class choice(matcher): foundctxts.extend(subctxts) return foundctxts + class one_or_none(matcher): + def match(self, ctxt, completions): return [ctxt] + list(self.arg.match(ctxt, completions)) + class repeat(matcher): + def match(self, ctxt, completions): found = [ctxt] ctxts = [ctxt] @@ -179,7 +193,9 @@ class repeat(matcher): found.extend(new_ctxts) ctxts = new_ctxts + class rule_reference(matcher): + def match(self, ctxt, completions): prevname = ctxt.productionname try: @@ -189,7 +205,9 @@ class rule_reference(matcher): output = rule.match(ctxt.with_production_named(self.arg), completions) return [c.with_production_named(prevname) for c in output] + class rule_series(matcher): + def match(self, ctxt, completions): ctxts = [ctxt] for patpiece in self.arg: @@ -201,7 +219,9 @@ class rule_series(matcher): ctxts = new_ctxts return ctxts + class named_symbol(matcher): + def __init__(self, name, arg): matcher.__init__(self, arg) self.name = name @@ -217,7 +237,9 @@ class named_symbol(matcher): def __repr__(self): return '%s(%r, %r)' % (self.__class__.__name__, self.name, self.arg) + class named_collector(named_symbol): + def match(self, ctxt, completions): pass_in_compls = completions if self.try_registered_completion(ctxt, self.name, completions): @@ -229,11 +251,15 @@ class named_collector(named_symbol): output.append(ctxt.with_binding(self.name, oldval + (ctxt.extract_orig(matchtoks),))) return output + class terminal_matcher(matcher): + def pattern(self): raise NotImplementedError + class regex_rule(terminal_matcher): + def __init__(self, pat): terminal_matcher.__init__(self, pat) self.regex = pat @@ -250,6 +276,7 @@ class regex_rule(terminal_matcher): def pattern(self): return self.regex + class text_match(terminal_matcher): alpha_re = re.compile(r'[a-zA-Z]') @@ -274,7 +301,9 @@ class text_match(terminal_matcher): return '[%s%s]' % (c.upper(), c.lower()) return self.alpha_re.sub(ignorecaseify, re.escape(self.arg)) + class case_match(text_match): + def match(self, ctxt, completions): if ctxt.remainder: if self.arg == ctxt.remainder[0][1]: @@ -286,15 +315,21 @@ class case_match(text_match): def pattern(self): return re.escape(self.arg) + class word_match(text_match): + def pattern(self): return r'\b' + text_match.pattern(self) + r'\b' + class case_word_match(case_match): + def pattern(self): return r'\b' + case_match.pattern(self) + r'\b' + class terminal_type_matcher(matcher): + def __init__(self, tokentype, submatcher): matcher.__init__(self, tokentype) self.tokentype = tokentype @@ -311,16 +346,17 @@ class terminal_type_matcher(matcher): def __repr__(self): return '%s(%r, %r)' % (self.__class__.__name__, self.tokentype, self.submatcher) + class ParsingRuleSet: RuleSpecScanner = SaferScanner([ - (r'::=', lambda s,t: t), - (r'\[[a-z0-9_]+\]=', lambda s,t: ('named_collector', t[1:-2])), - (r'[a-z0-9_]+=', lambda s,t: ('named_symbol', t[:-1])), - (r'/(\[\^?.[^]]*\]|[^/]|\\.)*/', lambda s,t: ('regex', t[1:-1].replace(r'\/', '/'))), - (r'"([^"]|\\.)*"', lambda s,t: ('litstring', t)), - (r'<[^>]*>', lambda s,t: ('reference', t[1:-1])), - (r'\bJUNK\b', lambda s,t: ('junk', t)), - (r'[@()|?*;]', lambda s,t: t), + (r'::=', lambda s, t: t), + (r'\[[a-z0-9_]+\]=', lambda s, t: ('named_collector', t[1:-2])), + (r'[a-z0-9_]+=', lambda s, t: ('named_symbol', t[:-1])), + (r'/(\[\^?.[^]]*\]|[^/]|\\.)*/', lambda s, t: ('regex', t[1:-1].replace(r'\/', '/'))), + (r'"([^"]|\\.)*"', lambda s, t: ('litstring', t)), + (r'<[^>]*>', lambda s, t: ('reference', t[1:-1])), + (r'\bJUNK\b', lambda s, t: ('junk', t)), + (r'[@()|?*;]', lambda s, t: t), (r'\s+', None), (r'#[^\n]*', None), ], re.I | re.S) @@ -482,7 +518,8 @@ class ParsingRuleSet: pattern.match(ctxt, completions) return completions -import sys, traceback +import sys + class Debugotron(set): depth = 10 http://git-wip-us.apache.org/repos/asf/cassandra/blob/72f2cc4b/pylib/cqlshlib/saferscanner.py ---------------------------------------------------------------------- diff --git a/pylib/cqlshlib/saferscanner.py b/pylib/cqlshlib/saferscanner.py index 58f0e7d..2a05608 100644 --- a/pylib/cqlshlib/saferscanner.py +++ b/pylib/cqlshlib/saferscanner.py @@ -21,7 +21,9 @@ import re from sre_constants import BRANCH, SUBPATTERN, GROUPREF, GROUPREF_IGNORE, GROUPREF_EXISTS + class SaferScanner(re.Scanner): + def __init__(self, lexicon, flags=0): self.lexicon = lexicon p = [] @@ -29,9 +31,9 @@ class SaferScanner(re.Scanner): s.flags = flags for phrase, action in lexicon: p.append(re.sre_parse.SubPattern(s, [ - (SUBPATTERN, (len(p)+1, self.subpat(phrase, flags))), - ])) - s.groups = len(p)+1 + (SUBPATTERN, (len(p) + 1, self.subpat(phrase, flags))), + ])) + s.groups = len(p) + 1 p = re.sre_parse.SubPattern(s, [(BRANCH, (None, p))]) self.p = p self.scanner = re.sre_compile.compile(p) http://git-wip-us.apache.org/repos/asf/cassandra/blob/72f2cc4b/pylib/cqlshlib/sslhandling.py ---------------------------------------------------------------------- diff --git a/pylib/cqlshlib/sslhandling.py b/pylib/cqlshlib/sslhandling.py index 2a90e26..8765ffa 100644 --- a/pylib/cqlshlib/sslhandling.py +++ b/pylib/cqlshlib/sslhandling.py @@ -22,7 +22,7 @@ import ssl def ssl_settings(host, config_file, env=os.environ): """ - Function wcich generates SSL setting for cassandra.Cluster + Function which generates SSL setting for cassandra.Cluster Params: * host .........: hostname of Cassandra node. @@ -73,7 +73,7 @@ def ssl_settings(host, config_file, env=os.environ): "to be specified. Please provide path to the certfile in [ssl] section " "as 'certfile' option in %s (or use [certfiles] section) or set SSL_CERTFILE " "environment variable." % (config_file,)) - if not ssl_certfile is None: + if ssl_certfile is not None: ssl_certfile = os.path.expanduser(ssl_certfile) userkey = get_option('ssl', 'userkey') http://git-wip-us.apache.org/repos/asf/cassandra/blob/72f2cc4b/pylib/cqlshlib/util.py ---------------------------------------------------------------------- diff --git a/pylib/cqlshlib/util.py b/pylib/cqlshlib/util.py index dfa480d..4d6cf8a 100644 --- a/pylib/cqlshlib/util.py +++ b/pylib/cqlshlib/util.py @@ -20,6 +20,7 @@ from datetime import timedelta, tzinfo ZERO = timedelta(0) + class UTC(tzinfo): """UTC""" http://git-wip-us.apache.org/repos/asf/cassandra/blob/72f2cc4b/pylib/cqlshlib/wcwidth.py ---------------------------------------------------------------------- diff --git a/pylib/cqlshlib/wcwidth.py b/pylib/cqlshlib/wcwidth.py index 43a4aca..bc55c31 100644 --- a/pylib/cqlshlib/wcwidth.py +++ b/pylib/cqlshlib/wcwidth.py @@ -77,20 +77,22 @@ # Latest C version: http://www.cl.cam.ac.uk/~mgk25/ucs/wcwidth.c # auxiliary function for binary search in interval table + + def bisearch(ucs, table): - min = 0 - max = len(table) - 1 - if ucs < table[0][0] or ucs > table[max][1]: + min = 0 + max = len(table) - 1 + if ucs < table[0][0] or ucs > table[max][1]: + return 0 + while max >= min: + mid = (min + max) / 2 + if ucs > table[mid][1]: + min = mid + 1 + elif ucs < table[mid][0]: + max = mid - 1 + else: + return 1 return 0 - while max >= min: - mid = (min + max) / 2 - if ucs > table[mid][1]: - min = mid + 1 - elif ucs < table[mid][0]: - max = mid - 1 - else: - return 1 - return 0 # The following two functions define the column width of an ISO 10646 # character as follows: @@ -126,152 +128,153 @@ def bisearch(ucs, table): # sorted list of non-overlapping intervals of non-spacing characters # generated by "uniset +cat=Me +cat=Mn +cat=Cf -00AD +1160-11FF +200B c" combining = ( - ( 0x0300, 0x036F ), ( 0x0483, 0x0486 ), ( 0x0488, 0x0489 ), - ( 0x0591, 0x05BD ), ( 0x05BF, 0x05BF ), ( 0x05C1, 0x05C2 ), - ( 0x05C4, 0x05C5 ), ( 0x05C7, 0x05C7 ), ( 0x0600, 0x0603 ), - ( 0x0610, 0x0615 ), ( 0x064B, 0x065E ), ( 0x0670, 0x0670 ), - ( 0x06D6, 0x06E4 ), ( 0x06E7, 0x06E8 ), ( 0x06EA, 0x06ED ), - ( 0x070F, 0x070F ), ( 0x0711, 0x0711 ), ( 0x0730, 0x074A ), - ( 0x07A6, 0x07B0 ), ( 0x07EB, 0x07F3 ), ( 0x0901, 0x0902 ), - ( 0x093C, 0x093C ), ( 0x0941, 0x0948 ), ( 0x094D, 0x094D ), - ( 0x0951, 0x0954 ), ( 0x0962, 0x0963 ), ( 0x0981, 0x0981 ), - ( 0x09BC, 0x09BC ), ( 0x09C1, 0x09C4 ), ( 0x09CD, 0x09CD ), - ( 0x09E2, 0x09E3 ), ( 0x0A01, 0x0A02 ), ( 0x0A3C, 0x0A3C ), - ( 0x0A41, 0x0A42 ), ( 0x0A47, 0x0A48 ), ( 0x0A4B, 0x0A4D ), - ( 0x0A70, 0x0A71 ), ( 0x0A81, 0x0A82 ), ( 0x0ABC, 0x0ABC ), - ( 0x0AC1, 0x0AC5 ), ( 0x0AC7, 0x0AC8 ), ( 0x0ACD, 0x0ACD ), - ( 0x0AE2, 0x0AE3 ), ( 0x0B01, 0x0B01 ), ( 0x0B3C, 0x0B3C ), - ( 0x0B3F, 0x0B3F ), ( 0x0B41, 0x0B43 ), ( 0x0B4D, 0x0B4D ), - ( 0x0B56, 0x0B56 ), ( 0x0B82, 0x0B82 ), ( 0x0BC0, 0x0BC0 ), - ( 0x0BCD, 0x0BCD ), ( 0x0C3E, 0x0C40 ), ( 0x0C46, 0x0C48 ), - ( 0x0C4A, 0x0C4D ), ( 0x0C55, 0x0C56 ), ( 0x0CBC, 0x0CBC ), - ( 0x0CBF, 0x0CBF ), ( 0x0CC6, 0x0CC6 ), ( 0x0CCC, 0x0CCD ), - ( 0x0CE2, 0x0CE3 ), ( 0x0D41, 0x0D43 ), ( 0x0D4D, 0x0D4D ), - ( 0x0DCA, 0x0DCA ), ( 0x0DD2, 0x0DD4 ), ( 0x0DD6, 0x0DD6 ), - ( 0x0E31, 0x0E31 ), ( 0x0E34, 0x0E3A ), ( 0x0E47, 0x0E4E ), - ( 0x0EB1, 0x0EB1 ), ( 0x0EB4, 0x0EB9 ), ( 0x0EBB, 0x0EBC ), - ( 0x0EC8, 0x0ECD ), ( 0x0F18, 0x0F19 ), ( 0x0F35, 0x0F35 ), - ( 0x0F37, 0x0F37 ), ( 0x0F39, 0x0F39 ), ( 0x0F71, 0x0F7E ), - ( 0x0F80, 0x0F84 ), ( 0x0F86, 0x0F87 ), ( 0x0F90, 0x0F97 ), - ( 0x0F99, 0x0FBC ), ( 0x0FC6, 0x0FC6 ), ( 0x102D, 0x1030 ), - ( 0x1032, 0x1032 ), ( 0x1036, 0x1037 ), ( 0x1039, 0x1039 ), - ( 0x1058, 0x1059 ), ( 0x1160, 0x11FF ), ( 0x135F, 0x135F ), - ( 0x1712, 0x1714 ), ( 0x1732, 0x1734 ), ( 0x1752, 0x1753 ), - ( 0x1772, 0x1773 ), ( 0x17B4, 0x17B5 ), ( 0x17B7, 0x17BD ), - ( 0x17C6, 0x17C6 ), ( 0x17C9, 0x17D3 ), ( 0x17DD, 0x17DD ), - ( 0x180B, 0x180D ), ( 0x18A9, 0x18A9 ), ( 0x1920, 0x1922 ), - ( 0x1927, 0x1928 ), ( 0x1932, 0x1932 ), ( 0x1939, 0x193B ), - ( 0x1A17, 0x1A18 ), ( 0x1B00, 0x1B03 ), ( 0x1B34, 0x1B34 ), - ( 0x1B36, 0x1B3A ), ( 0x1B3C, 0x1B3C ), ( 0x1B42, 0x1B42 ), - ( 0x1B6B, 0x1B73 ), ( 0x1DC0, 0x1DCA ), ( 0x1DFE, 0x1DFF ), - ( 0x200B, 0x200F ), ( 0x202A, 0x202E ), ( 0x2060, 0x2063 ), - ( 0x206A, 0x206F ), ( 0x20D0, 0x20EF ), ( 0x302A, 0x302F ), - ( 0x3099, 0x309A ), ( 0xA806, 0xA806 ), ( 0xA80B, 0xA80B ), - ( 0xA825, 0xA826 ), ( 0xFB1E, 0xFB1E ), ( 0xFE00, 0xFE0F ), - ( 0xFE20, 0xFE23 ), ( 0xFEFF, 0xFEFF ), ( 0xFFF9, 0xFFFB ), - ( 0x10A01, 0x10A03 ), ( 0x10A05, 0x10A06 ), ( 0x10A0C, 0x10A0F ), - ( 0x10A38, 0x10A3A ), ( 0x10A3F, 0x10A3F ), ( 0x1D167, 0x1D169 ), - ( 0x1D173, 0x1D182 ), ( 0x1D185, 0x1D18B ), ( 0x1D1AA, 0x1D1AD ), - ( 0x1D242, 0x1D244 ), ( 0xE0001, 0xE0001 ), ( 0xE0020, 0xE007F ), - ( 0xE0100, 0xE01EF ) + (0x0300, 0x036F), (0x0483, 0x0486), (0x0488, 0x0489), + (0x0591, 0x05BD), (0x05BF, 0x05BF), (0x05C1, 0x05C2), + (0x05C4, 0x05C5), (0x05C7, 0x05C7), (0x0600, 0x0603), + (0x0610, 0x0615), (0x064B, 0x065E), (0x0670, 0x0670), + (0x06D6, 0x06E4), (0x06E7, 0x06E8), (0x06EA, 0x06ED), + (0x070F, 0x070F), (0x0711, 0x0711), (0x0730, 0x074A), + (0x07A6, 0x07B0), (0x07EB, 0x07F3), (0x0901, 0x0902), + (0x093C, 0x093C), (0x0941, 0x0948), (0x094D, 0x094D), + (0x0951, 0x0954), (0x0962, 0x0963), (0x0981, 0x0981), + (0x09BC, 0x09BC), (0x09C1, 0x09C4), (0x09CD, 0x09CD), + (0x09E2, 0x09E3), (0x0A01, 0x0A02), (0x0A3C, 0x0A3C), + (0x0A41, 0x0A42), (0x0A47, 0x0A48), (0x0A4B, 0x0A4D), + (0x0A70, 0x0A71), (0x0A81, 0x0A82), (0x0ABC, 0x0ABC), + (0x0AC1, 0x0AC5), (0x0AC7, 0x0AC8), (0x0ACD, 0x0ACD), + (0x0AE2, 0x0AE3), (0x0B01, 0x0B01), (0x0B3C, 0x0B3C), + (0x0B3F, 0x0B3F), (0x0B41, 0x0B43), (0x0B4D, 0x0B4D), + (0x0B56, 0x0B56), (0x0B82, 0x0B82), (0x0BC0, 0x0BC0), + (0x0BCD, 0x0BCD), (0x0C3E, 0x0C40), (0x0C46, 0x0C48), + (0x0C4A, 0x0C4D), (0x0C55, 0x0C56), (0x0CBC, 0x0CBC), + (0x0CBF, 0x0CBF), (0x0CC6, 0x0CC6), (0x0CCC, 0x0CCD), + (0x0CE2, 0x0CE3), (0x0D41, 0x0D43), (0x0D4D, 0x0D4D), + (0x0DCA, 0x0DCA), (0x0DD2, 0x0DD4), (0x0DD6, 0x0DD6), + (0x0E31, 0x0E31), (0x0E34, 0x0E3A), (0x0E47, 0x0E4E), + (0x0EB1, 0x0EB1), (0x0EB4, 0x0EB9), (0x0EBB, 0x0EBC), + (0x0EC8, 0x0ECD), (0x0F18, 0x0F19), (0x0F35, 0x0F35), + (0x0F37, 0x0F37), (0x0F39, 0x0F39), (0x0F71, 0x0F7E), + (0x0F80, 0x0F84), (0x0F86, 0x0F87), (0x0F90, 0x0F97), + (0x0F99, 0x0FBC), (0x0FC6, 0x0FC6), (0x102D, 0x1030), + (0x1032, 0x1032), (0x1036, 0x1037), (0x1039, 0x1039), + (0x1058, 0x1059), (0x1160, 0x11FF), (0x135F, 0x135F), + (0x1712, 0x1714), (0x1732, 0x1734), (0x1752, 0x1753), + (0x1772, 0x1773), (0x17B4, 0x17B5), (0x17B7, 0x17BD), + (0x17C6, 0x17C6), (0x17C9, 0x17D3), (0x17DD, 0x17DD), + (0x180B, 0x180D), (0x18A9, 0x18A9), (0x1920, 0x1922), + (0x1927, 0x1928), (0x1932, 0x1932), (0x1939, 0x193B), + (0x1A17, 0x1A18), (0x1B00, 0x1B03), (0x1B34, 0x1B34), + (0x1B36, 0x1B3A), (0x1B3C, 0x1B3C), (0x1B42, 0x1B42), + (0x1B6B, 0x1B73), (0x1DC0, 0x1DCA), (0x1DFE, 0x1DFF), + (0x200B, 0x200F), (0x202A, 0x202E), (0x2060, 0x2063), + (0x206A, 0x206F), (0x20D0, 0x20EF), (0x302A, 0x302F), + (0x3099, 0x309A), (0xA806, 0xA806), (0xA80B, 0xA80B), + (0xA825, 0xA826), (0xFB1E, 0xFB1E), (0xFE00, 0xFE0F), + (0xFE20, 0xFE23), (0xFEFF, 0xFEFF), (0xFFF9, 0xFFFB), + (0x10A01, 0x10A03), (0x10A05, 0x10A06), (0x10A0C, 0x10A0F), + (0x10A38, 0x10A3A), (0x10A3F, 0x10A3F), (0x1D167, 0x1D169), + (0x1D173, 0x1D182), (0x1D185, 0x1D18B), (0x1D1AA, 0x1D1AD), + (0x1D242, 0x1D244), (0xE0001, 0xE0001), (0xE0020, 0xE007F), + (0xE0100, 0xE01EF) ) # sorted list of non-overlapping intervals of East Asian Ambiguous # characters, generated by "uniset +WIDTH-A -cat=Me -cat=Mn -cat=Cf c" ambiguous = ( - ( 0x00A1, 0x00A1 ), ( 0x00A4, 0x00A4 ), ( 0x00A7, 0x00A8 ), - ( 0x00AA, 0x00AA ), ( 0x00AE, 0x00AE ), ( 0x00B0, 0x00B4 ), - ( 0x00B6, 0x00BA ), ( 0x00BC, 0x00BF ), ( 0x00C6, 0x00C6 ), - ( 0x00D0, 0x00D0 ), ( 0x00D7, 0x00D8 ), ( 0x00DE, 0x00E1 ), - ( 0x00E6, 0x00E6 ), ( 0x00E8, 0x00EA ), ( 0x00EC, 0x00ED ), - ( 0x00F0, 0x00F0 ), ( 0x00F2, 0x00F3 ), ( 0x00F7, 0x00FA ), - ( 0x00FC, 0x00FC ), ( 0x00FE, 0x00FE ), ( 0x0101, 0x0101 ), - ( 0x0111, 0x0111 ), ( 0x0113, 0x0113 ), ( 0x011B, 0x011B ), - ( 0x0126, 0x0127 ), ( 0x012B, 0x012B ), ( 0x0131, 0x0133 ), - ( 0x0138, 0x0138 ), ( 0x013F, 0x0142 ), ( 0x0144, 0x0144 ), - ( 0x0148, 0x014B ), ( 0x014D, 0x014D ), ( 0x0152, 0x0153 ), - ( 0x0166, 0x0167 ), ( 0x016B, 0x016B ), ( 0x01CE, 0x01CE ), - ( 0x01D0, 0x01D0 ), ( 0x01D2, 0x01D2 ), ( 0x01D4, 0x01D4 ), - ( 0x01D6, 0x01D6 ), ( 0x01D8, 0x01D8 ), ( 0x01DA, 0x01DA ), - ( 0x01DC, 0x01DC ), ( 0x0251, 0x0251 ), ( 0x0261, 0x0261 ), - ( 0x02C4, 0x02C4 ), ( 0x02C7, 0x02C7 ), ( 0x02C9, 0x02CB ), - ( 0x02CD, 0x02CD ), ( 0x02D0, 0x02D0 ), ( 0x02D8, 0x02DB ), - ( 0x02DD, 0x02DD ), ( 0x02DF, 0x02DF ), ( 0x0391, 0x03A1 ), - ( 0x03A3, 0x03A9 ), ( 0x03B1, 0x03C1 ), ( 0x03C3, 0x03C9 ), - ( 0x0401, 0x0401 ), ( 0x0410, 0x044F ), ( 0x0451, 0x0451 ), - ( 0x2010, 0x2010 ), ( 0x2013, 0x2016 ), ( 0x2018, 0x2019 ), - ( 0x201C, 0x201D ), ( 0x2020, 0x2022 ), ( 0x2024, 0x2027 ), - ( 0x2030, 0x2030 ), ( 0x2032, 0x2033 ), ( 0x2035, 0x2035 ), - ( 0x203B, 0x203B ), ( 0x203E, 0x203E ), ( 0x2074, 0x2074 ), - ( 0x207F, 0x207F ), ( 0x2081, 0x2084 ), ( 0x20AC, 0x20AC ), - ( 0x2103, 0x2103 ), ( 0x2105, 0x2105 ), ( 0x2109, 0x2109 ), - ( 0x2113, 0x2113 ), ( 0x2116, 0x2116 ), ( 0x2121, 0x2122 ), - ( 0x2126, 0x2126 ), ( 0x212B, 0x212B ), ( 0x2153, 0x2154 ), - ( 0x215B, 0x215E ), ( 0x2160, 0x216B ), ( 0x2170, 0x2179 ), - ( 0x2190, 0x2199 ), ( 0x21B8, 0x21B9 ), ( 0x21D2, 0x21D2 ), - ( 0x21D4, 0x21D4 ), ( 0x21E7, 0x21E7 ), ( 0x2200, 0x2200 ), - ( 0x2202, 0x2203 ), ( 0x2207, 0x2208 ), ( 0x220B, 0x220B ), - ( 0x220F, 0x220F ), ( 0x2211, 0x2211 ), ( 0x2215, 0x2215 ), - ( 0x221A, 0x221A ), ( 0x221D, 0x2220 ), ( 0x2223, 0x2223 ), - ( 0x2225, 0x2225 ), ( 0x2227, 0x222C ), ( 0x222E, 0x222E ), - ( 0x2234, 0x2237 ), ( 0x223C, 0x223D ), ( 0x2248, 0x2248 ), - ( 0x224C, 0x224C ), ( 0x2252, 0x2252 ), ( 0x2260, 0x2261 ), - ( 0x2264, 0x2267 ), ( 0x226A, 0x226B ), ( 0x226E, 0x226F ), - ( 0x2282, 0x2283 ), ( 0x2286, 0x2287 ), ( 0x2295, 0x2295 ), - ( 0x2299, 0x2299 ), ( 0x22A5, 0x22A5 ), ( 0x22BF, 0x22BF ), - ( 0x2312, 0x2312 ), ( 0x2460, 0x24E9 ), ( 0x24EB, 0x254B ), - ( 0x2550, 0x2573 ), ( 0x2580, 0x258F ), ( 0x2592, 0x2595 ), - ( 0x25A0, 0x25A1 ), ( 0x25A3, 0x25A9 ), ( 0x25B2, 0x25B3 ), - ( 0x25B6, 0x25B7 ), ( 0x25BC, 0x25BD ), ( 0x25C0, 0x25C1 ), - ( 0x25C6, 0x25C8 ), ( 0x25CB, 0x25CB ), ( 0x25CE, 0x25D1 ), - ( 0x25E2, 0x25E5 ), ( 0x25EF, 0x25EF ), ( 0x2605, 0x2606 ), - ( 0x2609, 0x2609 ), ( 0x260E, 0x260F ), ( 0x2614, 0x2615 ), - ( 0x261C, 0x261C ), ( 0x261E, 0x261E ), ( 0x2640, 0x2640 ), - ( 0x2642, 0x2642 ), ( 0x2660, 0x2661 ), ( 0x2663, 0x2665 ), - ( 0x2667, 0x266A ), ( 0x266C, 0x266D ), ( 0x266F, 0x266F ), - ( 0x273D, 0x273D ), ( 0x2776, 0x277F ), ( 0xE000, 0xF8FF ), - ( 0xFFFD, 0xFFFD ), ( 0xF0000, 0xFFFFD ), ( 0x100000, 0x10FFFD ) + (0x00A1, 0x00A1), (0x00A4, 0x00A4), (0x00A7, 0x00A8), + (0x00AA, 0x00AA), (0x00AE, 0x00AE), (0x00B0, 0x00B4), + (0x00B6, 0x00BA), (0x00BC, 0x00BF), (0x00C6, 0x00C6), + (0x00D0, 0x00D0), (0x00D7, 0x00D8), (0x00DE, 0x00E1), + (0x00E6, 0x00E6), (0x00E8, 0x00EA), (0x00EC, 0x00ED), + (0x00F0, 0x00F0), (0x00F2, 0x00F3), (0x00F7, 0x00FA), + (0x00FC, 0x00FC), (0x00FE, 0x00FE), (0x0101, 0x0101), + (0x0111, 0x0111), (0x0113, 0x0113), (0x011B, 0x011B), + (0x0126, 0x0127), (0x012B, 0x012B), (0x0131, 0x0133), + (0x0138, 0x0138), (0x013F, 0x0142), (0x0144, 0x0144), + (0x0148, 0x014B), (0x014D, 0x014D), (0x0152, 0x0153), + (0x0166, 0x0167), (0x016B, 0x016B), (0x01CE, 0x01CE), + (0x01D0, 0x01D0), (0x01D2, 0x01D2), (0x01D4, 0x01D4), + (0x01D6, 0x01D6), (0x01D8, 0x01D8), (0x01DA, 0x01DA), + (0x01DC, 0x01DC), (0x0251, 0x0251), (0x0261, 0x0261), + (0x02C4, 0x02C4), (0x02C7, 0x02C7), (0x02C9, 0x02CB), + (0x02CD, 0x02CD), (0x02D0, 0x02D0), (0x02D8, 0x02DB), + (0x02DD, 0x02DD), (0x02DF, 0x02DF), (0x0391, 0x03A1), + (0x03A3, 0x03A9), (0x03B1, 0x03C1), (0x03C3, 0x03C9), + (0x0401, 0x0401), (0x0410, 0x044F), (0x0451, 0x0451), + (0x2010, 0x2010), (0x2013, 0x2016), (0x2018, 0x2019), + (0x201C, 0x201D), (0x2020, 0x2022), (0x2024, 0x2027), + (0x2030, 0x2030), (0x2032, 0x2033), (0x2035, 0x2035), + (0x203B, 0x203B), (0x203E, 0x203E), (0x2074, 0x2074), + (0x207F, 0x207F), (0x2081, 0x2084), (0x20AC, 0x20AC), + (0x2103, 0x2103), (0x2105, 0x2105), (0x2109, 0x2109), + (0x2113, 0x2113), (0x2116, 0x2116), (0x2121, 0x2122), + (0x2126, 0x2126), (0x212B, 0x212B), (0x2153, 0x2154), + (0x215B, 0x215E), (0x2160, 0x216B), (0x2170, 0x2179), + (0x2190, 0x2199), (0x21B8, 0x21B9), (0x21D2, 0x21D2), + (0x21D4, 0x21D4), (0x21E7, 0x21E7), (0x2200, 0x2200), + (0x2202, 0x2203), (0x2207, 0x2208), (0x220B, 0x220B), + (0x220F, 0x220F), (0x2211, 0x2211), (0x2215, 0x2215), + (0x221A, 0x221A), (0x221D, 0x2220), (0x2223, 0x2223), + (0x2225, 0x2225), (0x2227, 0x222C), (0x222E, 0x222E), + (0x2234, 0x2237), (0x223C, 0x223D), (0x2248, 0x2248), + (0x224C, 0x224C), (0x2252, 0x2252), (0x2260, 0x2261), + (0x2264, 0x2267), (0x226A, 0x226B), (0x226E, 0x226F), + (0x2282, 0x2283), (0x2286, 0x2287), (0x2295, 0x2295), + (0x2299, 0x2299), (0x22A5, 0x22A5), (0x22BF, 0x22BF), + (0x2312, 0x2312), (0x2460, 0x24E9), (0x24EB, 0x254B), + (0x2550, 0x2573), (0x2580, 0x258F), (0x2592, 0x2595), + (0x25A0, 0x25A1), (0x25A3, 0x25A9), (0x25B2, 0x25B3), + (0x25B6, 0x25B7), (0x25BC, 0x25BD), (0x25C0, 0x25C1), + (0x25C6, 0x25C8), (0x25CB, 0x25CB), (0x25CE, 0x25D1), + (0x25E2, 0x25E5), (0x25EF, 0x25EF), (0x2605, 0x2606), + (0x2609, 0x2609), (0x260E, 0x260F), (0x2614, 0x2615), + (0x261C, 0x261C), (0x261E, 0x261E), (0x2640, 0x2640), + (0x2642, 0x2642), (0x2660, 0x2661), (0x2663, 0x2665), + (0x2667, 0x266A), (0x266C, 0x266D), (0x266F, 0x266F), + (0x273D, 0x273D), (0x2776, 0x277F), (0xE000, 0xF8FF), + (0xFFFD, 0xFFFD), (0xF0000, 0xFFFFD), (0x100000, 0x10FFFD) ) -def mk_wcwidth(ucs): - # test for 8-bit control characters - if ucs == 0: - return 0 - if ucs < 32 or (ucs >= 0x7f and ucs < 0xa0): - return -1 - - # binary search in table of non-spacing characters - if bisearch(ucs, combining): - return 0 - - # if we arrive here, ucs is not a combining or C0/C1 control character - return 1 + \ - int(ucs >= 0x1100 and - (ucs <= 0x115f or # Hangul Jamo init. consonants - ucs == 0x2329 or ucs == 0x232a or - (ucs >= 0x2e80 and ucs <= 0xa4cf and - ucs != 0x303f) or # CJK ... Yi - (ucs >= 0xac00 and ucs <= 0xd7a3) or # Hangul Syllables - (ucs >= 0xf900 and ucs <= 0xfaff) or # CJK Compatibility Ideographs - (ucs >= 0xfe10 and ucs <= 0xfe19) or # Vertical forms - (ucs >= 0xfe30 and ucs <= 0xfe6f) or # CJK Compatibility Forms - (ucs >= 0xff00 and ucs <= 0xff60) or # Fullwidth Forms - (ucs >= 0xffe0 and ucs <= 0xffe6) or - (ucs >= 0x20000 and ucs <= 0x2fffd) or - (ucs >= 0x30000 and ucs <= 0x3fffd))) +def mk_wcwidth(ucs): + # test for 8-bit control characters + if ucs == 0: + return 0 + if ucs < 32 or (ucs >= 0x7f and ucs < 0xa0): + return -1 + + # binary search in table of non-spacing characters + if bisearch(ucs, combining): + return 0 + + # if we arrive here, ucs is not a combining or C0/C1 control character + + return 1 + \ + int(ucs >= 0x1100 and + (ucs <= 0x115f or # Hangul Jamo init. consonants + ucs == 0x2329 or ucs == 0x232a or + (ucs >= 0x2e80 and ucs <= 0xa4cf and + ucs != 0x303f) or # CJK ... Yi + (ucs >= 0xac00 and ucs <= 0xd7a3) or # Hangul Syllables + (ucs >= 0xf900 and ucs <= 0xfaff) or # CJK Compatibility Ideographs + (ucs >= 0xfe10 and ucs <= 0xfe19) or # Vertical forms + (ucs >= 0xfe30 and ucs <= 0xfe6f) or # CJK Compatibility Forms + (ucs >= 0xff00 and ucs <= 0xff60) or # Fullwidth Forms + (ucs >= 0xffe0 and ucs <= 0xffe6) or + (ucs >= 0x20000 and ucs <= 0x2fffd) or + (ucs >= 0x30000 and ucs <= 0x3fffd))) def mk_wcswidth(pwcs): - width = 0 - for c in pwcs: - w = mk_wcwidth(c) - if w < 0: - return -1 - else: - width += w + width = 0 + for c in pwcs: + w = mk_wcwidth(c) + if w < 0: + return -1 + else: + width += w - return width + return width # The following functions are the same as mk_wcwidth() and @@ -282,34 +285,39 @@ def mk_wcswidth(pwcs): # the traditional terminal character-width behaviour. It is not # otherwise recommended for general use. def mk_wcwidth_cjk(ucs): - # binary search in table of non-spacing characters - if bisearch(ucs, ambiguous): - return 2 + # binary search in table of non-spacing characters + if bisearch(ucs, ambiguous): + return 2 - return mk_wcwidth(ucs) + return mk_wcwidth(ucs) def mk_wcswidth_cjk(pwcs): - width = 0 + width = 0 - for c in pwcs: - w = mk_wcwidth_cjk(c) - if w < 0: - return -1 - width += w + for c in pwcs: + w = mk_wcwidth_cjk(c) + if w < 0: + return -1 + width += w - return width + return width # python-y versions, dealing with unicode objects + + def wcwidth(c): return mk_wcwidth(ord(c)) + def wcswidth(s): return mk_wcswidth(map(ord, s)) + def wcwidth_cjk(c): return mk_wcwidth_cjk(ord(c)) + def wcswidth_cjk(s): return mk_wcswidth_cjk(map(ord, s)) @@ -338,12 +346,12 @@ if __name__ == "__main__": uchr = unicodedata.lookup(name) calculatedwidth = wcwidth(uchr) assert calculatedwidth == printwidth, \ - 'width for %r should be %d, but is %d?' % (uchr, printwidth, calculatedwidth) + 'width for %r should be %d, but is %d?' % (uchr, printwidth, calculatedwidth) for c in nonprinting: calculatedwidth = wcwidth(c) assert calculatedwidth < 0, \ - '%r is a control character, but wcwidth gives %d' % (c, calculatedwidth) + '%r is a control character, but wcwidth gives %d' % (c, calculatedwidth) assert wcwidth('\0') == 0 # special case