You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by al...@apache.org on 2015/05/21 23:48:37 UTC

[1/2] cassandra git commit: (cqlsh) Add support for native protocol 4

Repository: cassandra
Updated Branches:
  refs/heads/trunk 615c1bac6 -> 15d424e86


(cqlsh) Add support for native protocol 4

patch by Stefania Alborghetti; reviewed by Aleksey Yeschenko for
CASSANDRA-9399


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

Branch: refs/heads/trunk
Commit: 33d71b825f5d5bf924629d7e70ebe018eb9d2872
Parents: 868e720
Author: Stefania Alborghetti <st...@datastax.com>
Authored: Tue May 19 12:18:20 2015 +0800
Committer: Aleksey Yeschenko <al...@apache.org>
Committed: Fri May 22 00:47:27 2015 +0300

----------------------------------------------------------------------
 CHANGES.txt                                     |   2 +-
 bin/cqlsh                                       |  58 +++++++++++++------
 ...driver-internal-only-2.5.1.post0-074650b.zip | Bin 0 -> 195907 bytes
 lib/cassandra-driver-internal-only-2.5.1.zip    | Bin 192609 -> 0 bytes
 pylib/cqlshlib/cql3handling.py                  |   2 +-
 pylib/cqlshlib/formatting.py                    |   4 +-
 pylib/cqlshlib/helptopics.py                    |  12 ++++
 pylib/cqlshlib/test/test_cqlsh_completion.py    |   8 +--
 pylib/cqlshlib/test/test_cqlsh_output.py        |   8 ++-
 pylib/cqlshlib/test/test_keyspace_init.cql      |  42 +++++++-------
 10 files changed, 86 insertions(+), 50 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/33d71b82/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index a97cf2f..8b59309 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,9 +1,9 @@
 2.2
+ * (cqlsh) Add support for native protocol 4 (CASSANDRA-9399)
  * Ensure that UDF and UDAs are keyspace-isolated (CASSANDRA-9409)
  * Revert CASSANDRA-7807 (tracing completion client notifications) (CASSANDRA-9429)
  * Add ability to stop compaction by ID (CASSANDRA-7207)
 Merged from 2.1:
-2.1.6
  * Improve estimated row count (CASSANDRA-9107)
  * Optimize range tombstone memory footprint (CASSANDRA-8603)
  * Use configured gcgs in anticompaction (CASSANDRA-9397)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/33d71b82/bin/cqlsh
----------------------------------------------------------------------
diff --git a/bin/cqlsh b/bin/cqlsh
index c73f9a4..b2a729c 100755
--- a/bin/cqlsh
+++ b/bin/cqlsh
@@ -135,7 +135,7 @@ from cqlshlib.tracing import print_trace_session, print_trace
 DEFAULT_HOST = '127.0.0.1'
 DEFAULT_PORT = 9042
 DEFAULT_CQLVER = '3.2.0'
-DEFAULT_PROTOCOL_VERSION = 3
+DEFAULT_PROTOCOL_VERSION = 4
 
 DEFAULT_FLOAT_PRECISION = 5
 DEFAULT_MAX_TRACE_WAIT = 10
@@ -550,7 +550,6 @@ 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 '':
@@ -581,7 +580,8 @@ class Shell(cmd.Cmd):
                  max_trace_wait=DEFAULT_MAX_TRACE_WAIT,
                  ssl=False,
                  single_statement=None,
-                 client_timeout=10):
+                 client_timeout=10,
+                 protocol_version=DEFAULT_PROTOCOL_VERSION):
         cmd.Cmd.__init__(self, completekey=completekey)
         self.hostname = hostname
         self.port = port
@@ -599,7 +599,7 @@ class Shell(cmd.Cmd):
             self.conn = use_conn
         else:
             self.conn = Cluster(contact_points=(self.hostname,), port=self.port, cql_version=cqlver,
-                                protocol_version=DEFAULT_PROTOCOL_VERSION,
+                                protocol_version=protocol_version,
                                 auth_provider=self.auth_provider,
                                 ssl_options=sslhandling.ssl_settings(hostname, CONFIG_FILE) if ssl else None,
                                 load_balancing_policy=WhiteListRoundRobinPolicy([self.hostname]))
@@ -1048,13 +1048,22 @@ class Shell(cmd.Cmd):
 
     def perform_statement(self, statement):
         stmt = SimpleStatement(statement, consistency_level=self.consistency_level, serial_consistency_level=self.serial_consistency_level, fetch_size=self.default_page_size if self.use_paging else None)
-        result = self.perform_simple_statement(stmt)
-        if self.tracing_enabled:
-            if stmt.trace:
-                print_trace(self, stmt.trace)
-            else:
-                msg = "Statement trace did not complete within %d seconds" % (self.session.max_trace_wait)
-                self.writeresult(msg, color=RED)
+        result, future = self.perform_simple_statement(stmt)
+
+        if future:
+            if future.warnings:
+                self.print_warnings(future.warnings)
+
+            if self.tracing_enabled:
+                try:
+                    trace = future.get_query_trace(self.max_trace_wait)
+                    if trace:
+                        print_trace(self, trace)
+                    else:
+                        msg = "Statement trace did not complete within %d seconds" % (self.session.max_trace_wait)
+                        self.writeresult(msg, color=RED)
+                except Exception, err:
+                    self.printerr("Unable to fetch query trace: %s" % (str(err),))
 
         return result
 
@@ -1069,19 +1078,20 @@ class Shell(cmd.Cmd):
 
     def perform_simple_statement(self, statement):
         if not statement:
-            return False
+            return False, None
         rows = None
         while True:
             try:
-                rows = self.session.execute(statement, trace=self.tracing_enabled)
+                future = self.session.execute_async(statement, trace=self.tracing_enabled)
+                rows = future.result(self.session.default_timeout)
                 break
             except CQL_ERRORS, err:
                 self.printerr(str(err.__class__.__name__) + ": " + str(err))
-                return False
+                return False, None
             except Exception, err:
                 import traceback
                 self.printerr(traceback.format_exc())
-                return False
+                return False, None
 
         if statement.query_string[:6].lower() == 'select':
             self.print_result(rows, self.parse_for_table_meta(statement.query_string))
@@ -1094,7 +1104,7 @@ class Shell(cmd.Cmd):
             self.writeresult("")
             self.print_static_result(rows, self.parse_for_table_meta(statement.query_string))
         self.flush_output()
-        return True
+        return True, future
 
     def print_result(self, rows, table_meta):
         self.decoding_errors = []
@@ -1189,6 +1199,16 @@ class Shell(cmd.Cmd):
                 self.writeresult(' ' + " | ".join([column, value]))
             self.writeresult('')
 
+    def print_warnings(self, warnings):
+        if warnings is None or len(warnings) == 0:
+            return;
+
+        self.writeresult('')
+        self.writeresult('Warnings :')
+        for warning in warnings:
+            self.writeresult(warning)
+            self.writeresult('')
+
     def emptyline(self):
         pass
 
@@ -1614,7 +1634,7 @@ class Shell(cmd.Cmd):
                 contact_points=(self.hostname,),
                 port=self.port,
                 cql_version=self.conn.cql_version,
-                protocol_version=DEFAULT_PROTOCOL_VERSION,
+                protocol_version=self.conn.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]),
@@ -1703,7 +1723,7 @@ class Shell(cmd.Cmd):
 
                 request_id = conn.get_request_id()
                 binary_message = query_message.to_binary(
-                    stream_id=request_id, protocol_version=DEFAULT_PROTOCOL_VERSION, compression=None)
+                    stream_id=request_id, protocol_version=self.conn.protocol_version, compression=None)
 
                 # add the message directly to the connection's queue
                 with conn.lock:
@@ -2036,7 +2056,7 @@ class Shell(cmd.Cmd):
         auth_provider = PlainTextAuthProvider(username=username, password=password)
 
         conn = Cluster(contact_points=(self.hostname,), port=self.port, cql_version=self.conn.cql_version,
-                       protocol_version=DEFAULT_PROTOCOL_VERSION,
+                       protocol_version=self.conn.protocol_version,
                        auth_provider=auth_provider,
                        ssl_options=self.conn.ssl_options,
                        load_balancing_policy=WhiteListRoundRobinPolicy([self.hostname]))

http://git-wip-us.apache.org/repos/asf/cassandra/blob/33d71b82/lib/cassandra-driver-internal-only-2.5.1.post0-074650b.zip
----------------------------------------------------------------------
diff --git a/lib/cassandra-driver-internal-only-2.5.1.post0-074650b.zip b/lib/cassandra-driver-internal-only-2.5.1.post0-074650b.zip
new file mode 100644
index 0000000..ce21a7a
Binary files /dev/null and b/lib/cassandra-driver-internal-only-2.5.1.post0-074650b.zip differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/33d71b82/lib/cassandra-driver-internal-only-2.5.1.zip
----------------------------------------------------------------------
diff --git a/lib/cassandra-driver-internal-only-2.5.1.zip b/lib/cassandra-driver-internal-only-2.5.1.zip
deleted file mode 100644
index ee6ace0..0000000
Binary files a/lib/cassandra-driver-internal-only-2.5.1.zip and /dev/null differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/33d71b82/pylib/cqlshlib/cql3handling.py
----------------------------------------------------------------------
diff --git a/pylib/cqlshlib/cql3handling.py b/pylib/cqlshlib/cql3handling.py
index 3e155d0..ae66a4e 100644
--- a/pylib/cqlshlib/cql3handling.py
+++ b/pylib/cqlshlib/cql3handling.py
@@ -20,7 +20,7 @@ from cassandra.metadata import escape_name
 
 
 simple_cql_types = set(('ascii', 'bigint', 'blob', 'boolean', 'counter', 'date', 'decimal', 'double', 'float', 'inet', 'int',
-                        'text', 'time', 'timestamp', 'timeuuid', 'uuid', 'varchar', 'varint'))
+                        'smallint', 'text', 'time', 'timestamp', 'timeuuid', 'tinyint', 'uuid', 'varchar', 'varint'))
 simple_cql_types.difference_update(('set', 'map', 'list'))
 
 from . import helptopics

http://git-wip-us.apache.org/repos/asf/cassandra/blob/33d71b82/pylib/cqlshlib/formatting.py
----------------------------------------------------------------------
diff --git a/pylib/cqlshlib/formatting.py b/pylib/cqlshlib/formatting.py
index 868ec28..2310fa9 100644
--- a/pylib/cqlshlib/formatting.py
+++ b/pylib/cqlshlib/formatting.py
@@ -192,8 +192,8 @@ def strftime(time_format, seconds):
     hours, minutes = divmod(abs(offset) / 60, 60)
     return formatted[:-5] + sign + '{0:0=2}{1:0=2}'.format(hours, minutes)
 
-@formatter_for('date')
-def format_value_uuid(val, colormap, **_):
+@formatter_for('Date')
+def format_value_date(val, colormap, **_):
     return format_python_formatted_type(val, colormap, 'date')
 
 @formatter_for('Time')

http://git-wip-us.apache.org/repos/asf/cassandra/blob/33d71b82/pylib/cqlshlib/helptopics.py
----------------------------------------------------------------------
diff --git a/pylib/cqlshlib/helptopics.py b/pylib/cqlshlib/helptopics.py
index 0a43882..b38b235 100644
--- a/pylib/cqlshlib/helptopics.py
+++ b/pylib/cqlshlib/helptopics.py
@@ -38,6 +38,7 @@ class CQLHelpTopics(object):
           HELP BLOB_INPUT
           HELP UUID_INPUT
           HELP BOOLEAN_INPUT
+          HELP INT_INPUT
 
           HELP TEXT_OUTPUT
           HELP TIMESTAMP_OUTPUT
@@ -119,6 +120,17 @@ class CQLHelpTopics(object):
         as input for boolean types.
         """
 
+    def help_int_input(self):
+        print """
+        Integer input
+
+        CQL accepts the following integer types:
+          tinyint  - 1-byte signed integer
+          smallint - 2-byte signed integer
+          int      - 4-byte signed integer
+          bigint   - 8-byte signed integer
+        """
+
     def help_timestamp_output(self):
         print """
         Timestamp output

http://git-wip-us.apache.org/repos/asf/cassandra/blob/33d71b82/pylib/cqlshlib/test/test_cqlsh_completion.py
----------------------------------------------------------------------
diff --git a/pylib/cqlshlib/test/test_cqlsh_completion.py b/pylib/cqlshlib/test/test_cqlsh_completion.py
index cf7cab2..d6ccaf7 100644
--- a/pylib/cqlshlib/test/test_cqlsh_completion.py
+++ b/pylib/cqlshlib/test/test_cqlsh_completion.py
@@ -143,8 +143,8 @@ class TestCqlshCompletion(CqlshCompletionCase):
     def test_complete_on_empty_string(self):
         self.trycompletions('', choices=('?', 'ALTER', 'BEGIN', 'CAPTURE', 'CONSISTENCY',
                                          'COPY', 'CREATE', 'DEBUG', 'DELETE', 'DESC', 'DESCRIBE',
-                                         'DROP', 'GRANT', 'HELP', 'INSERT', 'LIST', 'PAGING', 'REVOKE',
-                                         'SELECT', 'SHOW', 'SOURCE', 'TRACING', 'EXPAND', 'TRUNCATE',
+                                         'DROP', 'GRANT', 'HELP', 'INSERT', 'LIST', 'LOGIN', 'PAGING', 'REVOKE',
+                                         'SELECT', 'SHOW', 'SOURCE', 'TRACING', 'EXPAND', 'SERIAL', 'TRUNCATE',
                                          'UPDATE', 'USE', 'exit', 'quit'))
 
     def test_complete_command_words(self):
@@ -229,8 +229,8 @@ class TestCqlshCompletion(CqlshCompletionCase):
              "VALUES ( 'eggs', 'sausage', 'spam');"),
             choices=['?', 'ALTER', 'BEGIN', 'CAPTURE', 'CONSISTENCY', 'COPY',
                      'CREATE', 'DEBUG', 'DELETE', 'DESC', 'DESCRIBE', 'DROP',
-                     'EXPAND', 'GRANT', 'HELP', 'INSERT', 'LIST', 'PAGING',
-                     'REVOKE', 'SELECT', 'SHOW', 'SOURCE', 'TRACING',
+                     'EXPAND', 'GRANT', 'HELP', 'INSERT', 'LIST', 'LOGIN', 'PAGING',
+                     'REVOKE', 'SELECT', 'SHOW', 'SOURCE', 'SERIAL', 'TRACING',
                      'TRUNCATE', 'UPDATE', 'USE', 'exit', 'quit'])
 
         self.trycompletions(

http://git-wip-us.apache.org/repos/asf/cassandra/blob/33d71b82/pylib/cqlshlib/test/test_cqlsh_output.py
----------------------------------------------------------------------
diff --git a/pylib/cqlshlib/test/test_cqlsh_output.py b/pylib/cqlshlib/test/test_cqlsh_output.py
index 40c7efc..2fd0ac7 100644
--- a/pylib/cqlshlib/test/test_cqlsh_output.py
+++ b/pylib/cqlshlib/test/test_cqlsh_output.py
@@ -275,9 +275,9 @@ class TestCqlshOutput(BaseTestCase):
         # same query should show up as empty in cql 3
         self.assertQueriesGiveColoredOutput((
             (q, """
-             num | asciicol | bigintcol | blobcol | booleancol | decimalcol | doublecol | floatcol | intcol | textcol | timestampcol | uuidcol | varcharcol | varintcol
-             RRR   MMMMMMMM   MMMMMMMMM   MMMMMMM   MMMMMMMMMM   MMMMMMMMMM   MMMMMMMMM   MMMMMMMM   MMMMMM   MMMMMMM   MMMMMMMMMMMM   MMMMMMM   MMMMMMMMMM   MMMMMMMMM
-            -----+----------+-----------+---------+------------+------------+-----------+----------+--------+---------+--------------+---------+------------+-----------
+             num | asciicol | bigintcol | blobcol | booleancol | decimalcol | doublecol | floatcol | intcol | smallintcol | textcol | timestampcol | tinyintcol | uuidcol | varcharcol | varintcol
+             RRR   MMMMMMMM   MMMMMMMMM   MMMMMMM   MMMMMMMMMM   MMMMMMMMMM   MMMMMMMMM   MMMMMMMM   MMMMMM   MMMMMMMMMMM   MMMMMMM   MMMMMMMMMMMM   MMMMMMMMMM   MMMMMMM   MMMMMMMMMM   MMMMMMMMM
+            -----+----------+-----------+---------+------------+------------+-----------+----------+--------+-------------+---------+--------------+------------+---------+------------+-----------
 
 
             (0 rows)
@@ -602,8 +602,10 @@ class TestCqlshOutput(BaseTestCase):
                 doublecol double,
                 floatcol float,
                 intcol int,
+                smallintcol smallint,
                 textcol text,
                 timestampcol timestamp,
+                tinyintcol tinyint,
                 uuidcol uuid,
                 varcharcol text,
                 varintcol varint

http://git-wip-us.apache.org/repos/asf/cassandra/blob/33d71b82/pylib/cqlshlib/test/test_keyspace_init.cql
----------------------------------------------------------------------
diff --git a/pylib/cqlshlib/test/test_keyspace_init.cql b/pylib/cqlshlib/test/test_keyspace_init.cql
index 2433ca0..fda629e 100644
--- a/pylib/cqlshlib/test/test_keyspace_init.cql
+++ b/pylib/cqlshlib/test/test_keyspace_init.cql
@@ -8,49 +8,51 @@ CREATE TABLE has_all_types (
     decimalcol decimal,
     doublecol double,
     floatcol float,
+    smallintcol smallint,
     textcol text,
     timestampcol timestamp,
+    tinyintcol tinyint,
     uuidcol uuid,
     varcharcol varchar,
     varintcol varint
 ) WITH compression = {'sstable_compression':'LZ4Compressor'};
 
 INSERT INTO has_all_types (num, intcol, asciicol, bigintcol, blobcol, booleancol,
-                           decimalcol, doublecol, floatcol, textcol,
-                           timestampcol, uuidcol, varcharcol, varintcol)
+                           decimalcol, doublecol, floatcol, smallintcol, textcol,
+                           timestampcol, tinyintcol, uuidcol, varcharcol, varintcol)
 VALUES (0, -12, 'abcdefg', 1234567890123456789, 0x000102030405fffefd, true,
-        19952.11882, 1.0, -2.1, 'Voilá!',
-        '2012-05-14 12:53:20+0000', bd1924e1-6af8-44ae-b5e1-f24131dbd460, '"', 10000000000000000000000000);
+        19952.11882, 1.0, -2.1, 32767, 'Voilá!',
+        '2012-05-14 12:53:20+0000', 127, bd1924e1-6af8-44ae-b5e1-f24131dbd460, '"', 10000000000000000000000000);
 
 INSERT INTO has_all_types (num, intcol, asciicol, bigintcol, blobcol, booleancol,
-                           decimalcol, doublecol, floatcol, textcol,
-                           timestampcol, uuidcol, varcharcol, varintcol)
+                           decimalcol, doublecol, floatcol, smallintcol, textcol,
+                           timestampcol, tinyintcol, uuidcol, varcharcol, varintcol)
 VALUES (1, 2147483647, '__!''$#@!~"', 9223372036854775807, 0xffffffffffffffffff, true,
-        0.00000000000001, 9999999.999, 99999.99, '∭Ƕ⑮ฑ➳❏''',
-        '1950-01-01+0000', ffffffff-ffff-ffff-ffff-ffffffffffff, 'newline->
+        0.00000000000001, 9999999.999, 99999.99, 32767, '∭Ƕ⑮ฑ➳❏''',
+        '1950-01-01+0000', 127, ffffffff-ffff-ffff-ffff-ffffffffffff, 'newline->
 <-', 9);
 
 INSERT INTO has_all_types (num, intcol, asciicol, bigintcol, blobcol, booleancol,
-                           decimalcol, doublecol, floatcol, textcol,
-                           timestampcol, uuidcol, varcharcol, varintcol)
+                           decimalcol, doublecol, floatcol, smallintcol, textcol,
+                           timestampcol, tinyintcol, uuidcol, varcharcol, varintcol)
 VALUES (2, 0, '', 0, 0x, false,
-        0.0, 0.0, 0.0, '',
-        0, 00000000-0000-0000-0000-000000000000, '', 0);
+        0.0, 0.0, 0.0, 0, '',
+        0, 0, 00000000-0000-0000-0000-000000000000, '', 0);
 
 INSERT INTO has_all_types (num, intcol, asciicol, bigintcol, blobcol, booleancol,
-                           decimalcol, doublecol, floatcol, textcol,
-                           timestampcol, uuidcol, varcharcol, varintcol)
+                           decimalcol, doublecol, floatcol, smallintcol, textcol,
+                           timestampcol, tinyintcol, uuidcol, varcharcol, varintcol)
 VALUES (3, -2147483648, '''''''', -9223372036854775808, 0x80, false,
-        10.0000000000000, -1004.10, 100000000.9, '龍馭鬱',
-        '2038-01-19T03:14-1200', ffffffff-ffff-1fff-8fff-ffffffffffff,
+        10.0000000000000, -1004.10, 100000000.9, 32767, '龍馭鬱',
+        '2038-01-19T03:14-1200', 127, ffffffff-ffff-1fff-8fff-ffffffffffff,
 	    '''', -10000000000000000000000000);
 
 INSERT INTO has_all_types (num, intcol, asciicol, bigintcol, blobcol, booleancol,
-                           decimalcol, doublecol, floatcol, textcol,
-                           timestampcol, uuidcol, varcharcol, varintcol)
+                           decimalcol, doublecol, floatcol, smallintcol, textcol,
+                           timestampcol, tinyintcol, uuidcol, varcharcol, varintcol)
 VALUES (4, blobAsInt(0x), '', blobAsBigint(0x), 0x, blobAsBoolean(0x),
-	blobAsDecimal(0x), blobAsDouble(0x), blobAsFloat(0x), '',
-	blobAsTimestamp(0x), blobAsUuid(0x), '', blobAsVarint(0x));
+	blobAsDecimal(0x), blobAsDouble(0x), blobAsFloat(0x), blobAsSmallInt(0x0000), '',
+	blobAsTimestamp(0x), blobAsTinyInt(0x00), blobAsUuid(0x), '', blobAsVarint(0x));
 
 
 


[2/2] cassandra git commit: Merge branch 'cassandra-2.2' into trunk

Posted by al...@apache.org.
Merge branch 'cassandra-2.2' into trunk


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

Branch: refs/heads/trunk
Commit: 15d424e86f2f28538904a3b1208eef2066734392
Parents: 615c1ba 33d71b8
Author: Aleksey Yeschenko <al...@apache.org>
Authored: Fri May 22 00:48:13 2015 +0300
Committer: Aleksey Yeschenko <al...@apache.org>
Committed: Fri May 22 00:48:13 2015 +0300

----------------------------------------------------------------------
 CHANGES.txt                                     |   2 +-
 bin/cqlsh                                       |  58 +++++++++++++------
 ...driver-internal-only-2.5.1.post0-074650b.zip | Bin 0 -> 195907 bytes
 lib/cassandra-driver-internal-only-2.5.1.zip    | Bin 192609 -> 0 bytes
 pylib/cqlshlib/cql3handling.py                  |   2 +-
 pylib/cqlshlib/formatting.py                    |   4 +-
 pylib/cqlshlib/helptopics.py                    |  12 ++++
 pylib/cqlshlib/test/test_cqlsh_completion.py    |   8 +--
 pylib/cqlshlib/test/test_cqlsh_output.py        |   8 ++-
 pylib/cqlshlib/test/test_keyspace_init.cql      |  42 +++++++-------
 10 files changed, 86 insertions(+), 50 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/15d424e8/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 59ee0ac,8b59309..0d95a3b
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,9 -1,5 +1,10 @@@
 +3.0:
 + * Decommissioned nodes will not rejoin the cluster (CASSANDRA-8801)
 + * Change gossip stabilization to use endpoit size (CASSANDRA-9401)
 +
 +
  2.2
+  * (cqlsh) Add support for native protocol 4 (CASSANDRA-9399)
   * Ensure that UDF and UDAs are keyspace-isolated (CASSANDRA-9409)
   * Revert CASSANDRA-7807 (tracing completion client notifications) (CASSANDRA-9429)
   * Add ability to stop compaction by ID (CASSANDRA-7207)