You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by bl...@apache.org on 2020/03/16 15:19:10 UTC

[cassandra] branch trunk updated: Use execution timestamp in ALTER TABLE DROP column

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

blerer pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/cassandra.git


The following commit(s) were added to refs/heads/trunk by this push:
     new 722d10b  Use execution timestamp in ALTER TABLE DROP column
722d10b is described below

commit 722d10b2c5999f19f3a912dc2fbaeb71cef65a07
Author: Zhao Yang <zh...@gmail.com>
AuthorDate: Fri Sep 6 11:11:23 2019 +0800

    Use execution timestamp in ALTER TABLE DROP column
    
    patch by Zhao Yang; reviewed by Benjamin Lerer for CASSANDRA-15303
---
 .../apache/cassandra/cql3/MultiColumnRelation.java |   1 +
 .../cassandra/cql3/SingleColumnRelation.java       |   1 +
 .../org/apache/cassandra/cql3/TokenRelation.java   |   2 +-
 .../statements/schema/AlterTableStatement.java     |  16 +-
 test/unit/org/apache/cassandra/cql3/CQLTester.java |  14 ++
 .../cql3/validation/operations/AlterTest.java      | 181 ++++++++++++---------
 6 files changed, 136 insertions(+), 79 deletions(-)

diff --git a/src/java/org/apache/cassandra/cql3/MultiColumnRelation.java b/src/java/org/apache/cassandra/cql3/MultiColumnRelation.java
index 2d239fb..89d69ed 100644
--- a/src/java/org/apache/cassandra/cql3/MultiColumnRelation.java
+++ b/src/java/org/apache/cassandra/cql3/MultiColumnRelation.java
@@ -259,6 +259,7 @@ public class MultiColumnRelation extends Relation
 
         MultiColumnRelation mcr = (MultiColumnRelation) o;
         return Objects.equals(entities, mcr.entities)
+            && Objects.equals(relationType, mcr.relationType)
             && Objects.equals(valuesOrMarker, mcr.valuesOrMarker)
             && Objects.equals(inValues, mcr.inValues)
             && Objects.equals(inMarker, mcr.inMarker);
diff --git a/src/java/org/apache/cassandra/cql3/SingleColumnRelation.java b/src/java/org/apache/cassandra/cql3/SingleColumnRelation.java
index d9c5b26..bf453d7 100644
--- a/src/java/org/apache/cassandra/cql3/SingleColumnRelation.java
+++ b/src/java/org/apache/cassandra/cql3/SingleColumnRelation.java
@@ -172,6 +172,7 @@ public final class SingleColumnRelation extends Relation
 
         SingleColumnRelation scr = (SingleColumnRelation) o;
         return Objects.equals(entity, scr.entity)
+            && Objects.equals(relationType, scr.relationType)
             && Objects.equals(mapKey, scr.mapKey)
             && Objects.equals(value, scr.value)
             && Objects.equals(inValues, scr.inValues);
diff --git a/src/java/org/apache/cassandra/cql3/TokenRelation.java b/src/java/org/apache/cassandra/cql3/TokenRelation.java
index 4e3313d..0919c50 100644
--- a/src/java/org/apache/cassandra/cql3/TokenRelation.java
+++ b/src/java/org/apache/cassandra/cql3/TokenRelation.java
@@ -160,7 +160,7 @@ public final class TokenRelation extends Relation
             return false;
 
         TokenRelation tr = (TokenRelation) o;
-        return entities.equals(tr.entities) && value.equals(tr.value);
+        return relationType.equals(tr.relationType) && entities.equals(tr.entities) && value.equals(tr.value);
     }
 
     /**
diff --git a/src/java/org/apache/cassandra/cql3/statements/schema/AlterTableStatement.java b/src/java/org/apache/cassandra/cql3/statements/schema/AlterTableStatement.java
index 6410e67..d784796 100644
--- a/src/java/org/apache/cassandra/cql3/statements/schema/AlterTableStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/schema/AlterTableStatement.java
@@ -214,9 +214,9 @@ public abstract class AlterTableStatement extends AlterSchemaStatement
     private static class DropColumns extends AlterTableStatement
     {
         private final Collection<ColumnMetadata.Raw> removedColumns;
-        private final long timestamp;
+        private final Long timestamp;
 
-        private DropColumns(String keyspaceName, String tableName, Collection<ColumnMetadata.Raw> removedColumns, long timestamp)
+        private DropColumns(String keyspaceName, String tableName, Collection<ColumnMetadata.Raw> removedColumns, Long timestamp)
         {
             super(keyspaceName, tableName);
             this.removedColumns = removedColumns;
@@ -262,7 +262,15 @@ public abstract class AlterTableStatement extends AlterSchemaStatement
                 throw ire("Cannot drop column %s on base table %s with materialized views", currentColumn, table.name);
 
             builder.removeRegularOrStaticColumn(name);
-            builder.recordColumnDrop(currentColumn, timestamp);
+            builder.recordColumnDrop(currentColumn, getTimestamp());
+        }
+
+        /**
+         * @return timestamp from query, otherwise return current time in micros
+         */
+        private long getTimestamp()
+        {
+            return timestamp == null ? FBUtilities.timestampMicros() : timestamp;
         }
     }
 
@@ -395,7 +403,7 @@ public abstract class AlterTableStatement extends AlterSchemaStatement
 
         // DROP
         private final List<ColumnMetadata.Raw> droppedColumns = new ArrayList<>();
-        private long timestamp = FBUtilities.timestampMicros();
+        private Long timestamp = null; // will use execution timestamp if not provided by query
 
         // RENAME
         private final Map<ColumnMetadata.Raw, ColumnMetadata.Raw> renamedColumns = new HashMap<>();
diff --git a/test/unit/org/apache/cassandra/cql3/CQLTester.java b/test/unit/org/apache/cassandra/cql3/CQLTester.java
index 7da0e2e..2381c52 100644
--- a/test/unit/org/apache/cassandra/cql3/CQLTester.java
+++ b/test/unit/org/apache/cassandra/cql3/CQLTester.java
@@ -649,6 +649,20 @@ public abstract class CQLTester
         return currentKeyspace;
     }
 
+    protected void alterKeyspace(String query)
+    {
+        String fullQuery = String.format(query, currentKeyspace());
+        logger.info(fullQuery);
+        schemaChange(fullQuery);
+    }
+ 
+    protected void alterKeyspaceMayThrow(String query) throws Throwable
+    {
+        String fullQuery = String.format(query, currentKeyspace());
+        logger.info(fullQuery);
+        QueryProcessor.executeOnceInternal(fullQuery);
+    }
+    
     protected String createKeyspaceName()
     {
         String currentKeyspace = "keyspace_" + seqNumber.getAndIncrement();
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/AlterTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/AlterTest.java
index ea78f88..91a3c64 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/AlterTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/AlterTest.java
@@ -18,10 +18,10 @@
 package org.apache.cassandra.cql3.validation.operations;
 
 import java.util.UUID;
-
-import org.junit.Assert;
 import org.junit.Test;
 
+import com.datastax.driver.core.PreparedStatement;
+
 import org.apache.cassandra.dht.OrderPreservingPartitioner;
 import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.locator.TokenMetadata;
@@ -37,14 +37,33 @@ import org.apache.cassandra.utils.FBUtilities;
 
 import static java.lang.String.format;
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
 public class AlterTest extends CQLTester
 {
     @Test
+    public void testDropColumnAsPreparedStatement() throws Throwable
+    {
+        String table = createTable("CREATE TABLE %s (key int PRIMARY KEY, value int);");
+
+        PreparedStatement prepared = sessionNet().prepare("ALTER TABLE " + KEYSPACE + "." + table + " DROP value;");
+
+        executeNet("INSERT INTO %s (key, value) VALUES (1, 1)");
+        assertRowsNet(executeNet("SELECT * FROM %s"), row(1, 1));
+
+        sessionNet().execute(prepared.bind());
+
+        executeNet("ALTER TABLE %s ADD value int");
+
+        assertRows(execute("SELECT * FROM %s"), row(1, null));
+    }
+
+    @Test
     public void testAddList() throws Throwable
     {
         createTable("CREATE TABLE %s (id text PRIMARY KEY, content text);");
-        execute("ALTER TABLE %s ADD myCollection list<text>;");
+        alterTable("ALTER TABLE %s ADD myCollection list<text>;");
         execute("INSERT INTO %s (id, content , myCollection) VALUES ('test', 'first test', ['first element']);");
 
         assertRows(execute("SELECT * FROM %s;"), row("test", "first test", list("first element")));
@@ -55,7 +74,7 @@ public class AlterTest extends CQLTester
     {
         createTable("CREATE TABLE %s (id text PRIMARY KEY, content text, myCollection list<text>);");
         execute("INSERT INTO %s (id, content , myCollection) VALUES ('test', 'first test', ['first element']);");
-        execute("ALTER TABLE %s DROP myCollection;");
+        alterTable("ALTER TABLE %s DROP myCollection;");
 
         assertRows(execute("SELECT * FROM %s;"), row("test", "first test"));
     }
@@ -64,7 +83,7 @@ public class AlterTest extends CQLTester
     public void testAddMap() throws Throwable
     {
         createTable("CREATE TABLE %s (id text PRIMARY KEY, content text);");
-        execute("ALTER TABLE %s ADD myCollection map<text, text>;");
+        alterTable("ALTER TABLE %s ADD myCollection map<text, text>;");
         execute("INSERT INTO %s (id, content , myCollection) VALUES ('test', 'first test', { '1' : 'first element'});");
 
         assertRows(execute("SELECT * FROM %s;"), row("test", "first test", map("1", "first element")));
@@ -75,7 +94,7 @@ public class AlterTest extends CQLTester
     {
         createTable("CREATE TABLE %s (id text PRIMARY KEY, content text, myCollection map<text, text>);");
         execute("INSERT INTO %s (id, content , myCollection) VALUES ('test', 'first test', { '1' : 'first element'});");
-        execute("ALTER TABLE %s DROP myCollection;");
+        alterTable("ALTER TABLE %s DROP myCollection;");
 
         assertRows(execute("SELECT * FROM %s;"), row("test", "first test"));
     }
@@ -85,9 +104,8 @@ public class AlterTest extends CQLTester
     {
         createTable("CREATE TABLE %s (id text PRIMARY KEY, content text, myCollection list<text>);");
         execute("INSERT INTO %s (id, content , myCollection) VALUES ('test', 'first test', ['first element']);");
-        execute("ALTER TABLE %s DROP myCollection;");
-        execute("ALTER TABLE %s ADD myCollection list<text>;");
-
+        alterTable("ALTER TABLE %s DROP myCollection;");
+        alterTable("ALTER TABLE %s ADD myCollection list<text>;");
         assertRows(execute("SELECT * FROM %s;"), row("test", "first test", null));
         execute("UPDATE %s set myCollection = ['second element'] WHERE id = 'test';");
         assertRows(execute("SELECT * FROM %s;"), row("test", "first test", list("second element")));
@@ -98,7 +116,7 @@ public class AlterTest extends CQLTester
     {
         createTable("CREATE TABLE %s (id text PRIMARY KEY, content text, myCollection list<text>);");
         execute("INSERT INTO %s (id, content , myCollection) VALUES ('test', 'first test', ['first element']);");
-        execute("ALTER TABLE %s DROP myCollection;");
+        alterTable("ALTER TABLE %s DROP myCollection;");
 
         assertInvalid("ALTER TABLE %s ADD myCollection map<int, int>;");
     }
@@ -113,8 +131,8 @@ public class AlterTest extends CQLTester
         // flush is necessary since otherwise the values of `todrop` will get discarded during
         // alter statement
         flush(true);
-        execute("ALTER TABLE %s DROP todrop USING TIMESTAMP 20000;");
-        execute("ALTER TABLE %s ADD todrop int;");
+        alterTable("ALTER TABLE %s DROP todrop USING TIMESTAMP 20000;");
+        alterTable("ALTER TABLE %s ADD todrop int;");
         execute("INSERT INTO %s (id, c1, v1, todrop) VALUES (?, ?, ?, ?) USING TIMESTAMP ?", 1, 100, 100, 100, 30000L);
         assertRows(execute("SELECT id, c1, v1, todrop FROM %s"),
                    row(1, 0, 0, null),
@@ -130,8 +148,8 @@ public class AlterTest extends CQLTester
     {
         createTable("CREATE TABLE %s (a int, b int, c int, d int static, PRIMARY KEY (a, b));");
 
-        execute("ALTER TABLE %s DROP c;");
-        execute("ALTER TABLE %s DROP d;");
+        alterTable("ALTER TABLE %s DROP c;");
+        alterTable("ALTER TABLE %s DROP d;");
 
         assertInvalidMessage("Cannot re-add previously dropped column 'c' of kind STATIC, incompatible with previous kind REGULAR",
                              "ALTER TABLE %s ADD c int static;");
@@ -150,8 +168,8 @@ public class AlterTest extends CQLTester
         // flush is necessary since otherwise the values of `todrop` will get discarded during
         // alter statement
         flush(true);
-        execute("ALTER TABLE %s DROP todrop USING TIMESTAMP 20000;");
-        execute("ALTER TABLE %s ADD todrop int static;");
+        alterTable("ALTER TABLE %s DROP todrop USING TIMESTAMP 20000;");
+        alterTable("ALTER TABLE %s ADD todrop int static;");
         execute("INSERT INTO %s (id, c1, v1, todrop) VALUES (?, ?, ?, ?) USING TIMESTAMP ?", 1, 100, 100, 100, 30000L);
         // static column value with largest timestmap will be available again
         assertRows(execute("SELECT id, c1, v1, todrop FROM %s"),
@@ -173,9 +191,9 @@ public class AlterTest extends CQLTester
         // flush is necessary since otherwise the values of `todrop1` and `todrop2` will get discarded during
         // alter statement
         flush(true);
-        execute("ALTER TABLE %s DROP (todrop1, todrop2) USING TIMESTAMP 20000;");
-        execute("ALTER TABLE %s ADD todrop1 int;");
-        execute("ALTER TABLE %s ADD todrop2 int;");
+        alterTable("ALTER TABLE %s DROP (todrop1, todrop2) USING TIMESTAMP 20000;");
+        alterTable("ALTER TABLE %s ADD todrop1 int;");
+        alterTable("ALTER TABLE %s ADD todrop2 int;");
 
         execute("INSERT INTO %s (id, c1, v1, todrop1, todrop2) VALUES (?, ?, ?, ?, ?) USING TIMESTAMP ?", 1, 100, 100, 100, 100, 40000L);
         assertRows(execute("SELECT id, c1, v1, todrop1, todrop2 FROM %s"),
@@ -352,8 +370,8 @@ public class AlterTest extends CQLTester
         assertInvalidThrow(ConfigurationException.class, "ALTER KEYSPACE testXYZ WITH replication={ 'class' : 'SimpleStrategy' }");
 
         // Make sure that the alter works as expected
-        execute("ALTER KEYSPACE testABC WITH replication={ 'class' : 'NetworkTopologyStrategy', '" + DATA_CENTER + "' : 2 }");
-        execute("ALTER KEYSPACE testXYZ WITH replication={ 'class' : 'SimpleStrategy', 'replication_factor' : 2 }");
+        alterTable("ALTER KEYSPACE testABC WITH replication={ 'class' : 'NetworkTopologyStrategy', '" + DATA_CENTER + "' : 2 }");
+        alterTable("ALTER KEYSPACE testXYZ WITH replication={ 'class' : 'SimpleStrategy', 'replication_factor' : 2 }");
 
         // clean up
         execute("DROP KEYSPACE IF EXISTS testABC");
@@ -367,36 +385,32 @@ public class AlterTest extends CQLTester
     public void testAlterKeyspaceWithNTSOnlyAcceptsConfiguredDataCenterNames() throws Throwable
     {
         // Create a keyspace with expected DC name.
-        execute("CREATE KEYSPACE testABC WITH replication = {'class' : 'NetworkTopologyStrategy', '" + DATA_CENTER + "' : 2 }");
+        createKeyspace("CREATE KEYSPACE %s WITH replication = {'class' : 'NetworkTopologyStrategy', '" + DATA_CENTER + "' : 2 }");
 
         // try modifying the keyspace
-        assertInvalidThrow(ConfigurationException.class, "ALTER KEYSPACE testABC WITH replication = { 'class' : 'NetworkTopologyStrategy', 'INVALID_DC' : 2 }");
-        execute("ALTER KEYSPACE testABC WITH replication = {'class' : 'NetworkTopologyStrategy', '" + DATA_CENTER + "' : 3 }");
+        assertAlterKeyspaceThrowsException(ConfigurationException.class,
+                                           "Unrecognized strategy option {INVALID_DC} passed to NetworkTopologyStrategy for keyspace " + currentKeyspace(),
+                                           "ALTER KEYSPACE %s WITH replication = { 'class' : 'NetworkTopologyStrategy', 'INVALID_DC' : 2 }");
 
-        // Mix valid and invalid, should throw an exception
-        assertInvalidThrow(ConfigurationException.class, "ALTER KEYSPACE testABC WITH replication={ 'class' : 'NetworkTopologyStrategy', '" + DATA_CENTER + "' : 2 , 'INVALID_DC': 1}");
+        alterKeyspace("ALTER KEYSPACE %s WITH replication = {'class' : 'NetworkTopologyStrategy', '" + DATA_CENTER + "' : 3 }");
 
-        // clean-up
-        execute("DROP KEYSPACE IF EXISTS testABC");
+        // Mix valid and invalid, should throw an exception
+        assertAlterKeyspaceThrowsException(ConfigurationException.class,
+                                           "Unrecognized strategy option {INVALID_DC} passed to NetworkTopologyStrategy for keyspace " + currentKeyspace(),
+                                           "ALTER KEYSPACE %s WITH replication={ 'class' : 'NetworkTopologyStrategy', '" + DATA_CENTER + "' : 2 , 'INVALID_DC': 1}");
     }
 
     @Test
     public void testAlterKeyspaceWithMultipleInstancesOfSameDCThrowsSyntaxException() throws Throwable
     {
-        try
-        {
-            // Create a keyspace
-            execute("CREATE KEYSPACE testABC WITH replication = {'class' : 'NetworkTopologyStrategy', '" + DATA_CENTER + "' : 2}");
+        // Create a keyspace
+        createKeyspace("CREATE KEYSPACE %s WITH replication = {'class' : 'NetworkTopologyStrategy', '" + DATA_CENTER + "' : 2}");
 
-            // try modifying the keyspace
-            assertInvalidThrow(SyntaxException.class, "ALTER KEYSPACE testABC WITH replication = {'class' : 'NetworkTopologyStrategy', '" + DATA_CENTER + "' : 2, '" + DATA_CENTER + "' : 3 }");
-            execute("ALTER KEYSPACE testABC WITH replication = {'class' : 'NetworkTopologyStrategy', '" + DATA_CENTER + "' : 3}");
-        }
-        finally
-        {
-            // clean-up
-            execute("DROP KEYSPACE IF EXISTS testABC");
-        }
+        // try modifying the keyspace
+        assertAlterTableThrowsException(SyntaxException.class,
+                                        "",
+                                        "ALTER KEYSPACE %s WITH replication = {'class' : 'NetworkTopologyStrategy', '" + DATA_CENTER + "' : 2, '" + DATA_CENTER + "' : 3 }");
+        alterKeyspace("ALTER KEYSPACE %s WITH replication = {'class' : 'NetworkTopologyStrategy', '" + DATA_CENTER + "' : 3}");
     }
 
     /**
@@ -410,11 +424,11 @@ public class AlterTest extends CQLTester
 
         execute("UPDATE %s SET t = '111' WHERE id = 1");
 
-        execute("ALTER TABLE %s ADD l list<text>");
+        alterTable("ALTER TABLE %s ADD l list<text>");
         assertRows(execute("SELECT * FROM %s"),
                    row(1, null, "111"));
 
-        execute("ALTER TABLE %s ADD m map<int, text>");
+        alterTable("ALTER TABLE %s ADD m map<int, text>");
         assertRows(execute("SELECT * FROM %s"),
                    row(1, null, null, "111"));
     }
@@ -429,8 +443,8 @@ public class AlterTest extends CQLTester
         createTable("create table %s (k int primary key, v set<text>)");
         execute("insert into %s (k, v) VALUES (0, {'f'})");
         flush();
-        execute("alter table %s drop v");
-        execute("alter table %s add v1 int");
+        alterTable("alter table %s drop v");
+        alterTable("alter table %s add v1 int");
     }
 
     @Test
@@ -441,7 +455,7 @@ public class AlterTest extends CQLTester
                            "ALTER KEYSPACE ks WITH WITH DURABLE_WRITES = true" };
 
         for (String stmt : stmts) {
-            assertInvalidSyntaxMessage("no viable alternative at input 'WITH'", stmt);
+            assertAlterTableThrowsException(SyntaxException.class, "no viable alternative at input 'WITH'", stmt);
         }
     }
 
@@ -457,7 +471,7 @@ public class AlterTest extends CQLTester
                            currentTable()),
                    row(map("chunk_length_in_kb", "16", "class", "org.apache.cassandra.io.compress.LZ4Compressor")));
 
-        execute("ALTER TABLE %s WITH compression = { 'class' : 'SnappyCompressor', 'chunk_length_in_kb' : 32 };");
+        alterTable("ALTER TABLE %s WITH compression = { 'class' : 'SnappyCompressor', 'chunk_length_in_kb' : 32 };");
 
         assertRows(execute(format("SELECT compression FROM %s.%s WHERE keyspace_name = ? and table_name = ?;",
                                   SchemaConstants.SCHEMA_KEYSPACE_NAME,
@@ -466,7 +480,7 @@ public class AlterTest extends CQLTester
                            currentTable()),
                    row(map("chunk_length_in_kb", "32", "class", "org.apache.cassandra.io.compress.SnappyCompressor")));
 
-        execute("ALTER TABLE %s WITH compression = { 'class' : 'LZ4Compressor', 'chunk_length_in_kb' : 64 };");
+        alterTable("ALTER TABLE %s WITH compression = { 'class' : 'LZ4Compressor', 'chunk_length_in_kb' : 64 };");
 
         assertRows(execute(format("SELECT compression FROM %s.%s WHERE keyspace_name = ? and table_name = ?;",
                                   SchemaConstants.SCHEMA_KEYSPACE_NAME,
@@ -475,7 +489,7 @@ public class AlterTest extends CQLTester
                            currentTable()),
                    row(map("chunk_length_in_kb", "64", "class", "org.apache.cassandra.io.compress.LZ4Compressor")));
 
-        execute("ALTER TABLE %s WITH compression = { 'class' : 'LZ4Compressor', 'min_compress_ratio' : 2 };");
+        alterTable("ALTER TABLE %s WITH compression = { 'class' : 'LZ4Compressor', 'min_compress_ratio' : 2 };");
 
         assertRows(execute(format("SELECT compression FROM %s.%s WHERE keyspace_name = ? and table_name = ?;",
                                   SchemaConstants.SCHEMA_KEYSPACE_NAME,
@@ -484,7 +498,7 @@ public class AlterTest extends CQLTester
                            currentTable()),
                    row(map("chunk_length_in_kb", "16", "class", "org.apache.cassandra.io.compress.LZ4Compressor", "min_compress_ratio", "2.0")));
 
-        execute("ALTER TABLE %s WITH compression = { 'class' : 'LZ4Compressor', 'min_compress_ratio' : 1 };");
+        alterTable("ALTER TABLE %s WITH compression = { 'class' : 'LZ4Compressor', 'min_compress_ratio' : 1 };");
 
         assertRows(execute(format("SELECT compression FROM %s.%s WHERE keyspace_name = ? and table_name = ?;",
                                   SchemaConstants.SCHEMA_KEYSPACE_NAME,
@@ -493,7 +507,7 @@ public class AlterTest extends CQLTester
                            currentTable()),
                    row(map("chunk_length_in_kb", "16", "class", "org.apache.cassandra.io.compress.LZ4Compressor", "min_compress_ratio", "1.0")));
 
-        execute("ALTER TABLE %s WITH compression = { 'class' : 'LZ4Compressor', 'min_compress_ratio' : 0 };");
+        alterTable("ALTER TABLE %s WITH compression = { 'class' : 'LZ4Compressor', 'min_compress_ratio' : 0 };");
 
         assertRows(execute(format("SELECT compression FROM %s.%s WHERE keyspace_name = ? and table_name = ?;",
                                   SchemaConstants.SCHEMA_KEYSPACE_NAME,
@@ -502,8 +516,8 @@ public class AlterTest extends CQLTester
                            currentTable()),
                    row(map("chunk_length_in_kb", "16", "class", "org.apache.cassandra.io.compress.LZ4Compressor")));
 
-        execute("ALTER TABLE %s WITH compression = { 'class' : 'SnappyCompressor', 'chunk_length_in_kb' : 32 };");
-        execute("ALTER TABLE %s WITH compression = { 'enabled' : 'false'};");
+        alterTable("ALTER TABLE %s WITH compression = { 'class' : 'SnappyCompressor', 'chunk_length_in_kb' : 32 };");
+        alterTable("ALTER TABLE %s WITH compression = { 'enabled' : 'false'};");
 
         assertRows(execute(format("SELECT compression FROM %s.%s WHERE keyspace_name = ? and table_name = ?;",
                                   SchemaConstants.SCHEMA_KEYSPACE_NAME,
@@ -512,38 +526,57 @@ public class AlterTest extends CQLTester
                            currentTable()),
                    row(map("enabled", "false")));
 
-        assertThrowsConfigurationException("Missing sub-option 'class' for the 'compression' option.",
-                                           "ALTER TABLE %s WITH  compression = {'chunk_length_in_kb' : 32};");
+        assertAlterTableThrowsException(ConfigurationException.class,
+                                        "Missing sub-option 'class' for the 'compression' option.",
+                                        "ALTER TABLE %s WITH  compression = {'chunk_length_in_kb' : 32};");
+
+        assertAlterTableThrowsException(ConfigurationException.class,
+                                        "The 'class' option must not be empty. To disable compression use 'enabled' : false",
+                                        "ALTER TABLE %s WITH  compression = { 'class' : ''};");
 
-        assertThrowsConfigurationException("The 'class' option must not be empty. To disable compression use 'enabled' : false",
-                                           "ALTER TABLE %s WITH  compression = { 'class' : ''};");
+        assertAlterTableThrowsException(ConfigurationException.class,
+                                        "If the 'enabled' option is set to false no other options must be specified",
+                                        "ALTER TABLE %s WITH compression = { 'enabled' : 'false', 'class' : 'SnappyCompressor'};");
 
-        assertThrowsConfigurationException("If the 'enabled' option is set to false no other options must be specified",
-                                           "ALTER TABLE %s WITH compression = { 'enabled' : 'false', 'class' : 'SnappyCompressor'};");
+        assertAlterTableThrowsException(ConfigurationException.class,
+                                        "The 'sstable_compression' option must not be used if the compression algorithm is already specified by the 'class' option",
+                                        "ALTER TABLE %s WITH compression = { 'sstable_compression' : 'SnappyCompressor', 'class' : 'SnappyCompressor'};");
 
-        assertThrowsConfigurationException("The 'sstable_compression' option must not be used if the compression algorithm is already specified by the 'class' option",
-                                           "ALTER TABLE %s WITH compression = { 'sstable_compression' : 'SnappyCompressor', 'class' : 'SnappyCompressor'};");
+        assertAlterTableThrowsException(ConfigurationException.class,
+                                        "The 'chunk_length_kb' option must not be used if the chunk length is already specified by the 'chunk_length_in_kb' option",
+                                        "ALTER TABLE %s WITH compression = { 'class' : 'SnappyCompressor', 'chunk_length_kb' : 32 , 'chunk_length_in_kb' : 32 };");
 
-        assertThrowsConfigurationException("The 'chunk_length_kb' option must not be used if the chunk length is already specified by the 'chunk_length_in_kb' option",
-                                           "ALTER TABLE %s WITH compression = { 'class' : 'SnappyCompressor', 'chunk_length_kb' : 32 , 'chunk_length_in_kb' : 32 };");
+        assertAlterTableThrowsException(ConfigurationException.class,
+                                        "Invalid negative min_compress_ratio",
+                                        "ALTER TABLE %s WITH compression = { 'class' : 'SnappyCompressor', 'min_compress_ratio' : -1 };");
 
-        assertThrowsConfigurationException("Invalid negative min_compress_ratio",
-                                           "ALTER TABLE %s WITH compression = { 'class' : 'SnappyCompressor', 'min_compress_ratio' : -1 };");
+        assertAlterTableThrowsException(ConfigurationException.class,
+                                        "min_compress_ratio can either be 0 or greater than or equal to 1",
+                                        "ALTER TABLE %s WITH compression = { 'class' : 'SnappyCompressor', 'min_compress_ratio' : 0.5 };");
+    }
 
-        assertThrowsConfigurationException("min_compress_ratio can either be 0 or greater than or equal to 1",
-                                           "ALTER TABLE %s WITH compression = { 'class' : 'SnappyCompressor', 'min_compress_ratio' : 0.5 };");
+    private void assertAlterKeyspaceThrowsException(Class<? extends Throwable> clazz, String msg, String stmt)
+    {
+        assertThrowsException(clazz, msg, () -> {alterKeyspaceMayThrow(stmt);});
+    }
+    
+    private void assertAlterTableThrowsException(Class<? extends Throwable> clazz, String msg, String stmt)
+    {
+        assertThrowsException(clazz, msg, () -> {alterTableMayThrow(stmt);});
     }
 
-    private void assertThrowsConfigurationException(String errorMsg, String alterStmt) throws Throwable
+    private static void assertThrowsException(Class<? extends Throwable> clazz, String msg, CheckedFunction function)
     {
         try
         {
-            execute(alterStmt);
-            Assert.fail("Query should be invalid but no error was thrown. Query is: " + alterStmt);
+            function.apply();
+            fail("An error should havee been thrown but was not.");
         }
-        catch (ConfigurationException e)
+        catch (Throwable e)
         {
-            assertEquals(errorMsg, e.getMessage());
+            assertTrue("Unexpected exception type (expected: " + clazz + ", value: " + e.getClass() + ")",
+                       clazz.isAssignableFrom(e.getClass()));
+            assertTrue("Expecting the error message to contains: '" + msg + "' but was " + e.getMessage(), e.getMessage().contains(msg));
         }
     }
 
@@ -564,7 +597,7 @@ public class AlterTest extends CQLTester
 
         flush();
 
-        execute("ALTER TABLE %s DROP x");
+        alterTable("ALTER TABLE %s DROP x");
 
         compact();
 
@@ -593,7 +626,7 @@ public class AlterTest extends CQLTester
         if (flushAfterInsert)
             flush();
 
-        execute("ALTER TABLE %s DROP x");
+        alterTable("ALTER TABLE %s DROP x");
 
         assertEmpty(execute("SELECT * FROM %s"));
     }


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