mirror of https://github.com/apache/cassandra
1762 lines
62 KiB
Python
1762 lines
62 KiB
Python
# Licensed to the Apache Software Foundation (ASF) under one
|
|
# or more contributor license agreements. See the NOTICE file
|
|
# distributed with this work for additional information
|
|
# regarding copyright ownership. The ASF licenses this file
|
|
# to you under the Apache License, Version 2.0 (the
|
|
# "License"); you may not use this file except in compliance
|
|
# with the License. You may obtain a copy of the License at
|
|
#
|
|
# http://www.apache.org/licenses/LICENSE-2.0
|
|
#
|
|
# Unless required by applicable law or agreed to in writing, software
|
|
# distributed under the License is distributed on an "AS IS" BASIS,
|
|
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
|
# See the License for the specific language governing permissions and
|
|
# limitations under the License.
|
|
|
|
from cassandra.metadata import maybe_escape_name
|
|
from cqlshlib import helptopics
|
|
from cqlshlib.cqlhandling import CqlParsingRuleSet, Hint
|
|
|
|
simple_cql_types = {'ascii', 'bigint', 'blob', 'boolean', 'counter', 'date', 'decimal', 'double', 'duration', 'float',
|
|
'inet', 'int', 'smallint', 'text', 'time', 'timestamp', 'timeuuid', 'tinyint', 'uuid', 'varchar',
|
|
'varint'}
|
|
simple_cql_types.difference_update(('set', 'map', 'list', 'vector'))
|
|
|
|
|
|
class UnexpectedTableStructure(UserWarning):
|
|
|
|
def __init__(self, msg):
|
|
self.msg = msg
|
|
|
|
def __str__(self):
|
|
return 'Unexpected table structure; may not translate correctly to CQL. ' + self.msg
|
|
|
|
|
|
SYSTEM_KEYSPACES = ('system', 'system_schema', 'system_traces', 'system_auth', 'system_distributed', 'system_views',
|
|
'system_metrics', 'system_virtual_schema', 'system_cluster_metadata')
|
|
NONALTERBALE_KEYSPACES = ('system', 'system_schema', 'system_views', 'system_metrics', 'system_virtual_schema',
|
|
'system_cluster_metadata')
|
|
|
|
|
|
class Cql3ParsingRuleSet(CqlParsingRuleSet):
|
|
|
|
columnfamily_layout_options = (
|
|
('allow_auto_snapshot', None),
|
|
('bloom_filter_fp_chance', None),
|
|
('comment', None),
|
|
('gc_grace_seconds', None),
|
|
('incremental_backups', None),
|
|
('min_index_interval', None),
|
|
('max_index_interval', None),
|
|
('default_time_to_live', None),
|
|
('speculative_retry', None),
|
|
('additional_write_policy', None),
|
|
('memtable', None),
|
|
('memtable_flush_period_in_ms', None),
|
|
('cdc', None),
|
|
('read_repair', None),
|
|
)
|
|
|
|
columnfamily_layout_map_options = (
|
|
# (CQL3 option name, schema_columnfamilies column name (or None if same),
|
|
# list of known map keys)
|
|
('compaction', 'compaction_strategy_options',
|
|
('class', 'max_threshold', 'tombstone_compaction_interval', 'tombstone_threshold', 'enabled',
|
|
'unchecked_tombstone_compaction', 'only_purge_repaired_tombstones', 'provide_overlapping_tombstones')),
|
|
('compression', 'compression_parameters',
|
|
('class', 'chunk_length_in_kb', 'enabled', 'min_compress_ratio', 'max_compressed_length')),
|
|
('caching', None,
|
|
('rows_per_partition', 'keys')),
|
|
)
|
|
|
|
obsolete_cf_options = ()
|
|
|
|
consistency_levels = (
|
|
'ANY',
|
|
'ONE',
|
|
'TWO',
|
|
'THREE',
|
|
'QUORUM',
|
|
'ALL',
|
|
'LOCAL_QUORUM',
|
|
'EACH_QUORUM',
|
|
'SERIAL'
|
|
)
|
|
|
|
size_tiered_compaction_strategy_options = (
|
|
'min_sstable_size',
|
|
'min_threshold',
|
|
'bucket_high',
|
|
'bucket_low'
|
|
)
|
|
|
|
leveled_compaction_strategy_options = (
|
|
'sstable_size_in_mb',
|
|
'fanout_size'
|
|
)
|
|
|
|
time_window_compaction_strategy_options = (
|
|
'compaction_window_unit',
|
|
'compaction_window_size',
|
|
'min_threshold',
|
|
'timestamp_resolution'
|
|
)
|
|
|
|
unified_compaction_strategy_options = (
|
|
'scaling_parameters',
|
|
'min_sstable_size',
|
|
'flush_size_override',
|
|
'base_shard_count',
|
|
'target_sstable_size',
|
|
'sstable_growth',
|
|
'max_sstables_to_compact',
|
|
'expired_sstable_check_frequency_seconds',
|
|
'unsafe_aggressive_sstable_expiration',
|
|
'overlap_inclusion_method'
|
|
)
|
|
|
|
@classmethod
|
|
def escape_value(cls, value):
|
|
if value is None:
|
|
return 'NULL' # this totally won't work
|
|
if isinstance(value, bool):
|
|
value = str(value).lower()
|
|
elif isinstance(value, float):
|
|
return '%f' % value
|
|
elif isinstance(value, int):
|
|
return str(value)
|
|
return "'%s'" % value.replace("'", "''")
|
|
|
|
@classmethod
|
|
def escape_name(cls, name):
|
|
if name is None:
|
|
return 'NULL'
|
|
return "'%s'" % name.replace("'", "''")
|
|
|
|
@staticmethod
|
|
def dequote_name(name):
|
|
name = name.strip()
|
|
if name == '':
|
|
return name
|
|
if name[0] == '"' and name[-1] == '"':
|
|
return name[1:-1].replace('""', '"')
|
|
else:
|
|
return name.lower()
|
|
|
|
@staticmethod
|
|
def dequote_value(cqlword):
|
|
cqlword = cqlword.strip()
|
|
if cqlword == '':
|
|
return cqlword
|
|
if cqlword[0] == "'" and cqlword[-1] == "'":
|
|
cqlword = cqlword[1:-1].replace("''", "'")
|
|
return cqlword
|
|
|
|
|
|
CqlRuleSet = Cql3ParsingRuleSet()
|
|
|
|
# convenience for remainder of module
|
|
completer_for = CqlRuleSet.completer_for
|
|
explain_completion = CqlRuleSet.explain_completion
|
|
dequote_value = CqlRuleSet.dequote_value
|
|
dequote_name = CqlRuleSet.dequote_name
|
|
escape_value = CqlRuleSet.escape_value
|
|
|
|
# BEGIN SYNTAX/COMPLETION RULE DEFINITIONS
|
|
|
|
syntax_rules = r'''
|
|
<Start> ::= <CQL_Statement>*
|
|
;
|
|
|
|
<CQL_Statement> ::= [statements]=<statementBody> ";"
|
|
;
|
|
|
|
# The order of these terminal productions is significant. The input string is matched to the rule
|
|
# specified first in the grammar.
|
|
|
|
<endline> ::= /\n/ ;
|
|
|
|
JUNK ::= /([ \t\r\f\v]+|(--|[/][/])[^\n\r]*([\n\r]|$)|[/][*].*?[*][/])/ ;
|
|
|
|
<stringLiteral> ::= <quotedStringLiteral>
|
|
| <pgStringLiteral> ;
|
|
<quotedStringLiteral> ::= /'([^']|'')*'/ ;
|
|
<pgStringLiteral> ::= /\$\$(?:(?!\$\$).)*\$\$/;
|
|
<quotedName> ::= /"([^"]|"")*"/ ;
|
|
|
|
<unclosedPgString>::= /\$\$(?:(?!\$\$).)*/ ;
|
|
<unclosedString> ::= /'([^']|'')*/ ;
|
|
<unclosedName> ::= /"([^"]|"")*/ ;
|
|
<unclosedComment> ::= /[/][*].*$/ ;
|
|
|
|
<float> ::= /-?[0-9]+\.[0-9]+/ ;
|
|
<uuid> ::= /[0-9a-f]{8}-[0-9a-f]{4}-[0-9a-f]{4}-[0-9a-f]{4}-[0-9a-f]{12}/ ;
|
|
<blobLiteral> ::= /0x[0-9a-f]+/ ;
|
|
<wholenumber> ::= /[0-9]+/ ;
|
|
<identifier> ::= /[a-z][a-z0-9_]*/ ;
|
|
<colon> ::= ":" ;
|
|
<star> ::= "*" ;
|
|
<endtoken> ::= ";" ;
|
|
<op> ::= /[-+=%/,().]/ ;
|
|
<cmp> ::= /[<>!]=?/ ;
|
|
<brackets> ::= /[][{}]/ ;
|
|
|
|
<integer> ::= "-"? <wholenumber> ;
|
|
<boolean> ::= "true"
|
|
| "false"
|
|
;
|
|
|
|
<term> ::= <stringLiteral>
|
|
| <integer>
|
|
| <float>
|
|
| <uuid>
|
|
| <boolean>
|
|
| <blobLiteral>
|
|
| <collectionLiteral>
|
|
| <functionLiteral> <functionArguments>
|
|
| "NULL"
|
|
;
|
|
|
|
<functionLiteral> ::= (<identifier> ( "." <identifier> )?)
|
|
| "TOKEN"
|
|
;
|
|
|
|
<functionArguments> ::= "(" ( <term> ( "," <term> )* )? ")"
|
|
;
|
|
|
|
<tokenDefinition> ::= token="TOKEN" "(" <term> ( "," <term> )* ")"
|
|
| <term>
|
|
;
|
|
<cident> ::= <quotedName>
|
|
| <identifier>
|
|
| <unreservedKeyword>
|
|
;
|
|
<colname> ::= <cident> ; # just an alias
|
|
|
|
<collectionLiteral> ::= <listLiteral>
|
|
| <setLiteral>
|
|
| <mapLiteral>
|
|
;
|
|
<listLiteral> ::= "[" ( <term> ( "," <term> )* )? "]"
|
|
;
|
|
<setLiteral> ::= "{" ( <term> ( "," <term> )* )? "}"
|
|
;
|
|
<mapLiteral> ::= "{" <term> ":" <term> ( "," <term> ":" <term> )* "}"
|
|
;
|
|
|
|
<anyFunctionName> ::= ( ksname=<cfOrKsName> dot="." )? udfname=<cfOrKsName> ;
|
|
|
|
<userFunctionName> ::= ( ksname=<nonSystemKeyspaceName> dot="." )? udfname=<cfOrKsName> ;
|
|
|
|
<refUserFunctionName> ::= udfname=<cfOrKsName> ;
|
|
|
|
<userAggregateName> ::= ( ksname=<nonSystemKeyspaceName> dot="." )? udaname=<cfOrKsName> ;
|
|
|
|
<functionAggregateName> ::= ( ksname=<nonSystemKeyspaceName> dot="." )? functionname=<cfOrKsName> ;
|
|
|
|
<aggregateName> ::= <userAggregateName>
|
|
;
|
|
|
|
<functionName> ::= <functionAggregateName>
|
|
| "TOKEN"
|
|
;
|
|
|
|
<statementBody> ::= <useStatement>
|
|
| <selectStatement>
|
|
| <dataChangeStatement>
|
|
| <schemaChangeStatement>
|
|
| <authenticationStatement>
|
|
| <authorizationStatement>
|
|
;
|
|
|
|
<dataChangeStatement> ::= <insertStatement>
|
|
| <updateStatement>
|
|
| <deleteStatement>
|
|
| <truncateStatement>
|
|
| <batchStatement>
|
|
;
|
|
|
|
<schemaChangeStatement> ::= <createKeyspaceStatement>
|
|
| <createColumnFamilyStatement>
|
|
| <copyTableStatement>
|
|
| <createIndexStatement>
|
|
| <createMaterializedViewStatement>
|
|
| <createUserTypeStatement>
|
|
| <createFunctionStatement>
|
|
| <createAggregateStatement>
|
|
| <createTriggerStatement>
|
|
| <dropKeyspaceStatement>
|
|
| <dropColumnFamilyStatement>
|
|
| <dropIndexStatement>
|
|
| <dropMaterializedViewStatement>
|
|
| <dropUserTypeStatement>
|
|
| <dropFunctionStatement>
|
|
| <dropAggregateStatement>
|
|
| <dropTriggerStatement>
|
|
| <alterTableStatement>
|
|
| <alterKeyspaceStatement>
|
|
| <alterUserTypeStatement>
|
|
;
|
|
|
|
<authenticationStatement> ::= <createUserStatement>
|
|
| <alterUserStatement>
|
|
| <dropUserStatement>
|
|
| <listUsersStatement>
|
|
| <createRoleStatement>
|
|
| <alterRoleStatement>
|
|
| <dropRoleStatement>
|
|
| <listRolesStatement>
|
|
| <listSuperUsersStatement>
|
|
;
|
|
|
|
<authorizationStatement> ::= <grantStatement>
|
|
| <grantRoleStatement>
|
|
| <revokeStatement>
|
|
| <revokeRoleStatement>
|
|
| <listPermissionsStatement>
|
|
;
|
|
|
|
# timestamp is included here, since it's also a keyword
|
|
<simpleStorageType> ::= typename=( <identifier> | <stringLiteral> | "timestamp" ) ;
|
|
|
|
<userType> ::= utname=<cfOrKsName> ;
|
|
|
|
<storageType> ::= ( <simpleStorageType> | <collectionType> | <frozenCollectionType> | <vectorType> | <userType> ) ( <constraintsExpr> )? ( <column_mask> )? ;
|
|
|
|
<constraintsExpr> ::= "CHECK" <constraint> ( "AND" <constraint> )*
|
|
;
|
|
|
|
<constraint> ::= "NOT" "NULL"
|
|
| <constraintStandaloneFunction>
|
|
| <constraintComparableFunction> <functionArguments> <cmp> <term>
|
|
| <cident> <cmp> <term>
|
|
;
|
|
|
|
<constraintComparableFunction> ::= "LENGTH"
|
|
| "OCTET_LENGTH"
|
|
| "REGEXP"
|
|
;
|
|
|
|
<constraintStandaloneFunction> ::= "JSON"
|
|
;
|
|
|
|
<column_mask> ::= "MASKED" "WITH" ( "DEFAULT" | <functionName> <selectionFunctionArguments> );
|
|
|
|
# Note: autocomplete for frozen collection types does not handle nesting past depth 1 properly,
|
|
# but that's a lot of work to fix for little benefit.
|
|
<collectionType> ::= "map" "<" <simpleStorageType> "," ( <simpleStorageType> | <userType> ) ">"
|
|
| "list" "<" ( <simpleStorageType> | <userType> ) ">"
|
|
| "set" "<" ( <simpleStorageType> | <userType> ) ">"
|
|
;
|
|
|
|
<frozenCollectionType> ::= "frozen" "<" "map" "<" <storageType> "," <storageType> ">" ">"
|
|
| "frozen" "<" "list" "<" <storageType> ">" ">"
|
|
| "frozen" "<" "set" "<" <storageType> ">" ">"
|
|
;
|
|
|
|
<vectorType> ::= "vector" "<" <storageType> "," <wholenumber> ">" ;
|
|
|
|
<columnFamilyName> ::= ( ksname=<cfOrKsName> dot="." )? cfname=<cfOrKsName> ;
|
|
|
|
<materializedViewName> ::= ( ksname=<cfOrKsName> dot="." )? mvname=<cfOrKsName> ;
|
|
|
|
<userTypeName> ::= ( ksname=<cfOrKsName> dot="." )? utname=<cfOrKsName> ;
|
|
|
|
<keyspaceName> ::= ksname=<cfOrKsName> ;
|
|
|
|
<nonSystemKeyspaceName> ::= ksname=<cfOrKsName> ;
|
|
|
|
<alterableKeyspaceName> ::= ksname=<cfOrKsName> ;
|
|
|
|
<cfOrKsName> ::= <identifier>
|
|
| <quotedName>
|
|
| <unreservedKeyword>;
|
|
|
|
<unreservedKeyword> ::= nocomplete=
|
|
( "key"
|
|
| "clustering"
|
|
# | "count" -- to get count(*) completion, treat count as reserved
|
|
| "ttl"
|
|
| "compact"
|
|
| "storage"
|
|
| "type"
|
|
| "values" )
|
|
;
|
|
|
|
<property> ::= [propname]=<cident> propeq="=" [propval]=<propertyValue>
|
|
;
|
|
<propertyValue> ::= propsimpleval=( <stringLiteral>
|
|
| <identifier>
|
|
| <integer>
|
|
| <float>
|
|
| <unreservedKeyword> )
|
|
# we don't use <mapLiteral> here so we can get more targeted
|
|
# completions:
|
|
| propsimpleval="{" [propmapkey]=<term> ":" [propmapval]=<term>
|
|
( ender="," [propmapkey]=<term> ":" [propmapval]=<term> )*
|
|
ender="}"
|
|
;
|
|
<propertyOrOption> ::= <property>
|
|
| "INDEXES"
|
|
;
|
|
|
|
'''
|
|
|
|
|
|
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
|
|
# "clustering" that there won't actually be an equals sign, because
|
|
# there are no properties by those names. there are, on the other hand,
|
|
# table properties that start with those keywords which don't have
|
|
# equals signs at all.
|
|
curprop = ctxt.get_binding('propname')[-1].upper()
|
|
if curprop in ('COMPACT', 'CLUSTERING'):
|
|
return ()
|
|
return ['=']
|
|
|
|
|
|
completer_for('property', 'propeq')(prop_equals_completer)
|
|
|
|
|
|
@completer_for('property', 'propname')
|
|
def prop_name_completer(ctxt, cass):
|
|
if working_on_keyspace(ctxt):
|
|
return ks_prop_name_completer(ctxt, cass)
|
|
elif 'MATERIALIZED' == ctxt.get_binding('wat', '').upper():
|
|
props = cf_prop_name_completer(ctxt, cass)
|
|
props.remove('default_time_to_live')
|
|
props.remove('gc_grace_seconds')
|
|
return props
|
|
else:
|
|
return cf_prop_name_completer(ctxt, cass)
|
|
|
|
|
|
@completer_for('propertyValue', 'propsimpleval')
|
|
def prop_val_completer(ctxt, cass):
|
|
if working_on_keyspace(ctxt):
|
|
return ks_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):
|
|
return ks_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):
|
|
return ks_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):
|
|
return ks_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':
|
|
return ["'true'", "'false'"]
|
|
if optname == 'replication':
|
|
return ["{'class': '"]
|
|
return ()
|
|
|
|
|
|
def ks_prop_val_mapkey_completer(ctxt, cass):
|
|
optname = ctxt.get_binding('propname')[-1]
|
|
if optname != 'replication':
|
|
return ()
|
|
keysseen = list(map(dequote_value, ctxt.get_binding('propmapkey', ())))
|
|
valsseen = list(map(dequote_value, ctxt.get_binding('propmapval', ())))
|
|
for k, v in zip(keysseen, valsseen):
|
|
if k == 'class':
|
|
repclass = v
|
|
break
|
|
else:
|
|
return ["'class'"]
|
|
if repclass == 'SimpleStrategy':
|
|
opts = {'replication_factor'}
|
|
elif repclass == 'NetworkTopologyStrategy':
|
|
return [Hint('<dc_name>')]
|
|
return list(map(escape_value, opts.difference(keysseen)))
|
|
|
|
|
|
def ks_prop_val_mapval_completer(ctxt, cass):
|
|
optname = ctxt.get_binding('propname')[-1]
|
|
if optname != 'replication':
|
|
return ()
|
|
currentkey = dequote_value(ctxt.get_binding('propmapkey')[-1])
|
|
if currentkey == 'class':
|
|
return list(map(escape_value, CqlRuleSet.replication_strategies))
|
|
return [Hint('<term>')]
|
|
|
|
|
|
def ks_prop_val_mapender_completer(ctxt, cass):
|
|
optname = ctxt.get_binding('propname')[-1]
|
|
if optname != 'replication':
|
|
return [',']
|
|
keysseen = list(map(dequote_value, ctxt.get_binding('propmapkey', ())))
|
|
valsseen = list(map(dequote_value, ctxt.get_binding('propmapval', ())))
|
|
for k, v in zip(keysseen, valsseen):
|
|
if k == 'class':
|
|
repclass = v
|
|
break
|
|
else:
|
|
return [',']
|
|
if repclass == 'SimpleStrategy':
|
|
if 'replication_factor' not in keysseen:
|
|
return [',']
|
|
if repclass == 'NetworkTopologyStrategy' and len(keysseen) == 1:
|
|
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]
|
|
if this_opt == 'compression':
|
|
return ["{'class': '"]
|
|
if this_opt == 'compaction':
|
|
return ["{'class': '"]
|
|
if this_opt == 'caching':
|
|
return ["{'keys': '"]
|
|
if any(this_opt == opt[0] for opt in CqlRuleSet.obsolete_cf_options):
|
|
return ["'<obsolete_option>'"]
|
|
if this_opt == 'bloom_filter_fp_chance':
|
|
return [Hint('<float_between_0_and_1>')]
|
|
if this_opt in ('min_compaction_threshold', 'max_compaction_threshold',
|
|
'gc_grace_seconds', 'min_index_interval', 'max_index_interval'):
|
|
return [Hint('<integer>')]
|
|
if this_opt in ('cdc'):
|
|
return [Hint('<true|false>')]
|
|
if this_opt in ('read_repair'):
|
|
return [Hint('<\'none\'|\'blocking\'>')]
|
|
if this_opt == 'allow_auto_snapshot':
|
|
return [Hint('<boolean>')]
|
|
if this_opt == 'incremental_backups':
|
|
return [Hint('<boolean>')]
|
|
return [Hint('<option_value>')]
|
|
|
|
|
|
def cf_prop_val_mapkey_completer(ctxt, cass):
|
|
optname = ctxt.get_binding('propname')[-1]
|
|
for cql3option, _, subopts in CqlRuleSet.columnfamily_layout_map_options:
|
|
if optname == cql3option:
|
|
break
|
|
else:
|
|
return ()
|
|
keysseen = list(map(dequote_value, ctxt.get_binding('propmapkey', ())))
|
|
valsseen = list(map(dequote_value, ctxt.get_binding('propmapval', ())))
|
|
pairsseen = dict(list(zip(keysseen, valsseen)))
|
|
if optname == 'compression':
|
|
return list(map(escape_value, set(subopts).difference(keysseen)))
|
|
if optname == 'caching':
|
|
return list(map(escape_value, set(subopts).difference(keysseen)))
|
|
if optname == 'compaction':
|
|
opts = set(subopts)
|
|
try:
|
|
csc = pairsseen['class']
|
|
except KeyError:
|
|
return ["'class'"]
|
|
csc = csc.split('.')[-1]
|
|
if csc == 'SizeTieredCompactionStrategy':
|
|
opts = opts.union(set(CqlRuleSet.size_tiered_compaction_strategy_options))
|
|
elif csc == 'LeveledCompactionStrategy':
|
|
opts = opts.union(set(CqlRuleSet.leveled_compaction_strategy_options))
|
|
elif csc == 'TimeWindowCompactionStrategy':
|
|
opts = opts.union(set(CqlRuleSet.time_window_compaction_strategy_options))
|
|
elif csc == 'UnifiedCompactionStrategy':
|
|
opts = opts.union(set(CqlRuleSet.unified_compaction_strategy_options))
|
|
|
|
return list(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])
|
|
if opt == 'compaction':
|
|
if key == 'class':
|
|
return list(map(escape_value, CqlRuleSet.available_compaction_classes))
|
|
if key == 'provide_overlapping_tombstones':
|
|
return [Hint('<NONE|ROW|CELL>')]
|
|
return [Hint('<option_value>')]
|
|
elif opt == 'compression':
|
|
if key == 'class':
|
|
return list(map(escape_value, CqlRuleSet.available_compression_classes))
|
|
return [Hint('<option_value>')]
|
|
elif opt == 'caching':
|
|
if key == 'rows_per_partition':
|
|
return ["'ALL'", "'NONE'", Hint('#rows_per_partition')]
|
|
elif key == 'keys':
|
|
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 list(map(maybe_escape_name, cass.get_keyspace_names()))
|
|
|
|
|
|
@completer_for('nonSystemKeyspaceName', 'ksname')
|
|
def non_system_ks_name_completer(ctxt, cass):
|
|
ksnames = [n for n in cass.get_keyspace_names() if n not in SYSTEM_KEYSPACES]
|
|
return list(map(maybe_escape_name, ksnames))
|
|
|
|
|
|
@completer_for('alterableKeyspaceName', 'ksname')
|
|
def alterable_ks_name_completer(ctxt, cass):
|
|
ksnames = [n for n in cass.get_keyspace_names() if n not in NONALTERBALE_KEYSPACES]
|
|
return list(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)
|
|
completer_for('materializedViewName', '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('materializedViewName', 'dot')(cf_ks_dot_completer)
|
|
|
|
|
|
@completer_for('columnFamilyName', 'cfname')
|
|
def cf_name_completer(ctxt, cass):
|
|
ks = ctxt.get_binding('ksname', None)
|
|
if ks is not None:
|
|
ks = dequote_name(ks)
|
|
try:
|
|
cfnames = cass.get_columnfamily_names(ks)
|
|
except Exception:
|
|
if ks is None:
|
|
return ()
|
|
raise
|
|
return list(map(maybe_escape_name, cfnames))
|
|
|
|
|
|
@completer_for('materializedViewName', 'mvname')
|
|
def mv_name_completer(ctxt, cass):
|
|
ks = ctxt.get_binding('ksname', None)
|
|
if ks is not None:
|
|
ks = dequote_name(ks)
|
|
try:
|
|
mvnames = cass.get_materialized_view_names(ks)
|
|
except Exception:
|
|
if ks is None:
|
|
return ()
|
|
raise
|
|
return list(map(maybe_escape_name, mvnames))
|
|
|
|
|
|
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 list(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;
|
|
# this is always just to allow use of some keywords as column
|
|
# names, CF names, property values, etc.
|
|
return ()
|
|
|
|
|
|
def get_table_meta(ctxt, cass):
|
|
ks = ctxt.get_binding('ksname', None)
|
|
if ks is not None:
|
|
ks = dequote_name(ks)
|
|
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:
|
|
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'):
|
|
return True
|
|
return False
|
|
|
|
|
|
syntax_rules += r'''
|
|
<useStatement> ::= "USE" <keyspaceName>
|
|
;
|
|
<selectStatement> ::= "SELECT" ( "JSON" )? <selectClause>
|
|
"FROM" (cf=<columnFamilyName> | mv=<materializedViewName>)
|
|
( "WHERE" <whereClause> )?
|
|
( "GROUP" "BY" <groupByClause> ( "," <groupByClause> )* )?
|
|
( "ORDER" "BY" <orderByClause> ( "," <orderByClause> )* )?
|
|
( "PER" "PARTITION" "LIMIT" perPartitionLimit=<wholenumber> )?
|
|
( "LIMIT" limit=<wholenumber> )?
|
|
( "ALLOW" "FILTERING" )?
|
|
;
|
|
<whereClause> ::= <relation> ( "AND" <relation> )*
|
|
;
|
|
<relation> ::= [rel_lhs]=<cident> ( "[" <term> "]" )? ( "=" | "<" | ">" | "<=" | ">=" | "!=" | ( "NOT" )? "CONTAINS" ( "KEY" )? ) (<term> | <operandFunctions>)
|
|
| token="TOKEN" "(" [rel_tokname]=<cident>
|
|
( "," [rel_tokname]=<cident> )*
|
|
")" ("=" | "<" | ">" | "<=" | ">=") <tokenDefinition>
|
|
| [rel_lhs]=<cident> (( "NOT" )? "IN" ) "(" <term> ( "," <term> )* ")"
|
|
| [rel_lhs]=<cident> "BETWEEN" <term> "AND" <term>
|
|
| <operandFunctions>
|
|
;
|
|
<selectClause> ::= "DISTINCT"? <selector> ("AS" <cident>)? ("," <selector> ("AS" <cident>)?)*
|
|
| "*"
|
|
;
|
|
<udtSubfieldSelection> ::= <identifier> "." <identifier>
|
|
;
|
|
<selector> ::= [colname]=<cident> ( "[" ( <term> ( ".." <term> "]" )? | <term> ".." ) )?
|
|
| <udtSubfieldSelection>
|
|
| "CAST" "(" <selector> "AS" <storageType> ")"
|
|
| "TTL" "(" [colname]=<cident> ")"
|
|
| "TOKEN" "(" [colname]=<cident> ")"
|
|
| <aggregateMathFunctions>
|
|
| <scalarMathFunctions>
|
|
| <collectionFunctions>
|
|
| <currentTimeFunctions>
|
|
| <maskFunctions>
|
|
| <timeConversionFunctions>
|
|
| <writetimeFunctions>
|
|
| <functionName> <selectionFunctionArguments>
|
|
| <term>
|
|
;
|
|
|
|
<selectionFunctionArguments> ::= "(" ( <selector> ( "," <selector> )* )? ")"
|
|
;
|
|
<orderByClause> ::= [ordercol]=<cident> ( "ASC" | "DESC" )?
|
|
;
|
|
<groupByClause> ::= [groupcol]=<cident>
|
|
| <functionName><groupByFunctionArguments>
|
|
;
|
|
<groupByFunctionArguments> ::= "(" ( <groupByFunctionArgument> ( "," <groupByFunctionArgument> )* )? ")"
|
|
;
|
|
<groupByFunctionArgument> ::= [groupcol]=<cident>
|
|
| <term>
|
|
;
|
|
|
|
<aggregateMathFunctions> ::= "COUNT" "(" star=( "*" | "1" ) ")"
|
|
| "AVG" "(" [colname]=<cident> ")"
|
|
| "MIN" "(" [colname]=<cident> ")"
|
|
| "MAX" "(" [colname]=<cident> ")"
|
|
| "SUM" "(" [colname]=<cident> ")"
|
|
;
|
|
|
|
<scalarMathFunctions> ::= "ABS" "(" [colname]=<cident> ")"
|
|
| "EXP" "(" [colname]=<cident> ")"
|
|
| "LOG" "(" [colname]=<cident> ")"
|
|
| "LOG10" "(" [colname]=<cident> ")"
|
|
| "ROUND" "(" [colname]=<cident> ")"
|
|
;
|
|
|
|
<collectionFunctions> ::= "MAP_KEYS" "(" [colname]=<cident> ")"
|
|
| "MAP_VALUES" "(" [colname]=<cident> ")"
|
|
| "COLLECTION_AVG" "(" [colname]=<cident> ")"
|
|
| "COLLECTION_COUNT" "(" [colname]=<cident> ")"
|
|
| "COLLECTION_MIN" "(" [colname]=<cident> ")"
|
|
| "COLLECTION_MAX" "(" [colname]=<cident> ")"
|
|
| "COLLECTION_SUM" "(" [colname]=<cident> ")"
|
|
;
|
|
|
|
<currentTimeFunctions> ::= "CURRENT_DATE()"
|
|
| "CURRENT_TIME()"
|
|
| "CURRENT_TIMESTAMP()"
|
|
| "CURRENT_TIMEUUID()"
|
|
;
|
|
|
|
<maskFunctions> ::= "MASK_DEFAULT" "(" [colname]=<cident> ")"
|
|
| "MASK_HASH" "(" [colname]=<cident> ")"
|
|
| "MASK_INNER" "(" [colname]=<cident> "," <wholenumber> "," <wholenumber> ")"
|
|
| "MASK_NULL" "(" [colname]=<cident> ")"
|
|
| "MASK_REPLACE" "(" [colname]=<cident> "," <propertyValue> ")"
|
|
| "MASK_OUTER" "(" [colname]=<cident> "," <wholenumber> "," <wholenumber> ")"
|
|
;
|
|
|
|
<timeConversionFunctions> ::= "TO_DATE" "(" [colname]=<cident> ")"
|
|
| "TO_TIMESTAMP" "(" [colname]=<cident> ")"
|
|
| "TO_UNIX_TIMESTAMP" "(" [colname]=<cident> ")"
|
|
;
|
|
|
|
<timeuuidFunctions> ::= "MAX_TIMEUUID" "(" [colname]=<cident> ")"
|
|
| "MIN_TIMEUUID" "(" [colname]=<cident> ")"
|
|
;
|
|
|
|
<writetimeFunctions> ::= "MAX_WRITETIME" "(" [colname]=<cident> ")"
|
|
| "MIN_WRITETIME" "(" [colname]=<cident> ")"
|
|
| "WRITETIME" "(" [colname]=<cident> ")"
|
|
;
|
|
<operandFunctions> ::= <currentTimeFunctions> | <timeuuidFunctions>
|
|
;
|
|
|
|
'''
|
|
|
|
|
|
def udf_name_completer(ctxt, cass):
|
|
ks = ctxt.get_binding('ksname', None)
|
|
if ks is not None:
|
|
ks = dequote_name(ks)
|
|
try:
|
|
udfnames = cass.get_userfunction_names(ks)
|
|
except Exception:
|
|
if ks is None:
|
|
return ()
|
|
raise
|
|
return list(map(maybe_escape_name, udfnames))
|
|
|
|
|
|
def uda_name_completer(ctxt, cass):
|
|
ks = ctxt.get_binding('ksname', None)
|
|
if ks is not None:
|
|
ks = dequote_name(ks)
|
|
try:
|
|
udanames = cass.get_useraggregate_names(ks)
|
|
except Exception:
|
|
if ks is None:
|
|
return ()
|
|
raise
|
|
return list(map(maybe_escape_name, udanames))
|
|
|
|
|
|
def udf_uda_name_completer(ctxt, cass):
|
|
ks = ctxt.get_binding('ksname', None)
|
|
if ks is not None:
|
|
ks = dequote_name(ks)
|
|
try:
|
|
functionnames = cass.get_userfunction_names(ks) + cass.get_useraggregate_names(ks)
|
|
except Exception:
|
|
if ks is None:
|
|
return ()
|
|
raise
|
|
return list(map(maybe_escape_name, functionnames))
|
|
|
|
|
|
def ref_udf_name_completer(ctxt, cass):
|
|
try:
|
|
udanames = cass.get_userfunction_names(None)
|
|
except Exception:
|
|
return ()
|
|
return list(map(maybe_escape_name, udanames))
|
|
|
|
|
|
completer_for('functionAggregateName', 'ksname')(cf_ks_name_completer)
|
|
completer_for('functionAggregateName', 'dot')(cf_ks_dot_completer)
|
|
completer_for('functionAggregateName', 'functionname')(udf_uda_name_completer)
|
|
completer_for('anyFunctionName', 'ksname')(cf_ks_name_completer)
|
|
completer_for('anyFunctionName', 'dot')(cf_ks_dot_completer)
|
|
completer_for('anyFunctionName', 'udfname')(udf_name_completer)
|
|
completer_for('userFunctionName', 'ksname')(cf_ks_name_completer)
|
|
completer_for('userFunctionName', 'dot')(cf_ks_dot_completer)
|
|
completer_for('userFunctionName', 'udfname')(udf_name_completer)
|
|
completer_for('refUserFunctionName', 'udfname')(ref_udf_name_completer)
|
|
completer_for('userAggregateName', 'ksname')(cf_ks_name_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', ())
|
|
keyname = ctxt.get_binding('keyname')
|
|
if keyname is None:
|
|
keyname = ctxt.get_binding('rel_lhs', ())
|
|
if not keyname:
|
|
return [Hint("Can't ORDER BY here: need to specify partition key in WHERE clause")]
|
|
layout = get_table_meta(ctxt, cass)
|
|
order_by_candidates = [col.name for col in layout.clustering_key]
|
|
if len(order_by_candidates) > len(prev_order_cols):
|
|
return [maybe_escape_name(order_by_candidates[len(prev_order_cols)])]
|
|
return [Hint('No more orderable columns here.')]
|
|
|
|
|
|
@completer_for('groupByClause', 'groupcol')
|
|
def select_group_column_completer(ctxt, cass):
|
|
prev_group_cols = ctxt.get_binding('groupcol', ())
|
|
layout = get_table_meta(ctxt, cass)
|
|
group_by_candidates = [col.name for col in layout.primary_key]
|
|
if len(group_by_candidates) > len(prev_group_cols):
|
|
return [maybe_escape_name(group_by_candidates[len(prev_group_cols)])]
|
|
return [Hint('No more 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)
|
|
filterable = set()
|
|
already_filtered_on = list(map(dequote_name, ctxt.get_binding('rel_lhs', ())))
|
|
for num in range(0, len(layout.partition_key)):
|
|
if num == 0 or layout.partition_key[num - 1].name in already_filtered_on:
|
|
filterable.add(layout.partition_key[num].name)
|
|
else:
|
|
break
|
|
for num in range(0, len(layout.clustering_key)):
|
|
if num == 0 or layout.clustering_key[num - 1].name in already_filtered_on:
|
|
filterable.add(layout.clustering_key[num].name)
|
|
else:
|
|
break
|
|
for idx in layout.indexes.values():
|
|
filterable.add(idx.index_options["target"])
|
|
return list(map(maybe_escape_name, filterable))
|
|
|
|
|
|
explain_completion('selector', 'colname')
|
|
|
|
syntax_rules += r'''
|
|
<insertStatement> ::= "INSERT" "INTO" cf=<columnFamilyName>
|
|
( ( "(" [colname]=<cident> ( "," [colname]=<cident> )* ")"
|
|
"VALUES" "(" [newval]=<term> ( valcomma="," [newval]=<term> )* valcomma=")")
|
|
| ("JSON" <stringLiteral>))
|
|
( "IF" "NOT" "EXISTS")?
|
|
( "USING" [insertopt]=<usingOption>
|
|
( "AND" [insertopt]=<usingOption> )* )?
|
|
;
|
|
<usingOption> ::= "TIMESTAMP" <wholenumber>
|
|
| "TTL" <wholenumber>
|
|
;
|
|
'''
|
|
|
|
|
|
def regular_column_names(table_meta):
|
|
if not table_meta or not table_meta.columns:
|
|
return []
|
|
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):
|
|
layout = get_table_meta(ctxt, cass)
|
|
colnames = set(map(dequote_name, ctxt.get_binding('colname', ())))
|
|
keycols = layout.primary_key
|
|
for k in keycols:
|
|
if k.name not in colnames:
|
|
return [maybe_escape_name(k.name)]
|
|
normalcols = set(regular_column_names(layout)) - colnames
|
|
return list(map(maybe_escape_name, normalcols))
|
|
|
|
|
|
@completer_for('insertStatement', 'newval')
|
|
def insert_newval_completer(ctxt, cass):
|
|
layout = get_table_meta(ctxt, cass)
|
|
insertcols = list(map(dequote_name, ctxt.get_binding('colname')))
|
|
valuesdone = ctxt.get_binding('newval', ())
|
|
if len(valuesdone) >= len(insertcols):
|
|
return []
|
|
curcol = insertcols[len(valuesdone)]
|
|
coltype = layout.columns[curcol].cql_type
|
|
if coltype.startswith('map<') or coltype.startswith('set<'):
|
|
return ['{']
|
|
if coltype.startswith('list<') or coltype.startswith('vector<'):
|
|
return ['[']
|
|
if coltype == 'boolean':
|
|
return ['true', 'false']
|
|
|
|
return [Hint('<value for %s (%s)>' % (maybe_escape_name(curcol),
|
|
coltype))]
|
|
|
|
|
|
@completer_for('insertStatement', 'valcomma')
|
|
def insert_valcomma_completer(ctxt, cass):
|
|
numcols = len(ctxt.get_binding('colname', ()))
|
|
numvals = len(ctxt.get_binding('newval', ()))
|
|
if numcols > numvals:
|
|
return [',']
|
|
return [')']
|
|
|
|
|
|
@completer_for('insertStatement', 'insertopt')
|
|
def insert_option_completer(ctxt, cass):
|
|
opts = set('TIMESTAMP TTL'.split())
|
|
for opt in ctxt.get_binding('insertopt', ()):
|
|
opts.discard(opt.split()[0])
|
|
return opts
|
|
|
|
|
|
syntax_rules += r'''
|
|
<updateStatement> ::= "UPDATE" cf=<columnFamilyName>
|
|
( "USING" [updateopt]=<usingOption>
|
|
( "AND" [updateopt]=<usingOption> )* )?
|
|
"SET" <assignment> ( "," <assignment> )*
|
|
"WHERE" <whereClause>
|
|
( "IF" ( "EXISTS" | <conditions> ))?
|
|
;
|
|
<assignment> ::= updatecol=<cident>
|
|
(( "=" update_rhs=( <term> | <cident> )
|
|
( counterop=( "+" | "-" ) inc=<wholenumber>
|
|
| listadder="+" listcol=<cident> )? )
|
|
| ( indexbracket="[" <term> "]" "=" <term> )
|
|
| ( udt_field_dot="." udt_field=<identifier> "=" <term> ))
|
|
;
|
|
<conditions> ::= <condition> ( "AND" <condition> )*
|
|
;
|
|
<condition_op_and_rhs> ::= (("=" | "<" | ">" | "<=" | ">=" | "!=" | "CONTAINS" ( "KEY" )? ) <term>)
|
|
| ("IN" "(" <term> ( "," <term> )* ")" )
|
|
;
|
|
<condition> ::= conditioncol=<cident>
|
|
( (( indexbracket="[" <term> "]" )
|
|
|( udt_field_dot="." udt_field=<identifier> )) )?
|
|
<condition_op_and_rhs>
|
|
;
|
|
'''
|
|
|
|
|
|
@completer_for('updateStatement', 'updateopt')
|
|
def update_option_completer(ctxt, cass):
|
|
opts = {'TIMESTAMP', 'TTL'}
|
|
for opt in ctxt.get_binding('updateopt', ()):
|
|
opts.discard(opt.split()[0])
|
|
return opts
|
|
|
|
|
|
@completer_for('assignment', 'updatecol')
|
|
def update_col_completer(ctxt, cass):
|
|
layout = get_table_meta(ctxt, cass)
|
|
return list(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)
|
|
curcol = dequote_name(ctxt.get_binding('updatecol', ''))
|
|
coltype = layout.columns[curcol].cql_type
|
|
if coltype == 'counter':
|
|
return [maybe_escape_name(curcol)]
|
|
if coltype.startswith('map<') or coltype.startswith('set<'):
|
|
return ['{']
|
|
if coltype.startswith('list<') or coltype.startswith('vector<'):
|
|
return ['[']
|
|
return [Hint('<term (%s)>' % coltype)]
|
|
|
|
|
|
@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].cql_type == 'counter' else []
|
|
|
|
|
|
@completer_for('assignment', 'inc')
|
|
def update_counter_inc_completer(ctxt, cass):
|
|
layout = get_table_meta(ctxt, cass)
|
|
curcol = dequote_name(ctxt.get_binding('updatecol', ''))
|
|
if layout.columns[curcol].cql_type == 'counter':
|
|
return [Hint('<wholenumber>')]
|
|
return []
|
|
|
|
|
|
@completer_for('assignment', 'listadder')
|
|
def update_listadder_completer(ctxt, cass):
|
|
rhs = ctxt.get_binding('update_rhs')
|
|
if rhs.startswith('['):
|
|
return ['+']
|
|
return []
|
|
|
|
|
|
@completer_for('assignment', 'listcol')
|
|
def update_listcol_completer(ctxt, cass):
|
|
rhs = ctxt.get_binding('update_rhs')
|
|
if rhs.startswith('['):
|
|
colname = dequote_name(ctxt.get_binding('updatecol'))
|
|
return [maybe_escape_name(colname)]
|
|
return []
|
|
|
|
|
|
@completer_for('assignment', 'indexbracket')
|
|
def update_indexbracket_completer(ctxt, cass):
|
|
layout = get_table_meta(ctxt, cass)
|
|
curcol = dequote_name(ctxt.get_binding('updatecol', ''))
|
|
coltype = layout.columns[curcol].cql_type
|
|
if coltype in ('map', 'list'):
|
|
return ['[']
|
|
return []
|
|
|
|
|
|
@completer_for('assignment', 'udt_field_dot')
|
|
def update_udt_field_dot_completer(ctxt, cass):
|
|
layout = get_table_meta(ctxt, cass)
|
|
curcol = dequote_name(ctxt.get_binding('updatecol', ''))
|
|
return ["."] if _is_usertype(layout, curcol) else []
|
|
|
|
|
|
@completer_for('assignment', 'udt_field')
|
|
def assignment_udt_field_completer(ctxt, cass):
|
|
layout = get_table_meta(ctxt, cass)
|
|
curcol = dequote_name(ctxt.get_binding('updatecol', ''))
|
|
return _usertype_fields(ctxt, cass, layout, curcol)
|
|
|
|
|
|
def _is_usertype(layout, curcol):
|
|
coltype = layout.columns[curcol].cql_type
|
|
return coltype not in simple_cql_types and coltype not in ('map', 'set', 'list', 'vector')
|
|
|
|
|
|
def _usertype_fields(ctxt, cass, layout, curcol):
|
|
if not _is_usertype(layout, curcol):
|
|
return []
|
|
|
|
coltype = layout.columns[curcol].cql_type
|
|
ks = ctxt.get_binding('ksname', None)
|
|
if ks is not None:
|
|
ks = dequote_name(ks)
|
|
user_type = cass.get_usertype_layout(ks, coltype)
|
|
return [field_name for (field_name, field_type) in user_type]
|
|
|
|
|
|
@completer_for('condition', 'indexbracket')
|
|
def condition_indexbracket_completer(ctxt, cass):
|
|
layout = get_table_meta(ctxt, cass)
|
|
curcol = dequote_name(ctxt.get_binding('conditioncol', ''))
|
|
coltype = layout.columns[curcol].cql_type
|
|
if coltype in ('map', 'list'):
|
|
return ['[']
|
|
return []
|
|
|
|
|
|
@completer_for('condition', 'udt_field_dot')
|
|
def condition_udt_field_dot_completer(ctxt, cass):
|
|
layout = get_table_meta(ctxt, cass)
|
|
curcol = dequote_name(ctxt.get_binding('conditioncol', ''))
|
|
return ["."] if _is_usertype(layout, curcol) else []
|
|
|
|
|
|
@completer_for('condition', 'udt_field')
|
|
def condition_udt_field_completer(ctxt, cass):
|
|
layout = get_table_meta(ctxt, cass)
|
|
curcol = dequote_name(ctxt.get_binding('conditioncol', ''))
|
|
return _usertype_fields(ctxt, cass, layout, curcol)
|
|
|
|
|
|
syntax_rules += r'''
|
|
<deleteStatement> ::= "DELETE" ( <deleteSelector> ( "," <deleteSelector> )* )?
|
|
"FROM" cf=<columnFamilyName>
|
|
( "USING" [delopt]=<deleteOption> )?
|
|
"WHERE" <whereClause>
|
|
( "IF" ( "EXISTS" | <conditions> ) )?
|
|
;
|
|
<deleteSelector> ::= delcol=<cident>
|
|
( ( "[" <term> "]" )
|
|
| ( "." <identifier> ) )?
|
|
;
|
|
<deleteOption> ::= "TIMESTAMP" <wholenumber>
|
|
;
|
|
'''
|
|
|
|
|
|
@completer_for('deleteStatement', 'delopt')
|
|
def delete_opt_completer(ctxt, cass):
|
|
opts = set('TIMESTAMP'.split())
|
|
for opt in ctxt.get_binding('delopt', ()):
|
|
opts.discard(opt.split()[0])
|
|
return opts
|
|
|
|
|
|
@completer_for('deleteSelector', 'delcol')
|
|
def delete_delcol_completer(ctxt, cass):
|
|
layout = get_table_meta(ctxt, cass)
|
|
return list(map(maybe_escape_name, regular_column_names(layout)))
|
|
|
|
|
|
syntax_rules += r'''
|
|
<batchStatement> ::= "BEGIN" ( "UNLOGGED" | "COUNTER" )? "BATCH"
|
|
( "USING" [batchopt]=<usingOption>
|
|
( "AND" [batchopt]=<usingOption> )* )?
|
|
[batchstmt]=<batchStatementMember> ";"?
|
|
( [batchstmt]=<batchStatementMember> ";"? )*
|
|
"APPLY" "BATCH"
|
|
;
|
|
<batchStatementMember> ::= <insertStatement>
|
|
| <updateStatement>
|
|
| <deleteStatement>
|
|
;
|
|
'''
|
|
|
|
|
|
@completer_for('batchStatement', 'batchopt')
|
|
def batch_opt_completer(ctxt, cass):
|
|
opts = set('TIMESTAMP'.split())
|
|
for opt in ctxt.get_binding('batchopt', ()):
|
|
opts.discard(opt.split()[0])
|
|
return opts
|
|
|
|
|
|
syntax_rules += r'''
|
|
<truncateStatement> ::= "TRUNCATE" ("COLUMNFAMILY" | "TABLE")? cf=<columnFamilyName>
|
|
;
|
|
'''
|
|
|
|
syntax_rules += r'''
|
|
<createKeyspaceStatement> ::= "CREATE" wat=( "KEYSPACE" | "SCHEMA" ) ("IF" "NOT" "EXISTS")? ksname=<cfOrKsName>
|
|
"WITH" <property> ( "AND" <property> )*
|
|
;
|
|
'''
|
|
|
|
|
|
@completer_for('createKeyspaceStatement', 'wat')
|
|
def create_ks_wat_completer(ctxt, cass):
|
|
# would prefer to get rid of the "schema" nomenclature in cql3
|
|
if ctxt.get_binding('partial', '') == '':
|
|
return ['KEYSPACE']
|
|
return ['KEYSPACE', 'SCHEMA']
|
|
|
|
|
|
syntax_rules += r'''
|
|
<createColumnFamilyStatement> ::= "CREATE" wat=( "COLUMNFAMILY" | "TABLE" ) ("IF" "NOT" "EXISTS")?
|
|
( ks=<nonSystemKeyspaceName> dot="." )? cf=<cfOrKsName>
|
|
"(" ( <singleKeyCfSpec> | <compositeKeyCfSpec> ) ")"
|
|
( "WITH" <cfamProperty> ( "AND" <cfamProperty> )* )?
|
|
;
|
|
|
|
<cfamProperty> ::= <property>
|
|
| "COMPACT" "STORAGE" "CDC"
|
|
| "CLUSTERING" "ORDER" "BY" "(" <cfamOrdering>
|
|
( "," <cfamOrdering> )* ")"
|
|
;
|
|
|
|
<cfamOrdering> ::= [ordercol]=<cident> ( "ASC" | "DESC" )
|
|
;
|
|
|
|
<singleKeyCfSpec> ::= [newcolname]=<cident> <storageType> "PRIMARY" "KEY"
|
|
( "," [newcolname]=<cident> <storageType> )*
|
|
;
|
|
|
|
<compositeKeyCfSpec> ::= [newcolname]=<cident> <storageType>
|
|
"," [newcolname]=<cident> <storageType> ( "static" )?
|
|
( "," [newcolname]=<cident> <storageType> ( "static" )? )*
|
|
"," "PRIMARY" k="KEY" p="(" ( partkey=<pkDef> | [pkey]=<cident> )
|
|
( c="," [pkey]=<cident> )* ")"
|
|
;
|
|
|
|
<pkDef> ::= "(" [ptkey]=<cident> "," [ptkey]=<cident>
|
|
( "," [ptkey]=<cident> )* ")"
|
|
;
|
|
'''
|
|
|
|
|
|
@completer_for('cfamOrdering', 'ordercol')
|
|
def create_cf_clustering_order_colname_completer(ctxt, cass):
|
|
colnames = list(map(dequote_name, ctxt.get_binding('newcolname', ())))
|
|
# Definitely some of these aren't valid for ordering, but I'm not sure
|
|
# 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
|
|
if ctxt.get_binding('partial', '') == '':
|
|
return ['TABLE']
|
|
return ['TABLE', 'COLUMNFAMILY']
|
|
|
|
|
|
explain_completion('createColumnFamilyStatement', 'cf', '<new_table_name>')
|
|
explain_completion('compositeKeyCfSpec', 'newcolname', '<new_column_name>')
|
|
|
|
|
|
@completer_for('createColumnFamilyStatement', 'dot')
|
|
def create_cf_ks_dot_completer(ctxt, cass):
|
|
ks = dequote_name(ctxt.get_binding('ks'))
|
|
if ks in cass.get_keyspace_names():
|
|
return ['.']
|
|
return []
|
|
|
|
|
|
@completer_for('pkDef', 'ptkey')
|
|
def create_cf_pkdef_declaration_completer(ctxt, cass):
|
|
cols_declared = ctxt.get_binding('newcolname')
|
|
pieces_already = ctxt.get_binding('ptkey', ())
|
|
pieces_already = list(map(dequote_name, pieces_already))
|
|
while cols_declared[0] in pieces_already:
|
|
cols_declared = cols_declared[1:]
|
|
if len(cols_declared) < 2:
|
|
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')
|
|
pieces_already = ctxt.get_binding('ptkey', ()) + ctxt.get_binding('pkey', ())
|
|
pieces_already = list(map(dequote_name, pieces_already))
|
|
while cols_declared[0] in pieces_already:
|
|
cols_declared = cols_declared[1:]
|
|
if len(cols_declared) < 2:
|
|
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')
|
|
pieces_already = ctxt.get_binding('pkey', ())
|
|
if len(pieces_already) >= len(cols_declared) - 1:
|
|
return ()
|
|
return [',']
|
|
|
|
|
|
syntax_rules += r'''
|
|
<copyTableStatement> ::= "CREATE" wat=("COLUMNFAMILY" | "TABLE" ) ("IF" "NOT" "EXISTS")?
|
|
( tks=<nonSystemKeyspaceName> dot="." )? tcf=<cfOrKsName>
|
|
"LIKE" ( sks=<nonSystemKeyspaceName> dot="." )? scf=<cfOrKsName>
|
|
( "WITH" <propertyOrOption> ( "AND" <propertyOrOption> )* )?
|
|
;
|
|
'''
|
|
|
|
|
|
@completer_for('copyTableStatement', 'wat')
|
|
def create_tb_wat_completer(ctxt, cass):
|
|
# would prefer to get rid of the "schema" nomenclature in cql3
|
|
if ctxt.get_binding('partial', '') == '':
|
|
return ['TABLE']
|
|
return ['COLUMNFAMILY', 'TABLE']
|
|
|
|
|
|
explain_completion('copyTableStatement', 'tcf', '<new_table_name>')
|
|
explain_completion('copyTableStatement', 'scf', '<old_table_name>')
|
|
|
|
|
|
syntax_rules += r'''
|
|
|
|
<idxName> ::= <identifier>
|
|
| <quotedName>
|
|
| <unreservedKeyword>;
|
|
|
|
<createIndexStatement> ::= "CREATE" "CUSTOM"? "INDEX" ("IF" "NOT" "EXISTS")? indexname=<idxName>? "ON"
|
|
cf=<columnFamilyName> "(" (
|
|
col=<cident> |
|
|
"keys(" col=<cident> ")" |
|
|
"full(" col=<cident> ")"
|
|
) ")"
|
|
( "USING" <stringLiteral> ( "WITH" "OPTIONS" "=" <mapLiteral> )? )?
|
|
;
|
|
|
|
|
|
<colList> ::= "(" <cident> ( "," <cident> )* ")"
|
|
;
|
|
|
|
<createMaterializedViewStatement> ::= "CREATE" wat="MATERIALIZED" "VIEW" ("IF" "NOT" "EXISTS")? viewname=<materializedViewName>?
|
|
"AS" "SELECT" <selectClause>
|
|
"FROM" cf=<columnFamilyName>
|
|
"WHERE" <cident> "IS" "NOT" "NULL" ( "AND" <cident> "IS" "NOT" "NULL")*
|
|
"PRIMARY" "KEY" (<colList> | ( "(" <colList> ( "," <cident> )* ")" ))
|
|
( "WITH" <cfamProperty> ( "AND" <cfamProperty> )* )?
|
|
;
|
|
|
|
<createUserTypeStatement> ::= "CREATE" "TYPE" ("IF" "NOT" "EXISTS")? ( ks=<nonSystemKeyspaceName> dot="." )? typename=<cfOrKsName> "(" newcol=<cident> <storageType>
|
|
( "," [newcolname]=<cident> <storageType> )*
|
|
")"
|
|
;
|
|
|
|
<createFunctionStatement> ::= "CREATE" ("OR" "REPLACE")? "FUNCTION"
|
|
("IF" "NOT" "EXISTS")?
|
|
<userFunctionName>
|
|
( "(" ( newcol=<cident> <storageType>
|
|
( "," [newcolname]=<cident> <storageType> )* )?
|
|
")" )?
|
|
("RETURNS" "NULL" | "CALLED") "ON" "NULL" "INPUT"
|
|
"RETURNS" <storageType>
|
|
"LANGUAGE" <cident> "AS" <stringLiteral>
|
|
;
|
|
|
|
<createAggregateStatement> ::= "CREATE" ("OR" "REPLACE")? "AGGREGATE"
|
|
("IF" "NOT" "EXISTS")?
|
|
<userAggregateName>
|
|
( "("
|
|
( <storageType> ( "," <storageType> )* )?
|
|
")" )?
|
|
"SFUNC" <refUserFunctionName>
|
|
"STYPE" <storageType>
|
|
( "FINALFUNC" <refUserFunctionName> )?
|
|
( "INITCOND" <term> )?
|
|
;
|
|
|
|
'''
|
|
|
|
explain_completion('createIndexStatement', 'indexname', '<new_index_name>')
|
|
explain_completion('createMaterializedViewStatement', 'viewname', '<new_view_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):
|
|
""" Return the columns for which an index doesn't exist yet. """
|
|
layout = get_table_meta(ctxt, cass)
|
|
idx_targets = [idx.index_options["target"] for idx in layout.indexes.values()]
|
|
colnames = [cd.name for cd in list(layout.columns.values()) if cd.name not in idx_targets]
|
|
return list(map(maybe_escape_name, colnames))
|
|
|
|
|
|
syntax_rules += r'''
|
|
<dropKeyspaceStatement> ::= "DROP" "KEYSPACE" ("IF" "EXISTS")? ksname=<nonSystemKeyspaceName>
|
|
;
|
|
|
|
<dropColumnFamilyStatement> ::= "DROP" ( "COLUMNFAMILY" | "TABLE" ) ("IF" "EXISTS")? cf=<columnFamilyName>
|
|
;
|
|
|
|
<indexName> ::= ( ksname=<idxOrKsName> dot="." )? idxname=<idxOrKsName> ;
|
|
|
|
<idxOrKsName> ::= <identifier>
|
|
| <quotedName>
|
|
| <unreservedKeyword>;
|
|
|
|
<dropIndexStatement> ::= "DROP" "INDEX" ("IF" "EXISTS")? idx=<indexName>
|
|
;
|
|
|
|
<dropMaterializedViewStatement> ::= "DROP" "MATERIALIZED" "VIEW" ("IF" "EXISTS")? mv=<materializedViewName>
|
|
;
|
|
|
|
<dropUserTypeStatement> ::= "DROP" "TYPE" ( "IF" "EXISTS" )? ut=<userTypeName>
|
|
;
|
|
|
|
<dropFunctionStatement> ::= "DROP" "FUNCTION" ( "IF" "EXISTS" )? <userFunctionName>
|
|
;
|
|
|
|
<dropAggregateStatement> ::= "DROP" "AGGREGATE" ( "IF" "EXISTS" )? <userAggregateName>
|
|
;
|
|
|
|
'''
|
|
|
|
|
|
@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'))
|
|
if name in cass.get_keyspace_names():
|
|
return ['.']
|
|
return []
|
|
|
|
|
|
@completer_for('indexName', 'idxname')
|
|
def idx_ks_idx_name_completer(ctxt, cass):
|
|
ks = ctxt.get_binding('ksname', None)
|
|
if ks is not None:
|
|
ks = dequote_name(ks)
|
|
try:
|
|
idxnames = cass.get_index_names(ks)
|
|
except Exception:
|
|
if ks is None:
|
|
return ()
|
|
raise
|
|
return list(map(maybe_escape_name, idxnames))
|
|
|
|
|
|
syntax_rules += r'''
|
|
<alterTableStatement> ::= "ALTER" wat=( "COLUMNFAMILY" | "TABLE" ) ("IF" "EXISTS")? cf=<columnFamilyName>
|
|
<alterInstructions>
|
|
;
|
|
<alterInstructions> ::= "ADD" ("IF" "NOT" "EXISTS")? newcol=<cident> <storageType> ("static")?
|
|
| "DROP" ("IF" "EXISTS")? existcol=<cident>
|
|
| "WITH" <cfamProperty> ( "AND" <cfamProperty> )*
|
|
| "RENAME" ("IF" "EXISTS")? existcol=<cident> "TO" newcol=<cident>
|
|
( "AND" existcol=<cident> "TO" newcol=<cident> )*
|
|
| "ALTER" ("IF" "EXISTS")? existcol=<cident> ( <constraintsExpr> | <column_mask> | "DROP" ( "CHECK" | "MASKED" ) )
|
|
;
|
|
|
|
<alterUserTypeStatement> ::= "ALTER" "TYPE" ("IF" "EXISTS")? ut=<userTypeName>
|
|
<alterTypeInstructions>
|
|
;
|
|
<alterTypeInstructions> ::= "ADD" ("IF" "NOT" "EXISTS")? newcol=<cident> <storageType>
|
|
| "RENAME" ("IF" "EXISTS")? existcol=<cident> "TO" newcol=<cident>
|
|
( "AND" existcol=<cident> "TO" newcol=<cident> )*
|
|
;
|
|
'''
|
|
|
|
|
|
@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 list(map(maybe_escape_name, cols))
|
|
|
|
|
|
@completer_for('alterTypeInstructions', 'existcol')
|
|
def alter_type_field_completer(ctxt, cass):
|
|
layout = get_ut_layout(ctxt, cass)
|
|
fields = [atuple[0] for atuple in layout]
|
|
return list(map(maybe_escape_name, fields))
|
|
|
|
|
|
explain_completion('alterInstructions', 'newcol', '<new_column_name>')
|
|
explain_completion('alterTypeInstructions', 'newcol', '<new_field_name>')
|
|
|
|
|
|
syntax_rules += r'''
|
|
<alterKeyspaceStatement> ::= "ALTER" wat=( "KEYSPACE" | "SCHEMA" ) ("IF" "EXISTS")? ks=<alterableKeyspaceName>
|
|
"WITH" <property> ( "AND" <property> )*
|
|
;
|
|
'''
|
|
|
|
syntax_rules += r'''
|
|
<username> ::= name=( <identifier> | <stringLiteral> )
|
|
;
|
|
|
|
<createUserStatement> ::= "CREATE" "USER" ( "IF" "NOT" "EXISTS" )? <username>
|
|
( ("WITH" ("HASHED")? "PASSWORD" <stringLiteral>) | ("WITH" "GENERATED" "PASSWORD") )?
|
|
( "SUPERUSER" | "NOSUPERUSER" )?
|
|
;
|
|
|
|
<alterUserStatement> ::= "ALTER" "USER" ("IF" "EXISTS")? <username>
|
|
( ("WITH" "PASSWORD" <stringLiteral>) | ("WITH" "GENERATED" "PASSWORD") )?
|
|
( "SUPERUSER" | "NOSUPERUSER" )?
|
|
;
|
|
|
|
<dropUserStatement> ::= "DROP" "USER" ( "IF" "EXISTS" )? <username>
|
|
;
|
|
|
|
<listUsersStatement> ::= "LIST" "USERS"
|
|
;
|
|
'''
|
|
|
|
syntax_rules += r'''
|
|
<rolename> ::= role=( <identifier>
|
|
| <quotedName>
|
|
| <unreservedKeyword> )
|
|
;
|
|
|
|
<createRoleStatement> ::= "CREATE" "ROLE" ("IF" "NOT" "EXISTS")? <rolename>
|
|
( "WITH" <roleProperty> ("AND" <roleProperty>)*)?
|
|
;
|
|
|
|
<alterRoleStatement> ::= "ALTER" "ROLE" ("IF" "EXISTS")? <rolename>
|
|
( "WITH" <roleProperty> ("AND" <roleProperty>)*)
|
|
;
|
|
|
|
<roleProperty> ::= (("HASHED")? "PASSWORD") "=" <stringLiteral>
|
|
| "GENERATED" "PASSWORD"
|
|
| "OPTIONS" "=" <mapLiteral>
|
|
| "SUPERUSER" "=" <boolean>
|
|
| "LOGIN" "=" <boolean>
|
|
| "ACCESS" "TO" "DATACENTERS" <setLiteral>
|
|
| "ACCESS" "TO" "ALL" "DATACENTERS"
|
|
| "ACCESS" "FROM" "CIDRS" <setLiteral>
|
|
| "ACCESS" "FROM" "ALL" "CIDRS"
|
|
;
|
|
|
|
<dropRoleStatement> ::= "DROP" "ROLE" ("IF" "EXISTS")? <rolename>
|
|
;
|
|
|
|
<grantRoleStatement> ::= "GRANT" <rolename> "TO" <rolename>
|
|
;
|
|
|
|
<revokeRoleStatement> ::= "REVOKE" <rolename> "FROM" <rolename>
|
|
;
|
|
|
|
<listRolesStatement> ::= "LIST" "ROLES"
|
|
( "OF" <rolename> )? "NORECURSIVE"?
|
|
;
|
|
|
|
<listSuperUsersStatement> ::= "LIST" "SUPERUSERS"
|
|
;
|
|
'''
|
|
|
|
syntax_rules += r'''
|
|
<grantStatement> ::= "GRANT" <permissionExpr> "ON" <resource> "TO" <rolename>
|
|
;
|
|
|
|
<revokeStatement> ::= "REVOKE" <permissionExpr> "ON" <resource> "FROM" <rolename>
|
|
;
|
|
|
|
<listPermissionsStatement> ::= "LIST" <permissionExpr>
|
|
( "ON" <resource> )? ( "OF" <rolename> )? "NORECURSIVE"?
|
|
;
|
|
|
|
<permission> ::= "AUTHORIZE"
|
|
| "CREATE"
|
|
| "ALTER"
|
|
| "DROP"
|
|
| "SELECT"
|
|
| "MODIFY"
|
|
| "DESCRIBE"
|
|
| "EXECUTE"
|
|
| "UNMASK"
|
|
| "SELECT_MASKED"
|
|
;
|
|
|
|
<permissionExpr> ::= ( [newpermission]=<permission> "PERMISSION"? ( "," [newpermission]=<permission> "PERMISSION"? )* )
|
|
| ( "ALL" "PERMISSIONS"? )
|
|
;
|
|
|
|
<resource> ::= <dataResource>
|
|
| <roleResource>
|
|
| <functionResource>
|
|
| <jmxResource>
|
|
;
|
|
|
|
<dataResource> ::= ( "ALL" "KEYSPACES" )
|
|
| ( "KEYSPACE" <keyspaceName> )
|
|
| ( "ALL" "TABLES" "IN" "KEYSPACE" <keyspaceName> )
|
|
| ( "TABLE"? <columnFamilyName> )
|
|
;
|
|
|
|
<roleResource> ::= ("ALL" "ROLES")
|
|
| ("ROLE" <rolename>)
|
|
;
|
|
|
|
<functionResource> ::= ( "ALL" "FUNCTIONS" ("IN KEYSPACE" <keyspaceName>)? )
|
|
| ( "FUNCTION" <functionAggregateName>
|
|
( "(" ( newcol=<cident> <storageType>
|
|
( "," [newcolname]=<cident> <storageType> )* )?
|
|
")" )
|
|
)
|
|
;
|
|
|
|
<jmxResource> ::= ( "ALL" "MBEANS")
|
|
| ( ( "MBEAN" | "MBEANS" ) <stringLiteral> )
|
|
;
|
|
|
|
'''
|
|
|
|
|
|
@completer_for('permissionExpr', 'newpermission')
|
|
def permission_completer(ctxt, _):
|
|
new_permissions = set([permission.upper() for permission in ctxt.get_binding('newpermission')])
|
|
all_permissions = set([permission.arg for permission in ctxt.ruleset['permission'].arg])
|
|
suggestions = all_permissions - new_permissions
|
|
if len(suggestions) == 0:
|
|
return [Hint('No more permissions here.')]
|
|
return suggestions
|
|
|
|
|
|
@completer_for('username', 'name')
|
|
def username_name_completer(ctxt, cass):
|
|
# disable completion for CREATE USER.
|
|
if ctxt.matched[0][1].upper() == 'CREATE':
|
|
return [Hint('<username>')]
|
|
|
|
session = cass.session
|
|
return map(maybe_escape_name, [row['name'] for row in session.execute("LIST USERS")])
|
|
|
|
|
|
@completer_for('rolename', 'role')
|
|
def rolename_completer(ctxt, cass):
|
|
# disable completion for CREATE ROLE.
|
|
if ctxt.matched[0][1].upper() == 'CREATE':
|
|
return [Hint('<rolename>')]
|
|
|
|
session = cass.session
|
|
return map(maybe_escape_name, [row['role'] for row in session.execute("LIST ROLES")])
|
|
|
|
|
|
syntax_rules += r'''
|
|
<createTriggerStatement> ::= "CREATE" "TRIGGER" ( "IF" "NOT" "EXISTS" )? <cident>
|
|
"ON" cf=<columnFamilyName> "USING" class=<stringLiteral>
|
|
;
|
|
<dropTriggerStatement> ::= "DROP" "TRIGGER" ( "IF" "EXISTS" )? triggername=<cident>
|
|
"ON" cf=<columnFamilyName>
|
|
;
|
|
'''
|
|
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 drop_trigger_completer(ctxt, cass):
|
|
names = get_trigger_names(ctxt, cass)
|
|
return list(map(maybe_escape_name, names))
|
|
|
|
|
|
# END SYNTAX/COMPLETION RULE DEFINITIONS
|
|
|
|
CqlRuleSet.append_rules(syntax_rules)
|