You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by ad...@apache.org on 2021/07/13 12:35:49 UTC

[cassandra] branch cassandra-3.11 updated: Make cqlsh use the same set of reserved keywords than the server uses

This is an automated email from the ASF dual-hosted git repository.

adelapena pushed a commit to branch cassandra-3.11
in repository https://gitbox.apache.org/repos/asf/cassandra.git


The following commit(s) were added to refs/heads/cassandra-3.11 by this push:
     new 2a8e018  Make cqlsh use the same set of reserved keywords than the server uses
2a8e018 is described below

commit 2a8e01858f3aa18e72806acfed5f9cfa9d6c6d03
Author: Aleksandr Sorokoumov <al...@gmail.com>
AuthorDate: Tue Jul 13 11:45:25 2021 +0100

    Make cqlsh use the same set of reserved keywords than the server uses
    
    patch by Aleksandr Sorokoumov; reviewed by Andrés de la Peña and Ekaterina Dimitrova for CASSANDRA-15663
    
    Co-authored-by: Aleksandr Sorokoumov <al...@gmail.com>
    Co-authored-by: Ekaterina Dimitrova <ek...@datastax.com>
    Co-authored-by: Andrés de la Peña <a....@gmail.com>
---
 CHANGES.txt                                        |  1 +
 pylib/cqlshlib/cqlhandling.py                      | 25 ++++--
 pylib/cqlshlib/test/basecase.py                    | 12 ++-
 pylib/cqlshlib/test/cassconnect.py                 |  4 +-
 pylib/cqlshlib/test/run_cqlsh.py                   |  2 +-
 pylib/cqlshlib/test/test_constants.py              | 37 +++++++++
 src/antlr/Lexer.g                                  |  5 +-
 .../apache/cassandra/cql3/ReservedKeywords.java    | 96 +++++++---------------
 .../apache/cassandra/cql3/reserved_keywords.txt    | 62 ++++++++++++++
 .../cassandra/cql3/ReservedKeywordsTest.java       |  4 +-
 10 files changed, 166 insertions(+), 82 deletions(-)

diff --git a/CHANGES.txt b/CHANGES.txt
index 99e2798..526d8a2 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 3.11.11
+ * Make cqlsh use the same set of reserved keywords than the server uses (CASSANDRA-15663)
  * Optimize bytes skipping when reading SSTable files (CASSANDRA-14415)
  * Enable tombstone compactions when unchecked_tombstone_compaction is set in TWCS (CASSANDRA-14496)
  * Read only the required SSTables for single partition queries (CASSANDRA-16737)
diff --git a/pylib/cqlshlib/cqlhandling.py b/pylib/cqlshlib/cqlhandling.py
index 51d9726..a9c9c6f 100644
--- a/pylib/cqlshlib/cqlhandling.py
+++ b/pylib/cqlshlib/cqlhandling.py
@@ -18,11 +18,25 @@
 # i.e., stuff that's not necessarily cqlsh-specific
 
 import traceback
-from cassandra.metadata import cql_keywords_reserved
+import cassandra
 from . import pylexotron, util
 
 Hint = pylexotron.Hint
 
+cql_keywords_reserved = set((
+    'add', 'allow', 'alter', 'and', 'apply', 'asc', 'authorize', 'batch', 'begin', 'by', 'columnfamily', 'create',
+    'default', 'delete', 'desc', 'describe', 'drop', 'entries', 'execute', 'from', 'full', 'grant', 'if', 'in', 'index',
+    'infinity', 'insert', 'into', 'is', 'keyspace', 'limit', 'materialized', 'mbean', 'mbeans', 'modify', 'nan',
+    'norecursive', 'not', 'null', 'of', 'on', 'or', 'order', 'primary', 'rename', 'replace', 'revoke', 'schema',
+    'select', 'set', 'table', 'to', 'token', 'truncate', 'unlogged', 'unset', 'update', 'use', 'using', 'view', 'where',
+    'with'
+))
+"""
+Set of reserved keywords in CQL.
+
+Derived from .../cassandra/src/java/org/apache/cassandra/cql3/ReservedKeywords.java
+"""
+
 
 class CqlParsingRuleSet(pylexotron.ParsingRuleSet):
 
@@ -57,14 +71,15 @@ class CqlParsingRuleSet(pylexotron.ParsingRuleSet):
 
         # note: commands_end_with_newline may be extended by callers.
         self.commands_end_with_newline = set()
-        self.set_reserved_keywords(cql_keywords_reserved)
+        self.set_reserved_keywords()
 
-    def set_reserved_keywords(self, keywords):
+    def set_reserved_keywords(self):
         """
-        We cannot let resreved cql keywords be simple 'identifier' since this caused
+        We cannot let reserved cql keywords be simple 'identifier' since this caused
         problems with completion, see CASSANDRA-10415
         """
-        syntax = '<reserved_identifier> ::= /(' + '|'.join(r'\b{}\b'.format(k) for k in keywords) + ')/ ;'
+        cassandra.metadata.cql_keywords_reserved = cql_keywords_reserved
+        syntax = '<reserved_identifier> ::= /(' + '|'.join(r'\b{}\b'.format(k) for k in cql_keywords_reserved) + ')/ ;'
         self.append_rules(syntax)
 
     def completer_for(self, rulename, symname):
diff --git a/pylib/cqlshlib/test/basecase.py b/pylib/cqlshlib/test/basecase.py
index d393769..2c50b9a 100644
--- a/pylib/cqlshlib/test/basecase.py
+++ b/pylib/cqlshlib/test/basecase.py
@@ -30,13 +30,14 @@ try:
 except ImportError:
     import unittest
 
-rundir = dirname(__file__)
-cqlshdir = normpath(join(rundir, '..', '..', '..', 'bin'))
-path_to_cqlsh = normpath(join(cqlshdir, 'cqlsh.py'))
+test_dir = dirname(__file__)
+cassandra_dir = normpath(join(test_dir, '..', '..', '..'))
+cqlsh_dir = join(cassandra_dir, 'bin')
 
-sys.path.append(cqlshdir)
+sys.path.append(cqlsh_dir)
 
 import cqlsh
+
 cql = cqlsh.cassandra.cluster.Cluster
 policy = cqlsh.cassandra.policies.RoundRobinPolicy()
 quote_name = cqlsh.cassandra.metadata.maybe_escape_name
@@ -44,6 +45,7 @@ quote_name = cqlsh.cassandra.metadata.maybe_escape_name
 TEST_HOST = os.environ.get('CQL_TEST_HOST', '127.0.0.1')
 TEST_PORT = int(os.environ.get('CQL_TEST_PORT', 9042))
 
+
 class BaseTestCase(unittest.TestCase):
     def assertNicelyFormattedTableHeader(self, line, msg=None):
         return self.assertRegexpMatches(line, r'^ +\w+( +\| \w+)*\s*$', msg=msg)
@@ -54,6 +56,7 @@ class BaseTestCase(unittest.TestCase):
     def assertNicelyFormattedTableData(self, line, msg=None):
         return self.assertRegexpMatches(line, r'^ .* \| ', msg=msg)
 
+
 def dedent(s):
     lines = [ln.rstrip() for ln in s.splitlines()]
     if lines[0] == '':
@@ -62,5 +65,6 @@ def dedent(s):
     minspace = min(spaces if len(spaces) > 0 else (0,))
     return '\n'.join(line[minspace:] for line in lines)
 
+
 def at_a_time(i, num):
     return izip(*([iter(i)] * num))
diff --git a/pylib/cqlshlib/test/cassconnect.py b/pylib/cqlshlib/test/cassconnect.py
index 501850c..1cc561c 100644
--- a/pylib/cqlshlib/test/cassconnect.py
+++ b/pylib/cqlshlib/test/cassconnect.py
@@ -19,10 +19,10 @@ from __future__ import with_statement
 import contextlib
 import tempfile
 import os.path
-from .basecase import cql, cqlsh, cqlshlog, TEST_HOST, TEST_PORT, rundir, policy, quote_name
+from .basecase import cql, cqlsh, cqlshlog, TEST_HOST, TEST_PORT, test_dir, policy, quote_name
 from .run_cqlsh import run_cqlsh, call_cqlsh
 
-test_keyspace_init = os.path.join(rundir, 'test_keyspace_init.cql')
+test_keyspace_init = os.path.join(test_dir, 'test_keyspace_init.cql')
 
 def get_cassandra_connection(cql_version=None):
     conn = cql((TEST_HOST,), TEST_PORT, cql_version=cql_version, load_balancing_policy=policy)
diff --git a/pylib/cqlshlib/test/run_cqlsh.py b/pylib/cqlshlib/test/run_cqlsh.py
index fa010fe..83d0702 100644
--- a/pylib/cqlshlib/test/run_cqlsh.py
+++ b/pylib/cqlshlib/test/run_cqlsh.py
@@ -239,7 +239,7 @@ class CqlshRunner(ProcRunner):
             cqlsh_bin = 'cqlsh'
             if is_win():
                 cqlsh_bin = 'cqlsh.bat'
-            path = normpath(join(basecase.cqlshdir, cqlsh_bin))
+            path = join(basecase.cqlsh_dir, cqlsh_bin)
         if host is None:
             host = basecase.TEST_HOST
         if port is None:
diff --git a/pylib/cqlshlib/test/test_constants.py b/pylib/cqlshlib/test/test_constants.py
new file mode 100644
index 0000000..8107ffd
--- /dev/null
+++ b/pylib/cqlshlib/test/test_constants.py
@@ -0,0 +1,37 @@
+# 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 os.path import join
+
+from .basecase import BaseTestCase, cassandra_dir
+from cqlshlib.cqlhandling import cql_keywords_reserved as cql_keywords_reserved
+
+RESERVED_KEYWORDS_SOURCE = join(cassandra_dir, 'src', 'resources', 'org', 'apache', 'cassandra', 'cql3', 'reserved_keywords.txt')
+
+
+class TestConstants(BaseTestCase):
+
+    def test_cql_reserved_keywords(self):
+        with open(RESERVED_KEYWORDS_SOURCE) as f:
+            source_reserved_keywords = set(line.rstrip().lower() for line in f)
+
+        cqlsh_not_source = cql_keywords_reserved - source_reserved_keywords
+        self.assertFalse(cqlsh_not_source, "Reserved keywords in cqlsh not read from source %s."
+                         % (RESERVED_KEYWORDS_SOURCE,))
+
+        source_not_cqlsh = source_reserved_keywords - cql_keywords_reserved
+        self.assertFalse(source_not_cqlsh, "Reserved keywords in source %s not appearing in cqlsh."
+                         % (RESERVED_KEYWORDS_SOURCE,))
\ No newline at end of file
diff --git a/src/antlr/Lexer.g b/src/antlr/Lexer.g
index 1c52d4c..e7d5943 100644
--- a/src/antlr/Lexer.g
+++ b/src/antlr/Lexer.g
@@ -56,8 +56,9 @@ lexer grammar Lexer;
 }
 
 // Case-insensitive keywords
-// When adding a new reserved keyword, add entry to o.a.c.cql3.ReservedKeywords as well
-// When adding a new unreserved keyword, add entry to unreserved keywords in Parser.g
+// When adding a new reserved keyword, add entry to o.a.c.cql3.ReservedKeywords and
+// pylib/cqlshlib/cqlhandling.py::cql_keywords_reserved.
+// When adding a new unreserved keyword, add entry to unreserved keywords in Parser.g.
 K_SELECT:      S E L E C T;
 K_FROM:        F R O M;
 K_AS:          A S;
diff --git a/src/java/org/apache/cassandra/cql3/ReservedKeywords.java b/src/java/org/apache/cassandra/cql3/ReservedKeywords.java
index 30b1a6e..4dfc841 100644
--- a/src/java/org/apache/cassandra/cql3/ReservedKeywords.java
+++ b/src/java/org/apache/cassandra/cql3/ReservedKeywords.java
@@ -18,83 +18,47 @@
 
 package org.apache.cassandra.cql3;
 
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.nio.charset.StandardCharsets;
 import java.util.Set;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.ImmutableSet;
 
+import org.apache.cassandra.exceptions.ConfigurationException;
+
 class ReservedKeywords
 {
+
+    private static final String FILE_NAME = "reserved_keywords.txt";
+
     @VisibleForTesting
-    static final String[] reservedKeywords = new String[]
-                                                     {
-                                                     "SELECT",
-                                                     "FROM",
-                                                     "WHERE",
-                                                     "AND",
-                                                     "ENTRIES",
-                                                     "FULL",
-                                                     "INSERT",
-                                                     "UPDATE",
-                                                     "WITH",
-                                                     "LIMIT",
-                                                     "USING",
-                                                     "USE",
-                                                     "SET",
-                                                     "BEGIN",
-                                                     "UNLOGGED",
-                                                     "BATCH",
-                                                     "APPLY",
-                                                     "TRUNCATE",
-                                                     "DELETE",
-                                                     "IN",
-                                                     "CREATE",
-                                                     "KEYSPACE",
-                                                     "SCHEMA",
-                                                     "COLUMNFAMILY",
-                                                     "TABLE",
-                                                     "MATERIALIZED",
-                                                     "VIEW",
-                                                     "INDEX",
-                                                     "ON",
-                                                     "TO",
-                                                     "DROP",
-                                                     "PRIMARY",
-                                                     "INTO",
-                                                     "ALTER",
-                                                     "RENAME",
-                                                     "ADD",
-                                                     "ORDER",
-                                                     "BY",
-                                                     "ASC",
-                                                     "DESC",
-                                                     "ALLOW",
-                                                     "IF",
-                                                     "IS",
-                                                     "GRANT",
-                                                     "OF",
-                                                     "REVOKE",
-                                                     "MODIFY",
-                                                     "AUTHORIZE",
-                                                     "DESCRIBE",
-                                                     "EXECUTE",
-                                                     "NORECURSIVE",
-                                                     "TOKEN",
-                                                     "NULL",
-                                                     "NOT",
-                                                     "NAN",
-                                                     "INFINITY",
-                                                     "OR",
-                                                     "REPLACE",
-                                                     "DEFAULT",
-                                                     "UNSET",
-                                                     "MBEAN",
-                                                     "MBEANS"};
+    static final Set<String> reservedKeywords = getFromResource();
 
-    private static final Set<String> reservedSet = ImmutableSet.copyOf(reservedKeywords);
+    private static Set<String> getFromResource()
+    {
+        ImmutableSet.Builder<String> builder = ImmutableSet.builder();
+        try (InputStream is = ReservedKeywords.class.getResource(FILE_NAME).openConnection().getInputStream();
+             BufferedReader r = new BufferedReader(new InputStreamReader(is, StandardCharsets.UTF_8)))
+        {
+            String line;
+            while ((line = r.readLine()) != null)
+            {
+                builder.add(line.trim());
+            }
+        }
+        catch (IOException e)
+        {
+            throw new ConfigurationException(String.format("Unable to read reserved keywords file '%s'", FILE_NAME), e);
+        }
+        return builder.build();
+    }
 
     static boolean isReserved(String text)
     {
-        return reservedSet.contains(text.toUpperCase());
+        return reservedKeywords.contains(text.toUpperCase());
     }
 }
diff --git a/src/resources/org/apache/cassandra/cql3/reserved_keywords.txt b/src/resources/org/apache/cassandra/cql3/reserved_keywords.txt
new file mode 100644
index 0000000..51abcab
--- /dev/null
+++ b/src/resources/org/apache/cassandra/cql3/reserved_keywords.txt
@@ -0,0 +1,62 @@
+ADD
+ALLOW
+ALTER
+AND
+APPLY
+ASC
+AUTHORIZE
+BATCH
+BEGIN
+BY
+COLUMNFAMILY
+CREATE
+DEFAULT
+DELETE
+DESC
+DESCRIBE
+DROP
+ENTRIES
+EXECUTE
+FROM
+FULL
+GRANT
+IF
+IN
+INDEX
+INFINITY
+INSERT
+INTO
+IS
+KEYSPACE
+LIMIT
+MATERIALIZED
+MBEAN
+MBEANS
+MODIFY
+NAN
+NORECURSIVE
+NOT
+NULL
+OF
+ON
+OR
+ORDER
+PRIMARY
+RENAME
+REPLACE
+REVOKE
+SCHEMA
+SELECT
+SET
+TABLE
+TO
+TOKEN
+TRUNCATE
+UNLOGGED
+UNSET
+UPDATE
+USE
+USING
+VIEW
+WHERE
+WITH
diff --git a/test/unit/org/apache/cassandra/cql3/ReservedKeywordsTest.java b/test/unit/org/apache/cassandra/cql3/ReservedKeywordsTest.java
index aaf9824..eb860eb 100644
--- a/test/unit/org/apache/cassandra/cql3/ReservedKeywordsTest.java
+++ b/test/unit/org/apache/cassandra/cql3/ReservedKeywordsTest.java
@@ -20,13 +20,13 @@ package org.apache.cassandra.cql3;
 
 import org.junit.Test;
 
-import junit.framework.Assert;
+import org.junit.Assert;
 import org.apache.cassandra.exceptions.SyntaxException;
 
 public class ReservedKeywordsTest
 {
     @Test
-    public void testReservedWordsForColumns() throws Exception
+    public void testReservedWordsForColumns()
     {
         for (String reservedWord : ReservedKeywords.reservedKeywords)
         {

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@cassandra.apache.org
For additional commands, e-mail: commits-help@cassandra.apache.org