You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by aw...@apache.org on 2017/11/15 21:47:17 UTC

[49/50] cassandra git commit: Tests for CASSANDRA-10857

Tests for CASSANDRA-10857


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

Branch: refs/heads/master
Commit: 8e65211b9483f35f021c34a8bc7107c819eb0766
Parents: 01df7c4
Author: Alex Petrov <ol...@gmail.com>
Authored: Tue Oct 24 13:41:35 2017 +0200
Committer: Alex Petrov <ol...@gmail.com>
Committed: Thu Nov 9 13:37:03 2017 +0100

----------------------------------------------------------------------
 consistency_test.py                      |   2 +-
 counter_tests.py                         |   2 +
 cql_tests.py                             |  91 ++++++++--
 dtest.py                                 |   4 +
 materialized_views_test.py               |   4 +-
 paging_test.py                           | 231 ++++++++++++++++----------
 schema_metadata_test.py                  |   2 +
 schema_test.py                           |   2 +
 secondary_indexes_test.py                |  18 +-
 sstable_generation_loading_test.py       |   2 +-
 upgrade_tests/upgrade_compact_storage.py | 176 ++++++++++++++++++++
 write_failures_test.py                   |   6 +-
 12 files changed, 430 insertions(+), 110 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/8e65211b/consistency_test.py
----------------------------------------------------------------------
diff --git a/consistency_test.py b/consistency_test.py
index ccc7ee6..40c794d 100644
--- a/consistency_test.py
+++ b/consistency_test.py
@@ -170,7 +170,7 @@ class TestHelper(Tester):
                 firstname text,
                 lastname text,
                 age int
-            ) WITH COMPACT STORAGE"""
+            )"""
 
         if requires_local_reads:
             create_cmd += " AND " + self.get_local_reads_properties()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8e65211b/counter_tests.py
----------------------------------------------------------------------
diff --git a/counter_tests.py b/counter_tests.py
index da2ed09..48985a9 100644
--- a/counter_tests.py
+++ b/counter_tests.py
@@ -334,6 +334,7 @@ class TestCounters(Tester):
 
             self.assertEqual(v, count[0][0])
 
+    @since("2.0", max_version="3.X")
     def validate_empty_column_name_test(self):
         cluster = self.cluster
         cluster.populate(1).start()
@@ -379,6 +380,7 @@ class TestCounters(Tester):
 
         assert_invalid(session, "ALTER TABLE counter_bug add c counter", "Cannot re-add previously dropped counter column c")
 
+    @since("2.0", max_version="3.X") # Compact Storage
     def compact_counter_cluster_test(self):
         """
         @jira_ticket CASSANDRA-12219

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8e65211b/cql_tests.py
----------------------------------------------------------------------
diff --git a/cql_tests.py b/cql_tests.py
index 64d30ba..b4ac02f 100644
--- a/cql_tests.py
+++ b/cql_tests.py
@@ -19,6 +19,7 @@ from thrift_bindings.v22.ttypes import (CfDef, Column, ColumnOrSuperColumn,
 from thrift_tests import get_thrift_client
 from tools.assertions import (assert_all, assert_invalid, assert_length_equal,
                               assert_none, assert_one, assert_unavailable)
+from tools.data import rows_to_list
 from tools.decorators import since
 from tools.metadata_wrapper import (UpdatingClusterMetadataWrapper,
                                     UpdatingKeyspaceMetadataWrapper,
@@ -106,11 +107,8 @@ class StorageProxyCQLTester(CQLTester):
         """
         Smoke test that basic table operations work:
 
-        - create 2 tables, one with and one without COMPACT STORAGE
-        - ALTER the table without COMPACT STORAGE, adding a column
-
-        For each of those tables:
-
+        - create a table
+        - ALTER the table adding a column
         - insert 10 values
         - SELECT * and assert the values are there
         - TRUNCATE the table
@@ -125,8 +123,6 @@ class StorageProxyCQLTester(CQLTester):
 
         session.execute("CREATE TABLE test1 (k int PRIMARY KEY, v1 int)")
         self.assertIn('test1', ks_meta.tables)
-        session.execute("CREATE TABLE test2 (k int, c1 int, v1 int, PRIMARY KEY (k, c1)) WITH COMPACT STORAGE")
-        self.assertIn('test2', ks_meta.tables)
 
         t1_meta = UpdatingTableMetadataWrapper(session.cluster, ks_name='ks', table_name='test1')
 
@@ -135,21 +131,46 @@ class StorageProxyCQLTester(CQLTester):
 
         for i in range(0, 10):
             session.execute("INSERT INTO test1 (k, v1, v2) VALUES ({i}, {i}, {i})".format(i=i))
-            session.execute("INSERT INTO test2 (k, c1, v1) VALUES ({i}, {i}, {i})".format(i=i))
 
         assert_all(session, "SELECT * FROM test1", [[i, i, i] for i in range(0, 10)], ignore_order=True)
 
-        assert_all(session, "SELECT * FROM test2", [[i, i, i] for i in range(0, 10)], ignore_order=True)
-
         session.execute("TRUNCATE test1")
-        session.execute("TRUNCATE test2")
 
         assert_none(session, "SELECT * FROM test1")
 
-        assert_none(session, "SELECT * FROM test2")
-
         session.execute("DROP TABLE test1")
         self.assertNotIn('test1', ks_meta.tables)
+
+    @since("2.0", max_version="3.X")
+    def table_test_compact_storage(self):
+        """
+        Smoke test that basic table operations work:
+
+        - create a table with COMPACT STORAGE
+        - insert 10 values
+        - SELECT * and assert the values are there
+        - TRUNCATE the table
+        - SELECT * and assert there are no values
+        - DROP the table
+        - SELECT * and assert the statement raises an InvalidRequest
+        # TODO run SELECTs to make sure each statement works
+        """
+        session = self.prepare()
+
+        ks_meta = UpdatingKeyspaceMetadataWrapper(session.cluster, ks_name='ks')
+
+        session.execute("CREATE TABLE test2 (k int, c1 int, v1 int, PRIMARY KEY (k, c1)) WITH COMPACT STORAGE")
+        self.assertIn('test2', ks_meta.tables)
+
+        for i in range(0, 10):
+            session.execute("INSERT INTO test2 (k, c1, v1) VALUES ({i}, {i}, {i})".format(i=i))
+
+        assert_all(session, "SELECT * FROM test2", [[i, i, i] for i in range(0, 10)], ignore_order=True)
+
+        session.execute("TRUNCATE test2")
+
+        assert_none(session, "SELECT * FROM test2")
+
         session.execute("DROP TABLE test2")
         self.assertNotIn('test2', ks_meta.tables)
 
@@ -698,6 +719,50 @@ class MiscellaneousCQLTester(CQLTester):
                    ",".join(map(lambda i: "c_{}".format(i), range(width))) +
                    " FROM very_wide_table", [[i for i in range(width)]])
 
+    @since("3.11", max_version="3.X")
+    def drop_compact_storage_flag_test(self):
+        """
+        Test for CASSANDRA-10857, verifying the schema change
+        distribution across the other nodes.
+
+        """
+
+        cluster = self.cluster
+
+        cluster.populate(3).start()
+        node1, node2, node3 = cluster.nodelist()
+
+        session1 = self.patient_cql_connection(node1)
+        session2 = self.patient_cql_connection(node2)
+        session3 = self.patient_cql_connection(node3)
+        create_ks(session1, 'ks', 3)
+        sessions = [session1, session2, session3]
+
+        for session in sessions:
+            session.set_keyspace('ks')
+
+        session1.execute("""
+            CREATE TABLE test_drop_compact_storage (k int PRIMARY KEY, s1 int) WITH COMPACT STORAGE;
+        """)
+
+        session1.execute("INSERT INTO test_drop_compact_storage (k, s1) VALUES (1,1)")
+        session1.execute("INSERT INTO test_drop_compact_storage (k, s1) VALUES (2,2)")
+        session1.execute("INSERT INTO test_drop_compact_storage (k, s1) VALUES (3,3)")
+
+        for session in sessions:
+            res = session.execute("SELECT * from test_drop_compact_storage")
+            self.assertEqual(rows_to_list(res), [[1, 1],
+                                                 [2, 2],
+                                                 [3, 3]])
+
+        session1.execute("ALTER TABLE test_drop_compact_storage DROP COMPACT STORAGE")
+
+        for session in sessions:
+            assert_all(session, "SELECT * from test_drop_compact_storage",
+                       [[1, None, 1, None],
+                        [2, None, 2, None],
+                        [3, None, 3, None]])
+
 
 @since('3.2')
 class AbortedQueryTester(CQLTester):

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8e65211b/dtest.py
----------------------------------------------------------------------
diff --git a/dtest.py b/dtest.py
index 0fd5ec51..12b5dcf 100644
--- a/dtest.py
+++ b/dtest.py
@@ -34,6 +34,7 @@ from cassandra.policies import RetryPolicy, WhiteListRoundRobinPolicy
 from ccmlib.cluster import Cluster
 from ccmlib.cluster_factory import ClusterFactory
 from ccmlib.common import get_version_from_build, is_win
+from distutils.version import LooseVersion
 from nose.exc import SkipTest
 from nose.tools import assert_greater_equal
 from six import print_
@@ -1125,3 +1126,6 @@ def run_scenarios(scenarios, handler, deferred_exceptions=tuple()):
 
     if errors:
         raise MultiError(errors, tracebacks)
+
+def supports_v5_protocol(cluster_version):
+    return cluster_version >= LooseVersion('4.0')

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8e65211b/materialized_views_test.py
----------------------------------------------------------------------
diff --git a/materialized_views_test.py b/materialized_views_test.py
index 12728c6..419a92a 100644
--- a/materialized_views_test.py
+++ b/materialized_views_test.py
@@ -18,7 +18,7 @@ from nose.plugins.attrib import attr
 from nose.tools import (assert_equal)
 
 from distutils.version import LooseVersion
-from dtest import Tester, debug, get_ip_from_node, create_ks
+from dtest import Tester, debug, get_ip_from_node, create_ks, supports_v5_protocol
 from tools.assertions import (assert_all, assert_crc_check_chance_equal,
                               assert_invalid, assert_none, assert_one,
                               assert_unavailable)
@@ -2441,7 +2441,7 @@ class TestMaterializedViewsLockcontention(Tester):
 
     def _prepare_cluster(self):
         self.cluster.populate(1)
-        self.supports_v5_protocol = self.cluster.version() >= LooseVersion('3.10')
+        self.supports_v5_protocol = supports_v5_protocol(self.cluster.version())
         self.protocol_version = 5 if self.supports_v5_protocol else 4
 
         self.cluster.set_configuration_options(values={

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8e65211b/paging_test.py
----------------------------------------------------------------------
diff --git a/paging_test.py b/paging_test.py
index 55d0f34..24a4739 100644
--- a/paging_test.py
+++ b/paging_test.py
@@ -8,7 +8,7 @@ from cassandra.policies import FallthroughRetryPolicy
 from cassandra.query import (SimpleStatement, dict_factory,
                              named_tuple_factory, tuple_factory)
 
-from dtest import Tester, debug, run_scenarios, create_ks
+from dtest import Tester, debug, run_scenarios, create_ks, supports_v5_protocol
 from tools.assertions import (assert_all, assert_invalid, assert_length_equal,
                               assert_one)
 from tools.data import rows_to_list
@@ -20,8 +20,8 @@ from tools.paging import PageAssertionMixin, PageFetcher
 class BasePagingTester(Tester):
 
     def prepare(self, row_factory=dict_factory):
-        supports_v5_protocol = self.cluster.version() >= LooseVersion('3.10')
-        protocol_version = 5 if supports_v5_protocol else None
+        supports_v5 = supports_v5_protocol(self.cluster.version())
+        protocol_version = 5 if supports_v5 else None
         cluster = self.cluster
         cluster.populate(3).start(wait_for_binary_proto=True)
         node1 = cluster.nodelist()[0]
@@ -1920,8 +1920,17 @@ class TestPagingData(BasePagingTester, PageAssertionMixin):
         """
 
         session = self.prepare(row_factory=tuple_factory)
-
         self._test_paging_with_filtering_on_counter_columns(session, False)
+
+    @since("3.6", max_version="3.X")  # Compact Storage
+    def test_paging_with_filtering_on_counter_columns_compact(self):
+        """
+        test paging, when filtering on counter columns with compact storage
+        @jira_ticket CASSANDRA-11629
+        """
+
+        session = self.prepare(row_factory=tuple_factory)
+
         self._test_paging_with_filtering_on_counter_columns(session, True)
 
     def _test_paging_with_filtering_on_clustering_columns(self, session, with_compact_storage):
@@ -2004,6 +2013,15 @@ class TestPagingData(BasePagingTester, PageAssertionMixin):
 
         session = self.prepare(row_factory=tuple_factory)
         self._test_paging_with_filtering_on_clustering_columns(session, False)
+
+    @since('3.6', max_version="3.X")  # Compact Storage
+    def test_paging_with_filtering_on_clustering_columns_compact(self):
+        """
+        test paging, when filtering on clustering columns with compact storage
+        @jira_ticket CASSANDRA-11310
+        """
+
+        session = self.prepare(row_factory=tuple_factory)
         self._test_paging_with_filtering_on_clustering_columns(session, True)
 
     @since('3.6')
@@ -2307,6 +2325,16 @@ class TestPagingData(BasePagingTester, PageAssertionMixin):
         session = self.prepare(row_factory=tuple_factory)
 
         self._test_paging_with_filtering_on_partition_key_on_counter_columns(session, False)
+
+    @since('3.10', max_version="3.X")  # Compact Storage
+    def test_paging_with_filtering_on_partition_key_on_counter_columns_compact(self):
+        """
+        test paging, when filtering on partition key on counter columns with compact storage
+        @jira_ticket CASSANDRA-11031
+        """
+
+        session = self.prepare(row_factory=tuple_factory)
+
         self._test_paging_with_filtering_on_partition_key_on_counter_columns(session, True)
 
     def _test_paging_with_filtering_on_partition_key_on_clustering_columns(self, session, with_compact_storage):
@@ -2401,6 +2429,15 @@ class TestPagingData(BasePagingTester, PageAssertionMixin):
 
         session = self.prepare(row_factory=tuple_factory)
         self._test_paging_with_filtering_on_partition_key_on_clustering_columns(session, False)
+
+    @since('3.10', max_version="3.X")
+    def test_paging_with_filtering_on_partition_key_on_clustering_columns_compact(self):
+        """
+        test paging, when filtering on partition key clustering columns with compact storage
+        @jira_ticket CASSANDRA-11031
+        """
+
+        session = self.prepare(row_factory=tuple_factory)
         self._test_paging_with_filtering_on_partition_key_on_clustering_columns(session, True)
 
     @since('3.10')
@@ -2518,7 +2555,7 @@ class TestPagingData(BasePagingTester, PageAssertionMixin):
 
             assert_invalid(session, "SELECT * FROM test WHERE s > 1 AND a < 2 AND b > 4 ORDER BY b DESC ALLOW FILTERING", expected=InvalidRequest)
 
-    @since('2.1.14')
+    @since('2.1.14', max_version="3.X")  # Compact Storage
     def test_paging_on_compact_table_with_tombstone_on_first_column(self):
         """
         test paging, on  COMPACT tables without clustering columns, when the first column has a tombstone
@@ -2551,61 +2588,72 @@ class TestPagingData(BasePagingTester, PageAssertionMixin):
         session = self.prepare(row_factory=tuple_factory)
         create_ks(session, 'test_paging_with_no_clustering_columns', 2)
         session.execute("CREATE TABLE test (a int primary key, b int)")
+        self._test_paging_with_no_clustering_columns('test', session)
+
+    @since("2.0", max_version="3.X")
+    def test_paging_with_no_clustering_columns_compact(self):
+        """
+        test paging for tables without clustering columns
+        @jira_ticket CASSANDRA-11208
+        """
+
+        session = self.prepare(row_factory=tuple_factory)
+        create_ks(session, 'test_paging_with_no_clustering_columns', 2)
         session.execute("CREATE TABLE test_compact (a int primary key, b int) WITH COMPACT STORAGE")
+        self._test_paging_with_no_clustering_columns('test_compact', session)
 
-        for table in ('test', 'test_compact'):
+    def _test_paging_with_no_clustering_columns(self, table, session):
+        for i in xrange(5):
+            session.execute("INSERT INTO {} (a, b) VALUES ({}, {})".format(table, i, i))
 
-            for i in xrange(5):
-                session.execute("INSERT INTO {} (a, b) VALUES ({}, {})".format(table, i, i))
+        for page_size in (2, 3, 4, 5, 7, 10):
+            session.default_fetch_size = page_size
 
-            for page_size in (2, 3, 4, 5, 7, 10):
-                session.default_fetch_size = page_size
+        # Range query
+        assert_all(session, "SELECT * FROM {}".format(table), [[1, 1],
+                                                               [0, 0],
+                                                               [2, 2],
+                                                               [4, 4],
+                                                               [3, 3]])
 
-                # Range query
-                assert_all(session, "SELECT * FROM {}".format(table), [[1, 1],
+        # Range query with LIMIT
+        assert_all(session, "SELECT * FROM {} LIMIT 3".format(table), [[1, 1],
                                                                        [0, 0],
-                                                                       [2, 2],
-                                                                       [4, 4],
-                                                                       [3, 3]])
+                                                                       [2, 2]])
 
-                # Range query with LIMIT
-                assert_all(session, "SELECT * FROM {} LIMIT 3".format(table), [[1, 1],
-                                                                               [0, 0],
-                                                                               [2, 2]])
+        # Range query with DISTINCT
+        assert_all(session, "SELECT DISTINCT a FROM {}".format(table), [[1],
+                                                                        [0],
+                                                                        [2],
+                                                                        [4],
+                                                                        [3]])
 
-                # Range query with DISTINCT
-                assert_all(session, "SELECT DISTINCT a FROM {}".format(table), [[1],
+        # Range query with DISTINCT and LIMIT
+        assert_all(session, "SELECT DISTINCT a FROM {} LIMIT 3".format(table), [[1],
                                                                                 [0],
-                                                                                [2],
-                                                                                [4],
-                                                                                [3]])
+                                                                                [2]])
 
-                # Range query with DISTINCT and LIMIT
-                assert_all(session, "SELECT DISTINCT a FROM {} LIMIT 3".format(table), [[1],
-                                                                                        [0],
-                                                                                        [2]])
+        # Multi-partition query
+        assert_all(session, "SELECT * FROM {} WHERE a IN (1, 2, 3, 4)".format(table), [[1, 1],
+                                                                                       [2, 2],
+                                                                                       [3, 3],
+                                                                                       [4, 4]])
 
-                # Multi-partition query
-                assert_all(session, "SELECT * FROM {} WHERE a IN (1, 2, 3, 4)".format(table), [[1, 1],
+        # Multi-partition query with LIMIT
+        assert_all(session, "SELECT * FROM {} WHERE a IN (1, 2, 3, 4) LIMIT 3".format(table), [[1, 1],
                                                                                                [2, 2],
-                                                                                               [3, 3],
-                                                                                               [4, 4]])
+                                                                                               [3, 3]])
 
-                # Multi-partition query with LIMIT
-                assert_all(session, "SELECT * FROM {} WHERE a IN (1, 2, 3, 4) LIMIT 3".format(table), [[1, 1],
-                                                                                                       [2, 2],
-                                                                                                       [3, 3]])
+        # Multi-partition query with DISTINCT
+        assert_all(session, "SELECT DISTINCT a FROM {} WHERE a IN (1, 2, 3, 4)".format(table), [[1],
+                                                                                                [2],
+                                                                                                [3],
+                                                                                                [4]])
 
-                # Multi-partition query with DISTINCT
-                assert_all(session, "SELECT DISTINCT a FROM {} WHERE a IN (1, 2, 3, 4)".format(table), [[1],
+        # Multi-partition query with DISTINCT and LIMIT
+        assert_all(session, "SELECT DISTINCT a FROM {} WHERE a IN (1, 2, 3, 4) LIMIT 3".format(table), [[1],
                                                                                                         [2],
-                                                                                                        [3],
-                                                                                                        [4]])
-
-                # Multi-partition query with DISTINCT and LIMIT
-                assert_all(session, "SELECT DISTINCT a FROM {} WHERE a IN (1, 2, 3, 4) LIMIT 3".format(table), [[1],
-                                                                                                                [2],
-                                                                                                                [3]])
+                                                                                                        [3]])
 
     @since('3.6')
     def test_per_partition_limit_paging(self):
@@ -2699,49 +2747,62 @@ class TestPagingData(BasePagingTester, PageAssertionMixin):
         session = self.prepare(row_factory=tuple_factory)
         create_ks(session, 'test_paging_for_range_name_queries', 2)
         session.execute("CREATE TABLE test (a int, b int, c int, d int, PRIMARY KEY(a, b, c))")
+
+        self._test_paging_for_range_name_queries('test', session)
+
+    @since("2.0", max_version="3.X")  # Compact Storage
+    def test_paging_for_range_name_queries_compact(self):
+        """
+        test paging for range name queries with compact storage
+        @jira_ticket CASSANDRA-11669
+        """
+
+        session = self.prepare(row_factory=tuple_factory)
+        create_ks(session, 'test_paging_for_range_name_queries', 2)
         session.execute("CREATE TABLE test_compact (a int, b int, c int, d int, PRIMARY KEY(a, b, c)) WITH COMPACT STORAGE")
 
-        for table in ('test', 'test_compact'):
+        self._test_paging_for_range_name_queries('test_compact', session)
 
-            for i in xrange(4):
-                for j in xrange(4):
-                    for k in xrange(4):
-                        session.execute("INSERT INTO {} (a, b, c, d) VALUES ({}, {}, {}, {})".format(table, i, j, k, i + j))
-
-            for page_size in (2, 3, 4, 5, 7, 10):
-                session.default_fetch_size = page_size
-
-                assert_all(session, "SELECT * FROM {} WHERE b = 1 AND c = 1  ALLOW FILTERING".format(table), [[1, 1, 1, 2],
-                                                                                                              [0, 1, 1, 1],
-                                                                                                              [2, 1, 1, 3],
-                                                                                                              [3, 1, 1, 4]])
-
-                assert_all(session, "SELECT * FROM {} WHERE b = 1 AND c IN (1, 2) ALLOW FILTERING".format(table), [[1, 1, 1, 2],
-                                                                                                                   [1, 1, 2, 2],
-                                                                                                                   [0, 1, 1, 1],
-                                                                                                                   [0, 1, 2, 1],
-                                                                                                                   [2, 1, 1, 3],
-                                                                                                                   [2, 1, 2, 3],
-                                                                                                                   [3, 1, 1, 4],
-                                                                                                                   [3, 1, 2, 4]])
-
-                if self.cluster.version() >= '2.2':
-                    assert_all(session, "SELECT * FROM {} WHERE b IN (1, 2) AND c IN (1, 2)  ALLOW FILTERING".format(table), [[1, 1, 1, 2],
-                                                                                                                              [1, 1, 2, 2],
-                                                                                                                              [1, 2, 1, 3],
-                                                                                                                              [1, 2, 2, 3],
-                                                                                                                              [0, 1, 1, 1],
-                                                                                                                              [0, 1, 2, 1],
-                                                                                                                              [0, 2, 1, 2],
-                                                                                                                              [0, 2, 2, 2],
-                                                                                                                              [2, 1, 1, 3],
-                                                                                                                              [2, 1, 2, 3],
-                                                                                                                              [2, 2, 1, 4],
-                                                                                                                              [2, 2, 2, 4],
-                                                                                                                              [3, 1, 1, 4],
-                                                                                                                              [3, 1, 2, 4],
-                                                                                                                              [3, 2, 1, 5],
-                                                                                                                              [3, 2, 2, 5]])
+    def _test_paging_for_range_name_queries(self, table, session):
+        for i in xrange(4):
+            for j in xrange(4):
+                for k in xrange(4):
+                    session.execute("INSERT INTO {} (a, b, c, d) VALUES ({}, {}, {}, {})".format(table, i, j, k, i + j))
+
+        for page_size in (2, 3, 4, 5, 7, 10):
+            session.default_fetch_size = page_size
+
+            assert_all(session, "SELECT * FROM {} WHERE b = 1 AND c = 1  ALLOW FILTERING".format(table), [[1, 1, 1, 2],
+                                                                                                          [0, 1, 1, 1],
+                                                                                                          [2, 1, 1, 3],
+                                                                                                          [3, 1, 1, 4]])
+
+            assert_all(session, "SELECT * FROM {} WHERE b = 1 AND c IN (1, 2) ALLOW FILTERING".format(table), [[1, 1, 1, 2],
+                                                                                                               [1, 1, 2, 2],
+                                                                                                               [0, 1, 1, 1],
+                                                                                                               [0, 1, 2, 1],
+                                                                                                               [2, 1, 1, 3],
+                                                                                                               [2, 1, 2, 3],
+                                                                                                               [3, 1, 1, 4],
+                                                                                                               [3, 1, 2, 4]])
+
+            if self.cluster.version() >= '2.2':
+                assert_all(session, "SELECT * FROM {} WHERE b IN (1, 2) AND c IN (1, 2)  ALLOW FILTERING".format(table), [[1, 1, 1, 2],
+                                                                                                                          [1, 1, 2, 2],
+                                                                                                                          [1, 2, 1, 3],
+                                                                                                                          [1, 2, 2, 3],
+                                                                                                                          [0, 1, 1, 1],
+                                                                                                                          [0, 1, 2, 1],
+                                                                                                                          [0, 2, 1, 2],
+                                                                                                                          [0, 2, 2, 2],
+                                                                                                                          [2, 1, 1, 3],
+                                                                                                                          [2, 1, 2, 3],
+                                                                                                                          [2, 2, 1, 4],
+                                                                                                                          [2, 2, 2, 4],
+                                                                                                                          [3, 1, 1, 4],
+                                                                                                                          [3, 1, 2, 4],
+                                                                                                                          [3, 2, 1, 5],
+                                                                                                                          [3, 2, 2, 5]])
 
     @since('2.1')
     def test_paging_with_empty_row_and_empty_static_columns(self):

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8e65211b/schema_metadata_test.py
----------------------------------------------------------------------
diff --git a/schema_metadata_test.py b/schema_metadata_test.py
index c64b5ea..baf8b5a 100644
--- a/schema_metadata_test.py
+++ b/schema_metadata_test.py
@@ -655,10 +655,12 @@ class TestSchemaMetadata(Tester):
         establish_clustering_order_table(self.cluster.version(), self.session)
         verify_clustering_order_table(self.cluster.version(), self.cluster.version(), 'ks', self.session)
 
+    @since("2.0", max_version="3.X")  # Compact Storage
     def compact_storage_test(self):
         establish_compact_storage_table(self.cluster.version(), self.session)
         verify_compact_storage_table(self.cluster.version(), self.cluster.version(), 'ks', self.session)
 
+    @since("2.0", max_version="3.X")  # Compact Storage
     def compact_storage_composite_test(self):
         establish_compact_storage_composite_table(self.cluster.version(), self.session)
         verify_compact_storage_composite_table(self.cluster.version(), self.cluster.version(), 'ks', self.session)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8e65211b/schema_test.py
----------------------------------------------------------------------
diff --git a/schema_test.py b/schema_test.py
index a600364..1553adc 100644
--- a/schema_test.py
+++ b/schema_test.py
@@ -3,6 +3,7 @@ import time
 from cassandra.concurrent import execute_concurrent_with_args
 
 from tools.assertions import assert_invalid, assert_all, assert_one
+from tools.decorators import since
 from dtest import Tester, create_ks
 
 
@@ -49,6 +50,7 @@ class TestSchema(Tester):
                 self.assertEqual(row.c2, 'ddd')
                 self.assertFalse(hasattr(row, 'c0'))
 
+    @since("2.0", max_version="3.X")  # Compact Storage
     def drop_column_compact_test(self):
         session = self.prepare()
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8e65211b/secondary_indexes_test.py
----------------------------------------------------------------------
diff --git a/secondary_indexes_test.py b/secondary_indexes_test.py
index a7580c2..183aae3 100644
--- a/secondary_indexes_test.py
+++ b/secondary_indexes_test.py
@@ -238,6 +238,14 @@ class TestSecondaryIndexes(Tester):
                                             "INSERT INTO %s (a, b, c) VALUES (?, 0, 0)",
                                             session)
 
+    @since("2.0", max_version="3.X")
+    def test_8280_validate_indexed_values_compact(self):
+        cluster = self.cluster
+        cluster.populate(1).start()
+        node1 = cluster.nodelist()[0]
+        session = self.patient_cql_connection(node1)
+
+        create_ks(session, 'ks', 1)
         self.insert_row_with_oversize_value("CREATE TABLE %s(a int, b text, PRIMARY KEY (a)) WITH COMPACT STORAGE",
                                             "CREATE INDEX ON %s(b)",
                                             "INSERT INTO %s (a, b) VALUES (0, ?)",
@@ -341,7 +349,7 @@ class TestSecondaryIndexes(Tester):
         session.execute("INSERT INTO k.t(k, v) VALUES (2, 3)")
 
         # Verify that the index is marked as built and it can answer queries
-        assert_one(session, """SELECT * FROM system."IndexInfo" WHERE table_name='k'""", ['k', 'idx'])
+        assert_one(session, """SELECT table_name, index_name FROM system."IndexInfo" WHERE table_name='k'""", ['k', 'idx'])
         assert_one(session, "SELECT * FROM k.t WHERE v = 1", [0, 1])
 
         # Simulate a failing index rebuild
@@ -367,7 +375,7 @@ class TestSecondaryIndexes(Tester):
 
         # Verify that, the index is rebuilt, marked as built, and it can answer queries
         self.assertNotEqual(before_files, after_files)
-        assert_one(session, """SELECT * FROM system."IndexInfo" WHERE table_name='k'""", ['k', 'idx'])
+        assert_one(session, """SELECT table_name, index_name FROM system."IndexInfo" WHERE table_name='k'""", ['k', 'idx'])
         assert_one(session, "SELECT * FROM k.t WHERE v = 1", [0, 1])
 
         # Simulate another failing index rebuild
@@ -390,7 +398,7 @@ class TestSecondaryIndexes(Tester):
 
         # Verify that the index is rebuilt, marked as built, and it can answer queries
         self.assertNotEqual(before_files, after_files)
-        assert_one(session, """SELECT * FROM system."IndexInfo" WHERE table_name='k'""", ['k', 'idx'])
+        assert_one(session, """SELECT table_name, index_name FROM system."IndexInfo" WHERE table_name='k'""", ['k', 'idx'])
         assert_one(session, "SELECT * FROM k.t WHERE v = 1", [0, 1])
 
     @since('4.0')
@@ -454,7 +462,7 @@ class TestSecondaryIndexes(Tester):
         before_files = self._index_sstables_files(node, 'k', 't', 'idx')
 
         debug("Verify the index is marked as built and it can be queried")
-        assert_one(session, """SELECT * FROM system."IndexInfo" WHERE table_name='k'""", ['k', 'idx'])
+        assert_one(session, """SELECT table_name, index_name FROM system."IndexInfo" WHERE table_name='k'""", ['k', 'idx'])
         assert_one(session, "SELECT * FROM k.t WHERE v = 1", [0, 1])
 
         debug("Restart the node and verify the index build is not submitted")
@@ -465,7 +473,7 @@ class TestSecondaryIndexes(Tester):
 
         debug("Verify the index is still marked as built and it can be queried")
         session = self.patient_cql_connection(node)
-        assert_one(session, """SELECT * FROM system."IndexInfo" WHERE table_name='k'""", ['k', 'idx'])
+        assert_one(session, """SELECT table_name, index_name FROM system."IndexInfo" WHERE table_name='k'""", ['k', 'idx'])
         assert_one(session, "SELECT * FROM k.t WHERE v = 1", [0, 1])
 
     def test_multi_index_filtering_query(self):

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8e65211b/sstable_generation_loading_test.py
----------------------------------------------------------------------
diff --git a/sstable_generation_loading_test.py b/sstable_generation_loading_test.py
index 37ea8e3..b45b338 100644
--- a/sstable_generation_loading_test.py
+++ b/sstable_generation_loading_test.py
@@ -356,6 +356,6 @@ class TestSSTableGenerationAndLoading(BaseSStableLoaderTest):
         session = self.patient_cql_connection(node)
 
         # Check that the index is marked as built and the index has been rebuilt
-        assert_one(session, """SELECT * FROM system."IndexInfo" WHERE table_name='k'""", ['k', 'idx'])
+        assert_one(session, """SELECT * FROM system."IndexInfo" WHERE table_name='k'""", ['k', 'idx', None])
         assert_all(session, "SELECT * FROM k.t", [[0, 1, 8], [0, 2, 8]])
         assert_all(session, "SELECT * FROM k.t WHERE v = 8", [[0, 1, 8], [0, 2, 8]])

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8e65211b/upgrade_tests/upgrade_compact_storage.py
----------------------------------------------------------------------
diff --git a/upgrade_tests/upgrade_compact_storage.py b/upgrade_tests/upgrade_compact_storage.py
new file mode 100644
index 0000000..085d3a3
--- /dev/null
+++ b/upgrade_tests/upgrade_compact_storage.py
@@ -0,0 +1,176 @@
+# coding: utf-8
+
+import time
+
+from cassandra.query import dict_factory
+from nose.tools import assert_equal, assert_true
+from ccmlib.node import NodeError
+
+from dtest import Tester, debug
+from cassandra.protocol import ConfigurationException
+from tools.decorators import since
+
+VERSION_311 = 'github:apache/cassandra-3.11'
+VERSION_TRUNK = 'github:apache/trunk'
+
+
+@since('4.0')
+class UpgradeSuperColumnsThrough(Tester):
+    def upgrade_to_version(self, tag, start_rpc=True, wait=True, nodes=None):
+        debug('Upgrading to ' + tag)
+        if nodes is None:
+            nodes = self.cluster.nodelist()
+
+        for node in nodes:
+            debug('Shutting down node: ' + node.name)
+            node.drain()
+            node.watch_log_for("DRAINED")
+            node.stop(wait_other_notice=False)
+
+        # Update Cassandra Directory
+        for node in nodes:
+            node.set_install_dir(version=tag)
+            debug("Set new cassandra dir for %s: %s" % (node.name, node.get_install_dir()))
+        self.cluster.set_install_dir(version=tag)
+
+        # Restart nodes on new version
+        for node in nodes:
+            debug('Starting %s on new version (%s)' % (node.name, tag))
+            node.start(wait_other_notice=wait, wait_for_binary_proto=wait)
+
+    def prepare(self, num_nodes=1, cassandra_version="github:apache/cassandra-2.2"):
+        cluster = self.cluster
+
+        # Forcing cluster version on purpose
+        cluster.set_install_dir(version=cassandra_version)
+
+        cluster.populate(num_nodes)
+
+        cluster.start()
+        return cluster
+
+    def upgrade_compact_storage_test(self):
+        cluster = self.prepare(cassandra_version='github:apache/cassandra-3.0')
+        node = self.cluster.nodelist()[0]
+        session = self.patient_cql_connection(node, row_factory=dict_factory)
+
+        session.execute("CREATE KEYSPACE ks WITH replication = {'class': 'SimpleStrategy','replication_factor': '1' };")
+        session.execute("CREATE TABLE ks.compact_table (pk int PRIMARY KEY, col1 int, col2 int) WITH COMPACT STORAGE")
+
+        for i in xrange(1, 5):
+            session.execute("INSERT INTO ks.compact_table (pk, col1, col2) VALUES ({i}, {i}, {i})".format(i=i))
+
+        self.upgrade_to_version(VERSION_TRUNK, wait=False)
+        self.allow_log_errors = True
+
+        time.sleep(5)
+        # After restart, it won't start
+        errors = len(node.grep_log("Compact Tables are not allowed in Cassandra starting with 4.0 version"))
+        assert_true(errors > 0)
+
+    def mixed_cluster_test(self):
+        cluster = self.prepare(num_nodes=2, cassandra_version=VERSION_311)
+        node1, node2 = self.cluster.nodelist()
+
+        node1.drain()
+        node1.watch_log_for("DRAINED")
+        node1.stop(wait_other_notice=False)
+        node1.set_install_dir(version=VERSION_TRUNK)
+        node1.start(wait_other_notice=True, wait_for_binary_proto=True)
+
+        session = self.patient_cql_connection(node2, row_factory=dict_factory)
+
+        # Schema propagation will time out
+        session.execute("CREATE KEYSPACE ks WITH replication = {'class': 'SimpleStrategy','replication_factor': '2' };")
+        thrown = False
+        try:
+            session.execute("CREATE TABLE ks.compact_table (pk int PRIMARY KEY, col1 int, col2 int) WITH COMPACT STORAGE")
+        except ConfigurationException:
+            thrown = True
+
+        assert_true(thrown)
+
+    def upgrade_with_dropped_compact_storage_test(self):
+        cluster = self.prepare(cassandra_version=VERSION_311)
+        node = self.cluster.nodelist()[0]
+        session = self.patient_cql_connection(node, row_factory=dict_factory)
+
+        session.execute("CREATE KEYSPACE ks WITH replication = {'class': 'SimpleStrategy','replication_factor': '1' };")
+        session.execute("CREATE TABLE ks.compact_table (pk int PRIMARY KEY, col1 int, col2 int) WITH COMPACT STORAGE")
+
+        for i in xrange(1, 5):
+            session.execute("INSERT INTO ks.compact_table (pk, col1, col2) VALUES ({i}, {i}, {i})".format(i=i))
+
+        session.execute("ALTER TABLE ks.compact_table DROP COMPACT STORAGE")
+
+        self.upgrade_to_version(VERSION_TRUNK, wait=True)
+
+        session = self.patient_cql_connection(node, row_factory=dict_factory)
+        assert_equal(list(session.execute("SELECT * FROM ks.compact_table WHERE pk = 1")),
+                     [{u'col2': 1, u'pk': 1, u'column1': None, u'value': None, u'col1': 1}])
+
+    def force_readd_compact_storage_test(self):
+        cluster = self.prepare(cassandra_version=VERSION_311)
+        node = self.cluster.nodelist()[0]
+        session = self.patient_cql_connection(node, row_factory=dict_factory)
+
+        session.execute("CREATE KEYSPACE ks WITH replication = {'class': 'SimpleStrategy','replication_factor': '1' };")
+        session.execute("CREATE TABLE ks.compact_table (pk int PRIMARY KEY, col1 int, col2 int) WITH COMPACT STORAGE")
+
+        for i in xrange(1, 5):
+            session.execute("INSERT INTO ks.compact_table (pk, col1, col2) VALUES ({i}, {i}, {i})".format(i=i))
+
+        session.execute("ALTER TABLE ks.compact_table DROP COMPACT STORAGE")
+
+        self.upgrade_to_version(VERSION_TRUNK, wait=True)
+
+        session = self.patient_cql_connection(node, row_factory=dict_factory)
+        session.execute("update system_schema.tables set flags={} where keyspace_name='ks' and table_name='compact_table';")
+
+        assert_equal(list(session.execute("SELECT * FROM ks.compact_table WHERE pk = 1")),
+                     [{u'col2': 1, u'pk': 1, u'column1': None, u'value': None, u'col1': 1}])
+
+        self.allow_log_errors = True
+
+        node.stop(wait_other_notice=False)
+        node.set_install_dir(version=VERSION_TRUNK)
+        try:
+            node.start(wait_other_notice=False, wait_for_binary_proto=False, verbose=False)
+        except (NodeError):
+            print "error"  # ignore
+        time.sleep(5)
+        # After restart, it won't start
+        errors = len(node.grep_log("Compact Tables are not allowed in Cassandra starting with 4.0 version"))
+        assert_true(errors > 0)
+
+    def upgrade_with_dropped_compact_storage_index_test(self):
+        cluster = self.prepare(cassandra_version=VERSION_311)
+        node = self.cluster.nodelist()[0]
+        session = self.patient_cql_connection(node, row_factory=dict_factory)
+
+        session.execute("CREATE KEYSPACE ks WITH replication = {'class': 'SimpleStrategy','replication_factor': '1' };")
+        session.execute("CREATE TABLE ks.compact_table (pk ascii PRIMARY KEY, col1 ascii) WITH COMPACT STORAGE")
+        session.execute("CREATE INDEX ON ks.compact_table(col1)")
+
+        for i in xrange(1, 10):
+            session.execute("INSERT INTO ks.compact_table (pk, col1) VALUES ('{pk}', '{col1}')".format(pk=i, col1=i * 10))
+
+        assert_equal(list(session.execute("SELECT * FROM ks.compact_table WHERE col1 = '50'")),
+                     [{u'pk': '5', u'col1': '50'}])
+        assert_equal(list(session.execute("SELECT * FROM ks.compact_table WHERE pk = '5'")),
+                     [{u'pk': '5', u'col1': '50'}])
+        session.execute("ALTER TABLE ks.compact_table DROP COMPACT STORAGE")
+
+        assert_equal(list(session.execute("SELECT * FROM ks.compact_table WHERE col1 = '50'")),
+                     [{u'col1': '50', u'column1': None, u'pk': '5', u'value': None}])
+        assert_equal(list(session.execute("SELECT * FROM ks.compact_table WHERE pk = '5'")),
+                     [{u'col1': '50', u'column1': None, u'pk': '5', u'value': None}])
+
+        self.upgrade_to_version(VERSION_TRUNK, wait=True)
+
+        session = self.patient_cql_connection(node, row_factory=dict_factory)
+
+        assert_equal(list(session.execute("SELECT * FROM ks.compact_table WHERE col1 = '50'")),
+                     [{u'col1': '50', u'column1': None, u'pk': '5', u'value': None}])
+        assert_equal(list(session.execute("SELECT * FROM ks.compact_table WHERE pk = '5'")),
+                     [{u'col1': '50', u'column1': None, u'pk': '5', u'value': None}])

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8e65211b/write_failures_test.py
----------------------------------------------------------------------
diff --git a/write_failures_test.py b/write_failures_test.py
index 5a0c052..a83e212 100644
--- a/write_failures_test.py
+++ b/write_failures_test.py
@@ -3,7 +3,7 @@ import uuid
 from cassandra import ConsistencyLevel, WriteFailure, WriteTimeout
 
 from distutils.version import LooseVersion
-from dtest import Tester
+from dtest import Tester, supports_v5_protocol
 from thrift_bindings.v22 import ttypes as thrift_types
 from thrift_tests import get_thrift_client
 from tools.decorators import since
@@ -32,7 +32,7 @@ class TestWriteFailures(Tester):
             "MigrationStage"           # This occurs sometimes due to node down (because of restart)
         ]
 
-        self.supports_v5_protocol = self.cluster.version() >= LooseVersion('3.10')
+        self.supports_v5_protocol = supports_v5_protocol(self.cluster.version())
         self.expected_expt = WriteFailure
         self.protocol_version = 5 if self.supports_v5_protocol else 4
         self.replication_factor = 3
@@ -59,7 +59,7 @@ class TestWriteFailures(Tester):
             """ % (KEYSPACE, self.replication_factor))
         session.set_keyspace(KEYSPACE)
 
-        session.execute("CREATE TABLE IF NOT EXISTS mytable (key text PRIMARY KEY, value text) WITH COMPACT STORAGE")
+        session.execute("CREATE TABLE IF NOT EXISTS mytable (key text PRIMARY KEY, value text)")
         session.execute("CREATE TABLE IF NOT EXISTS countertable (key uuid PRIMARY KEY, value counter)")
 
         for idx in self.failing_nodes:


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