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 2018/07/17 15:40:24 UTC

[02/13] cassandra git commit: Make all DDL statements idempotent and not dependent on global state

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/test/unit/org/apache/cassandra/cql3/validation/entities/TypeTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/entities/TypeTest.java b/test/unit/org/apache/cassandra/cql3/validation/entities/TypeTest.java
index bb43d75..ceb96b6 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/entities/TypeTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/TypeTest.java
@@ -28,8 +28,8 @@ public class TypeTest extends CQLTester
     @Test
     public void testNonExistingOnes() throws Throwable
     {
-        assertInvalidMessage("No user type named", "DROP TYPE " + KEYSPACE + ".type_does_not_exist");
-        assertInvalidMessage("Cannot drop type in unknown keyspace", "DROP TYPE keyspace_does_not_exist.type_does_not_exist");
+        assertInvalidMessage(String.format("Type '%s.type_does_not_exist' doesn't exist", KEYSPACE), "DROP TYPE " + KEYSPACE + ".type_does_not_exist");
+        assertInvalidMessage("Type 'keyspace_does_not_exist.type_does_not_exist' doesn't exist", "DROP TYPE keyspace_does_not_exist.type_does_not_exist");
 
         execute("DROP TYPE IF EXISTS " + KEYSPACE + ".type_does_not_exist");
         execute("DROP TYPE IF EXISTS keyspace_does_not_exist.type_does_not_exist");

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/test/unit/org/apache/cassandra/cql3/validation/entities/UFAuthTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/entities/UFAuthTest.java b/test/unit/org/apache/cassandra/cql3/validation/entities/UFAuthTest.java
index ecff0cc..4a2d71f 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/entities/UFAuthTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/UFAuthTest.java
@@ -226,7 +226,7 @@ public class UFAuthTest extends CQLTester
             functions.add(functionName);
             statements.add(stmt);
         }
-        BatchStatement batch = new BatchStatement(-1, BatchStatement.Type.LOGGED, statements, Attributes.none());
+        BatchStatement batch = new BatchStatement(BatchStatement.Type.LOGGED, VariableSpecifications.empty(), statements, Attributes.none());
         assertUnauthorized(batch, functions);
 
         grantExecuteOnFunction(functions.get(0));
@@ -236,7 +236,7 @@ public class UFAuthTest extends CQLTester
         assertUnauthorized(batch, functions.subList(2, functions.size()));
 
         grantExecuteOnFunction(functions.get(2));
-        batch.checkAccess(clientState);
+        batch.authorize(clientState);
     }
 
     @Test
@@ -313,7 +313,7 @@ public class UFAuthTest extends CQLTester
         // with terminal arguments, so evaluated at prepare time
         String cql = String.format("UPDATE %s SET v2 = 0 WHERE k = blobasint(intasblob(0)) and v1 = 0",
                                    KEYSPACE + "." + currentTable());
-        getStatement(cql).checkAccess(clientState);
+        getStatement(cql).authorize(clientState);
 
         // with non-terminal arguments, so evaluated at execution
         String functionName = createSimpleFunction();
@@ -321,7 +321,7 @@ public class UFAuthTest extends CQLTester
         cql = String.format("UPDATE %s SET v2 = 0 WHERE k = blobasint(intasblob(%s)) and v1 = 0",
                             KEYSPACE + "." + currentTable(),
                             functionCall(functionName));
-        getStatement(cql).checkAccess(clientState);
+        getStatement(cql).authorize(clientState);
     }
 
     @Test
@@ -343,7 +343,7 @@ public class UFAuthTest extends CQLTester
         assertUnauthorized(aggDef, fFunc, "int");
         grantExecuteOnFunction(fFunc);
 
-        getStatement(aggDef).checkAccess(clientState);
+        getStatement(aggDef).authorize(clientState);
     }
 
     @Test
@@ -361,24 +361,24 @@ public class UFAuthTest extends CQLTester
         String cql = String.format("SELECT %s(v1) FROM %s",
                                    aggregate,
                                    KEYSPACE + "." + currentTable());
-        getStatement(cql).checkAccess(clientState);
+        getStatement(cql).authorize(clientState);
 
         // check that revoking EXECUTE permission on any one of the
         // component functions means we lose the ability to execute it
         revokeExecuteOnFunction(aggregate);
         assertUnauthorized(cql, aggregate, "int");
         grantExecuteOnFunction(aggregate);
-        getStatement(cql).checkAccess(clientState);
+        getStatement(cql).authorize(clientState);
 
         revokeExecuteOnFunction(sFunc);
         assertUnauthorized(cql, sFunc, "int, int");
         grantExecuteOnFunction(sFunc);
-        getStatement(cql).checkAccess(clientState);
+        getStatement(cql).authorize(clientState);
 
         revokeExecuteOnFunction(fFunc);
         assertUnauthorized(cql, fFunc, "int");
         grantExecuteOnFunction(fFunc);
-        getStatement(cql).checkAccess(clientState);
+        getStatement(cql).authorize(clientState);
     }
 
     @Test
@@ -410,7 +410,7 @@ public class UFAuthTest extends CQLTester
         assertUnauthorized(cql, aggregate, "int");
         grantExecuteOnFunction(aggregate);
 
-        getStatement(cql).checkAccess(clientState);
+        getStatement(cql).authorize(clientState);
     }
 
     @Test
@@ -442,7 +442,7 @@ public class UFAuthTest extends CQLTester
         assertUnauthorized(cql, innerFunc, "int");
         grantExecuteOnFunction(innerFunc);
 
-        getStatement(cql).checkAccess(clientState);
+        getStatement(cql).authorize(clientState);
     }
 
     @Test
@@ -484,7 +484,7 @@ public class UFAuthTest extends CQLTester
         grantExecuteOnFunction(innerFunction);
 
         // now execution of both is permitted
-        getStatement(cql).checkAccess(clientState);
+        getStatement(cql).authorize(clientState);
     }
 
     private void assertPermissionsOnFunction(String cql, String functionName) throws Throwable
@@ -496,14 +496,14 @@ public class UFAuthTest extends CQLTester
     {
         assertUnauthorized(cql, functionName, argTypes);
         grantExecuteOnFunction(functionName);
-        getStatement(cql).checkAccess(clientState);
+        getStatement(cql).authorize(clientState);
     }
 
     private void assertUnauthorized(BatchStatement batch, Iterable<String> functionNames) throws Throwable
     {
         try
         {
-            batch.checkAccess(clientState);
+            batch.authorize(clientState);
             fail("Expected an UnauthorizedException, but none was thrown");
         }
         catch (UnauthorizedException e)
@@ -520,7 +520,7 @@ public class UFAuthTest extends CQLTester
     {
         try
         {
-            getStatement(cql).checkAccess(clientState);
+            getStatement(cql).authorize(clientState);
             fail("Expected an UnauthorizedException, but none was thrown");
         }
         catch (UnauthorizedException e)
@@ -625,7 +625,7 @@ public class UFAuthTest extends CQLTester
 
     private CQLStatement getStatement(String cql)
     {
-        return QueryProcessor.getStatement(cql, clientState).statement;
+        return QueryProcessor.getStatement(cql, clientState);
     }
 
     private FunctionResource functionResource(String functionName)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/test/unit/org/apache/cassandra/cql3/validation/entities/UFIdentificationTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/entities/UFIdentificationTest.java b/test/unit/org/apache/cassandra/cql3/validation/entities/UFIdentificationTest.java
index b2288e4..bba5c92 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/entities/UFIdentificationTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/UFIdentificationTest.java
@@ -28,6 +28,7 @@ import org.junit.Test;
 import org.apache.cassandra.cql3.Attributes;
 import org.apache.cassandra.cql3.CQLStatement;
 import org.apache.cassandra.cql3.QueryProcessor;
+import org.apache.cassandra.cql3.VariableSpecifications;
 import org.apache.cassandra.cql3.functions.Function;
 import org.apache.cassandra.cql3.statements.BatchStatement;
 import org.apache.cassandra.cql3.statements.ModificationStatement;
@@ -307,7 +308,7 @@ public class UFIdentificationTest extends CQLTester
         statements.add(modificationStatement(cql("INSERT INTO %s (key, i_cc, t_cc) VALUES (2, 2, %s)",
                                                  functionCall(tFunc, "'foo'"))));
 
-        BatchStatement batch = new BatchStatement(-1, BatchStatement.Type.LOGGED, statements, Attributes.none());
+        BatchStatement batch = new BatchStatement(BatchStatement.Type.LOGGED, VariableSpecifications.empty(), statements, Attributes.none());
         assertFunctions(batch, iFunc, iFunc2, tFunc);
     }
 
@@ -320,18 +321,18 @@ public class UFIdentificationTest extends CQLTester
         statements.add(modificationStatement(cql("UPDATE %s SET i_val = %s WHERE key=0 AND i_cc=1 and t_cc='foo' IF s_val = %s",
                                                  functionCall(iFunc, "0"), functionCall(sFunc, "{1}"))));
 
-        BatchStatement batch = new BatchStatement(-1, BatchStatement.Type.LOGGED, statements, Attributes.none());
+        BatchStatement batch = new BatchStatement(BatchStatement.Type.LOGGED, VariableSpecifications.empty(), statements, Attributes.none());
         assertFunctions(batch, iFunc, lFunc, sFunc);
     }
 
     private ModificationStatement modificationStatement(String cql)
     {
-        return (ModificationStatement) QueryProcessor.getStatement(cql, ClientState.forInternalCalls()).statement;
+        return (ModificationStatement) QueryProcessor.getStatement(cql, ClientState.forInternalCalls());
     }
 
     private void assertFunctions(String cql, String... function)
     {
-        CQLStatement stmt = QueryProcessor.getStatement(cql, ClientState.forInternalCalls()).statement;
+        CQLStatement stmt = QueryProcessor.getStatement(cql, ClientState.forInternalCalls());
         assertFunctions(stmt, function);
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/test/unit/org/apache/cassandra/cql3/validation/entities/UFJavaTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/entities/UFJavaTest.java b/test/unit/org/apache/cassandra/cql3/validation/entities/UFJavaTest.java
index 187871a..975b21a 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/entities/UFJavaTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/UFJavaTest.java
@@ -113,8 +113,8 @@ public class UFJavaTest extends CQLTester
     @Test
     public void testJavaFunctionInvalidReturn() throws Throwable
     {
-        assertInvalidMessage("system keyspace is not user-modifiable",
-                             "CREATE OR REPLACE FUNCTION jfir(val double) " +
+        assertInvalidMessage("cannot convert from long to Double",
+                             "CREATE OR REPLACE FUNCTION " + KEYSPACE + ".jfir(val double) " +
                              "RETURNS NULL ON NULL INPUT " +
                              "RETURNS double " +
                              "LANGUAGE JAVA\n" +

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/test/unit/org/apache/cassandra/cql3/validation/entities/UFTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/entities/UFTest.java b/test/unit/org/apache/cassandra/cql3/validation/entities/UFTest.java
index 7940b92..d21d159 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/entities/UFTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/UFTest.java
@@ -58,10 +58,21 @@ public class UFTest extends CQLTester
     @Test
     public void testNonExistingOnes() throws Throwable
     {
-        assertInvalidThrowMessage("Cannot drop non existing function", InvalidRequestException.class, "DROP FUNCTION " + KEYSPACE + ".func_does_not_exist");
-        assertInvalidThrowMessage("Cannot drop non existing function", InvalidRequestException.class, "DROP FUNCTION " + KEYSPACE + ".func_does_not_exist(int,text)");
-        assertInvalidThrowMessage("Cannot drop non existing function", InvalidRequestException.class, "DROP FUNCTION keyspace_does_not_exist.func_does_not_exist");
-        assertInvalidThrowMessage("Cannot drop non existing function", InvalidRequestException.class, "DROP FUNCTION keyspace_does_not_exist.func_does_not_exist(int,text)");
+        assertInvalidThrowMessage(String.format("Function '%s.func_does_not_exist' doesn't exist", KEYSPACE),
+                                  InvalidRequestException.class,
+                                  "DROP FUNCTION " + KEYSPACE + ".func_does_not_exist");
+
+        assertInvalidThrowMessage(String.format("Function '%s.func_does_not_exist(int, text)' doesn't exist", KEYSPACE),
+                                  InvalidRequestException.class,
+                                  "DROP FUNCTION " + KEYSPACE + ".func_does_not_exist(int, text)");
+
+        assertInvalidThrowMessage("Function 'keyspace_does_not_exist.func_does_not_exist' doesn't exist",
+                                  InvalidRequestException.class,
+                                  "DROP FUNCTION keyspace_does_not_exist.func_does_not_exist");
+
+        assertInvalidThrowMessage("Function 'keyspace_does_not_exist.func_does_not_exist(int, text)' doesn't exist",
+                                  InvalidRequestException.class,
+                                  "DROP FUNCTION keyspace_does_not_exist.func_does_not_exist(int, text)");
 
         execute("DROP FUNCTION IF EXISTS " + KEYSPACE + ".func_does_not_exist");
         execute("DROP FUNCTION IF EXISTS " + KEYSPACE + ".func_does_not_exist(int,text)");
@@ -100,7 +111,7 @@ public class UFTest extends CQLTester
                      "RETURNS NULL ON NULL INPUT " +
                      "RETURNS int " +
                      "LANGUAGE javascript " +
-                     "AS '\"string\";';");
+                     "AS '\"string1\";';");
 
         assertLastSchemaChange(Event.SchemaChange.Change.UPDATED, Event.SchemaChange.Target.FUNCTION,
                                KEYSPACE, parseFunctionName(f).name,
@@ -369,7 +380,7 @@ public class UFTest extends CQLTester
                              "CREATE OR REPLACE FUNCTION " + fSin + " ( input double ) " +
                              "CALLED ON NULL INPUT " +
                              "RETURNS text " +
-                             "LANGUAGE java AS 'return Double.valueOf(42d);'");
+                             "LANGUAGE java AS 'return \"42d\";'");
 
         // proper replacement
         execute("CREATE OR REPLACE FUNCTION " + fSin + " ( input double ) " +
@@ -402,13 +413,13 @@ public class UFTest extends CQLTester
         execute("DROP FUNCTION " + fSin2);
 
         // Drop unexisting function
-        assertInvalidMessage("Cannot drop non existing function", "DROP FUNCTION " + fSin);
+        assertInvalidMessage(String.format("Function '%s' doesn't exist", fSin), "DROP FUNCTION " + fSin);
         // but don't complain with "IF EXISTS"
         execute("DROP FUNCTION IF EXISTS " + fSin);
 
         // can't drop native functions
-        assertInvalidMessage("system keyspace is not user-modifiable", "DROP FUNCTION totimestamp");
-        assertInvalidMessage("system keyspace is not user-modifiable", "DROP FUNCTION uuid");
+        assertInvalidMessage("System keyspace 'system' is not user-modifiable", "DROP FUNCTION totimestamp");
+        assertInvalidMessage("System keyspace 'system' is not user-modifiable", "DROP FUNCTION uuid");
 
         // sin() no longer exists
         assertInvalidMessage("Unknown function", "SELECT key, sin(d) FROM %s");
@@ -509,8 +520,8 @@ public class UFTest extends CQLTester
         assertEmpty(execute("SELECT v FROM %s WHERE k = " + fOverload + "((varchar)?)", "foo"));
 
         // no such functions exist...
-        assertInvalidMessage("non existing function", "DROP FUNCTION " + fOverload + "(boolean)");
-        assertInvalidMessage("non existing function", "DROP FUNCTION " + fOverload + "(bigint)");
+        assertInvalidMessage(String.format("Function '%s(boolean)' doesn't exist", fOverload), "DROP FUNCTION " + fOverload + "(boolean)");
+        assertInvalidMessage(String.format("Function '%s(bigint)' doesn't exist", fOverload), "DROP FUNCTION " + fOverload + "(bigint)");
 
         // 'overloaded' has multiple overloads - so it has to fail (CASSANDRA-7812)
         assertInvalidMessage("matches multiple function definitions", "DROP FUNCTION " + fOverload);
@@ -641,43 +652,43 @@ public class UFTest extends CQLTester
 
                 "AS 'return null;';");
 
-        assertInvalidMessage("system keyspace is not user-modifiable",
+        assertInvalidMessage("System keyspace 'system' is not user-modifiable",
                              "CREATE OR REPLACE FUNCTION system.jnft(val double) " +
                              "RETURNS NULL ON NULL INPUT " +
                              "RETURNS double " +
                              "LANGUAGE JAVA\n" +
                              "AS 'return null;';");
-        assertInvalidMessage("system keyspace is not user-modifiable",
+        assertInvalidMessage("System keyspace 'system' is not user-modifiable",
                              "CREATE OR REPLACE FUNCTION system.totimestamp(val timeuuid) " +
                              "RETURNS NULL ON NULL INPUT " +
                              "RETURNS timestamp " +
                              "LANGUAGE JAVA\n" +
 
                              "AS 'return null;';");
-        assertInvalidMessage("system keyspace is not user-modifiable",
+        assertInvalidMessage("System keyspace 'system' is not user-modifiable",
                              "DROP FUNCTION system.now");
 
-        // KS for executeInternal() is system
-        assertInvalidMessage("system keyspace is not user-modifiable",
+        // KS for executeLocally() is system
+        assertInvalidMessage("System keyspace 'system' is not user-modifiable",
                              "CREATE OR REPLACE FUNCTION jnft(val double) " +
                              "RETURNS NULL ON NULL INPUT " +
                              "RETURNS double " +
                              "LANGUAGE JAVA\n" +
                              "AS 'return null;';");
-        assertInvalidMessage("system keyspace is not user-modifiable",
+        assertInvalidMessage("System keyspace 'system' is not user-modifiable",
                              "CREATE OR REPLACE FUNCTION totimestamp(val timeuuid) " +
                              "RETURNS NULL ON NULL INPUT " +
                              "RETURNS timestamp " +
                              "LANGUAGE JAVA\n" +
                              "AS 'return null;';");
-        assertInvalidMessage("system keyspace is not user-modifiable",
+        assertInvalidMessage("System keyspace 'system' is not user-modifiable",
                              "DROP FUNCTION now");
     }
 
     @Test
     public void testFunctionNonExistingKeyspace() throws Throwable
     {
-        assertInvalidMessage("Keyspace this_ks_does_not_exist doesn't exist",
+        assertInvalidMessage("Keyspace 'this_ks_does_not_exist' doesn't exist",
                              "CREATE OR REPLACE FUNCTION this_ks_does_not_exist.jnft(val double) " +
                              "RETURNS NULL ON NULL INPUT " +
                              "RETURNS double " +
@@ -690,7 +701,7 @@ public class UFTest extends CQLTester
     {
         dropPerTestKeyspace();
 
-        assertInvalidMessage("Keyspace " + KEYSPACE_PER_TEST + " doesn't exist",
+        assertInvalidMessage("Keyspace '" + KEYSPACE_PER_TEST + "' doesn't exist",
                              "CREATE OR REPLACE FUNCTION " + KEYSPACE_PER_TEST + ".jnft(val double) " +
                              "RETURNS NULL ON NULL INPUT " +
                              "RETURNS double " +
@@ -761,7 +772,7 @@ public class UFTest extends CQLTester
     @Test
     public void testDuplicateArgNames() throws Throwable
     {
-        assertInvalidMessage("duplicate argument names for given function",
+        assertInvalidMessage("Duplicate argument names for given function",
                              "CREATE OR REPLACE FUNCTION " + KEYSPACE + ".scrinv(val double, val text) " +
                              "RETURNS NULL ON NULL INPUT " +
                              "RETURNS text " +

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/test/unit/org/apache/cassandra/cql3/validation/entities/UFTypesTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/entities/UFTypesTest.java b/test/unit/org/apache/cassandra/cql3/validation/entities/UFTypesTest.java
index 3f1bcb1..9680bd5 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/entities/UFTypesTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/UFTypesTest.java
@@ -293,14 +293,14 @@ public class UFTypesTest extends CQLTester
         execute("INSERT INTO %s (a, b) VALUES (?, ?)", 2, set(4, 5, 6));
         execute("INSERT INTO %s (a, b) VALUES (?, ?)", 3, set(7, 8, 9));
 
-        assertInvalidMessage("The function arguments should not be frozen",
+        assertInvalidMessage("Argument 'frozen<set<int>>' cannot be frozen; remove frozen<> modifier from 'frozen<set<int>>'",
                              "CREATE OR REPLACE FUNCTION " + KEYSPACE + ".frozenSetArg(values frozen<set<int>>) " +
                              "CALLED ON NULL INPUT " +
                              "RETURNS int " +
                              "LANGUAGE java\n" +
                              "AS 'int sum = 0; for (Object value : values) {sum += value;} return sum;';");
 
-        assertInvalidMessage("The function return type should not be frozen",
+        assertInvalidMessage("Return type 'frozen<set<int>>' cannot be frozen; remove frozen<> modifier from 'frozen<set<int>>'",
                              "CREATE OR REPLACE FUNCTION " + KEYSPACE + ".frozenReturnType(values set<int>) " +
                              "CALLED ON NULL INPUT " +
                              "RETURNS frozen<set<int>> " +
@@ -331,7 +331,7 @@ public class UFTypesTest extends CQLTester
         assertRows(execute("SELECT a FROM %s WHERE b = " + functionName + "(?)", set(1, 2, 3)),
                    row(1));
 
-        assertInvalidMessage("The function arguments should not be frozen",
+        assertInvalidMessage("Argument 'frozen<set<int>>' cannot be frozen; remove frozen<> modifier from 'frozen<set<int>>'",
                              "DROP FUNCTION " + functionName + "(frozen<set<int>>);");
     }
 
@@ -346,14 +346,14 @@ public class UFTypesTest extends CQLTester
         execute("INSERT INTO %s (a, b) VALUES (?, ?)", 2, list(4, 5, 6));
         execute("INSERT INTO %s (a, b) VALUES (?, ?)", 3, list(7, 8, 9));
 
-        assertInvalidMessage("The function arguments should not be frozen",
+        assertInvalidMessage("Argument 'frozen<list<int>>' cannot be frozen; remove frozen<> modifier from 'frozen<list<int>>'",
                              "CREATE OR REPLACE FUNCTION " + KEYSPACE + ".withFrozenArg(values frozen<list<int>>) " +
                              "CALLED ON NULL INPUT " +
                              "RETURNS int " +
                              "LANGUAGE java\n" +
                              "AS 'int sum = 0; for (Object value : values) {sum += value;} return sum;';");
 
-        assertInvalidMessage("The function return type should not be frozen",
+        assertInvalidMessage("Return type 'frozen<list<int>>' cannot be frozen; remove frozen<> modifier from 'frozen<list<int>>'",
                              "CREATE OR REPLACE FUNCTION " + KEYSPACE + ".frozenReturnType(values list<int>) " +
                              "CALLED ON NULL INPUT " +
                              "RETURNS frozen<list<int>> " +
@@ -384,7 +384,7 @@ public class UFTypesTest extends CQLTester
         assertRows(execute("SELECT a FROM %s WHERE b = " + functionName + "(?)", set(1, 2, 3)),
                    row(1));
 
-        assertInvalidMessage("The function arguments should not be frozen",
+        assertInvalidMessage("frozen<list<int>>' cannot be frozen; remove frozen<> modifier from 'frozen<list<int>>'",
                              "DROP FUNCTION " + functionName + "(frozen<list<int>>);");
     }
 
@@ -399,14 +399,14 @@ public class UFTypesTest extends CQLTester
         execute("INSERT INTO %s (a, b) VALUES (?, ?)", 2, map(4, 4, 5, 5, 6, 6));
         execute("INSERT INTO %s (a, b) VALUES (?, ?)", 3, map(7, 7, 8, 8, 9, 9));
 
-        assertInvalidMessage("The function arguments should not be frozen",
+        assertInvalidMessage("Argument 'frozen<map<int, int>>' cannot be frozen; remove frozen<> modifier from 'frozen<map<int, int>>'",
                              "CREATE OR REPLACE FUNCTION " + KEYSPACE + ".withFrozenArg(values frozen<map<int, int>>) " +
                              "CALLED ON NULL INPUT " +
                              "RETURNS int " +
                              "LANGUAGE java\n" +
                              "AS 'int sum = 0; for (Object value : values.values()) {sum += value;} return sum;';");
 
-        assertInvalidMessage("The function return type should not be frozen",
+        assertInvalidMessage("Return type 'frozen<map<int, int>>' cannot be frozen; remove frozen<> modifier from 'frozen<map<int, int>>'",
                              "CREATE OR REPLACE FUNCTION " + KEYSPACE + ".frozenReturnType(values map<int, int>) " +
                              "CALLED ON NULL INPUT " +
                              "RETURNS frozen<map<int, int>> " +
@@ -437,7 +437,7 @@ public class UFTypesTest extends CQLTester
         assertRows(execute("SELECT a FROM %s WHERE b = " + functionName + "(?)", map(1, 1, 2, 2, 3, 3)),
                    row(1));
 
-        assertInvalidMessage("The function arguments should not be frozen",
+        assertInvalidMessage("frozen<map<int, int>>' cannot be frozen; remove frozen<> modifier from 'frozen<map<int, int>>",
                              "DROP FUNCTION " + functionName + "(frozen<map<int, int>>);");
     }
 
@@ -452,14 +452,14 @@ public class UFTypesTest extends CQLTester
         execute("INSERT INTO %s (a, b) VALUES (?, ?)", 2, tuple(4, 5));
         execute("INSERT INTO %s (a, b) VALUES (?, ?)", 3, tuple(7, 8));
 
-        assertInvalidMessage("The function arguments should not be frozen",
+        assertInvalidMessage("Argument 'tuple<int, int>' cannot be frozen; remove frozen<> modifier from 'tuple<int, int>'",
                              "CREATE OR REPLACE FUNCTION " + KEYSPACE + ".withFrozenArg(values frozen<tuple<int, int>>) " +
                              "CALLED ON NULL INPUT " +
                              "RETURNS text " +
                              "LANGUAGE java\n" +
                              "AS 'return values.toString();';");
 
-        assertInvalidMessage("The function return type should not be frozen",
+        assertInvalidMessage("Return type 'tuple<int, int>' cannot be frozen; remove frozen<> modifier from 'tuple<int, int>'",
                              "CREATE OR REPLACE FUNCTION " + KEYSPACE + ".frozenReturnType(values tuple<int, int>) " +
                              "CALLED ON NULL INPUT " +
                              "RETURNS frozen<tuple<int, int>> " +
@@ -490,7 +490,7 @@ public class UFTypesTest extends CQLTester
         assertRows(execute("SELECT a FROM %s WHERE b = " + functionName + "(?)", tuple(1, 2)),
                    row(1));
 
-        assertInvalidMessage("The function arguments should not be frozen",
+        assertInvalidMessage("Argument 'tuple<int, int>' cannot be frozen; remove frozen<> modifier from 'tuple<int, int>'",
                              "DROP FUNCTION " + functionName + "(frozen<tuple<int, int>>);");
     }
 
@@ -506,14 +506,14 @@ public class UFTypesTest extends CQLTester
         execute("INSERT INTO %s (a, b) VALUES (?, {f : ?})", 2, 4);
         execute("INSERT INTO %s (a, b) VALUES (?, {f : ?})", 3, 7);
 
-        assertInvalidMessage("The function arguments should not be frozen",
+        assertInvalidMessage("cannot be frozen; remove frozen<> modifier",
                              "CREATE OR REPLACE FUNCTION " + KEYSPACE + ".withFrozenArg(values frozen<" + myType + ">) " +
                              "CALLED ON NULL INPUT " +
                              "RETURNS text " +
                              "LANGUAGE java\n" +
                              "AS 'return values.toString();';");
 
-        assertInvalidMessage("The function return type should not be frozen",
+        assertInvalidMessage("cannot be frozen; remove frozen<> modifier",
                              "CREATE OR REPLACE FUNCTION " + KEYSPACE + ".frozenReturnType(values " + myType + ") " +
                              "CALLED ON NULL INPUT " +
                              "RETURNS frozen<" + myType + "> " +
@@ -544,7 +544,7 @@ public class UFTypesTest extends CQLTester
         assertRows(execute("SELECT a FROM %s WHERE b = " + functionName + "({f: ?})", 1),
                    row(1));
 
-        assertInvalidMessage("The function arguments should not be frozen",
+        assertInvalidMessage(String.format("frozen<%s>' cannot be frozen; remove frozen<> modifier from 'frozen<%s>'", myType, myType),
                              "DROP FUNCTION " + functionName + "(frozen<" + myType + ">);");
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/test/unit/org/apache/cassandra/cql3/validation/entities/UserTypesTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/entities/UserTypesTest.java b/test/unit/org/apache/cassandra/cql3/validation/entities/UserTypesTest.java
index 9ea5572..e39dd35 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/entities/UserTypesTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/UserTypesTest.java
@@ -109,9 +109,9 @@ public class UserTypesTest extends CQLTester
         String myType = KEYSPACE + '.' + typename;
 
         // non-frozen UDTs in a table PK
-        assertInvalidMessage("Invalid non-frozen user-defined type for PRIMARY KEY component k",
+        assertInvalidMessage("Invalid non-frozen user-defined type \"" + myType + "\" for PRIMARY KEY column 'k'",
                 "CREATE TABLE " + KEYSPACE + ".wrong (k " + myType + " PRIMARY KEY , v int)");
-        assertInvalidMessage("Invalid non-frozen user-defined type for PRIMARY KEY component k2",
+        assertInvalidMessage("Invalid non-frozen user-defined type \"" + myType + "\" for PRIMARY KEY column 'k2'",
                 "CREATE TABLE " + KEYSPACE + ".wrong (k1 int, k2 " + myType + ", v int, PRIMARY KEY (k1, k2))");
 
         // non-frozen UDTs in a collection
@@ -620,7 +620,7 @@ public class UserTypesTest extends CQLTester
     private void assertInvalidAlterDropStatements(String t) throws Throwable
     {
         assertInvalidMessage("Cannot alter user type " + typeWithKs(t), "ALTER TYPE " + typeWithKs(t) + " RENAME foo TO bar;");
-        assertInvalidMessage("Cannot drop user type " + typeWithKs(t), "DROP TYPE " + typeWithKs(t) + ';');
+        assertInvalidMessage("Cannot drop user type '" + typeWithKs(t), "DROP TYPE " + typeWithKs(t) + ';');
     }
 
     @Test

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/test/unit/org/apache/cassandra/cql3/validation/entities/VirtualTableTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/entities/VirtualTableTest.java b/test/unit/org/apache/cassandra/cql3/validation/entities/VirtualTableTest.java
index 3581a73..a503a60 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/entities/VirtualTableTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/VirtualTableTest.java
@@ -279,34 +279,34 @@ public class VirtualTableTest extends CQLTester
     @Test
     public void testInvalidDDLOperations() throws Throwable
     {
-        assertInvalidMessage("Cannot drop virtual keyspaces",
+        assertInvalidMessage("Virtual keyspace 'test_virtual_ks' is not user-modifiable",
                              "DROP KEYSPACE test_virtual_ks");
 
-        assertInvalidMessage("Cannot alter virtual keyspaces",
+        assertInvalidMessage("Virtual keyspace 'test_virtual_ks' is not user-modifiable",
                              "ALTER KEYSPACE test_virtual_ks WITH durable_writes = false");
 
-        assertInvalidMessage("Cannot create tables in virtual keyspaces",
+        assertInvalidMessage("Virtual keyspace 'test_virtual_ks' is not user-modifiable",
                              "CREATE TABLE test_virtual_ks.test (id int PRIMARY KEY)");
 
-        assertInvalidMessage("Cannot create types in virtual keyspaces",
+        assertInvalidMessage("Virtual keyspace 'test_virtual_ks' is not user-modifiable",
                              "CREATE TYPE test_virtual_ks.type (id int)");
 
-        assertInvalidMessage("Cannot drop virtual tables",
+        assertInvalidMessage("Virtual keyspace 'test_virtual_ks' is not user-modifiable",
                              "DROP TABLE test_virtual_ks.vt1");
 
-        assertInvalidMessage("Cannot alter virtual tables",
+        assertInvalidMessage("Virtual keyspace 'test_virtual_ks' is not user-modifiable",
                              "ALTER TABLE test_virtual_ks.vt1 DROP v1");
 
-        assertInvalidMessage("Cannot truncate virtual tables",
+        assertInvalidMessage("Error during truncate: Cannot truncate virtual tables",
                              "TRUNCATE TABLE test_virtual_ks.vt1");
 
-        assertInvalidMessage("Secondary indexes are not supported on virtual tables",
+        assertInvalidMessage("Virtual keyspace 'test_virtual_ks' is not user-modifiable",
                              "CREATE INDEX ON test_virtual_ks.vt1 (v1)");
 
-        assertInvalidMessage("Materialized views are not supported on virtual tables",
+        assertInvalidMessage("Virtual keyspace 'test_virtual_ks' is not user-modifiable",
                              "CREATE MATERIALIZED VIEW test_virtual_ks.mvt1 AS SELECT c, v1 FROM test_virtual_ks.vt1 WHERE c IS NOT NULL PRIMARY KEY(c)");
 
-        assertInvalidMessage("Cannot CREATE TRIGGER against a virtual table",
+        assertInvalidMessage("Virtual keyspace 'test_virtual_ks' is not user-modifiable",
                              "CREATE TRIGGER test_trigger ON test_virtual_ks.vt1 USING '" + TestTrigger.class.getName() + '\'');
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/test/unit/org/apache/cassandra/cql3/validation/operations/AggregationTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/AggregationTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/AggregationTest.java
index f2f6614..703d0ad 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/AggregationTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/AggregationTest.java
@@ -42,18 +42,15 @@ import ch.qos.logback.classic.joran.ReconfigureOnChangeTask;
 import ch.qos.logback.classic.spi.TurboFilterList;
 import ch.qos.logback.classic.turbo.ReconfigureOnChangeFilter;
 import ch.qos.logback.classic.turbo.TurboFilter;
-import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.schema.SchemaConstants;
 import org.apache.cassandra.cql3.CQLTester;
 import org.apache.cassandra.cql3.QueryProcessor;
 import org.apache.cassandra.cql3.UntypedResultSet;
 import org.apache.cassandra.cql3.UntypedResultSet.Row;
-import org.apache.cassandra.cql3.functions.UDAggregate;
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.db.marshal.TypeParser;
 import org.apache.cassandra.exceptions.FunctionExecutionException;
 import org.apache.cassandra.exceptions.InvalidRequestException;
-import org.apache.cassandra.schema.KeyspaceMetadata;
 import org.apache.cassandra.service.ClientState;
 import org.apache.cassandra.transport.Event;
 import org.apache.cassandra.transport.ProtocolVersion;
@@ -70,10 +67,21 @@ public class AggregationTest extends CQLTester
     @Test
     public void testNonExistingOnes() throws Throwable
     {
-        assertInvalidThrowMessage("Cannot drop non existing aggregate", InvalidRequestException.class, "DROP AGGREGATE " + KEYSPACE + ".aggr_does_not_exist");
-        assertInvalidThrowMessage("Cannot drop non existing aggregate", InvalidRequestException.class, "DROP AGGREGATE " + KEYSPACE + ".aggr_does_not_exist(int,text)");
-        assertInvalidThrowMessage("Cannot drop non existing aggregate", InvalidRequestException.class, "DROP AGGREGATE keyspace_does_not_exist.aggr_does_not_exist");
-        assertInvalidThrowMessage("Cannot drop non existing aggregate", InvalidRequestException.class, "DROP AGGREGATE keyspace_does_not_exist.aggr_does_not_exist(int,text)");
+        assertInvalidThrowMessage(String.format("Aggregate '%s.aggr_does_not_exist' doesn't exist", KEYSPACE),
+                                  InvalidRequestException.class,
+                                  "DROP AGGREGATE " + KEYSPACE + ".aggr_does_not_exist");
+
+        assertInvalidThrowMessage(String.format("Aggregate '%s.aggr_does_not_exist(int, text)' doesn't exist", KEYSPACE),
+                                  InvalidRequestException.class,
+                                  "DROP AGGREGATE " + KEYSPACE + ".aggr_does_not_exist(int,text)");
+
+        assertInvalidThrowMessage("Aggregate 'keyspace_does_not_exist.aggr_does_not_exist' doesn't exist",
+                                  InvalidRequestException.class,
+                                  "DROP AGGREGATE keyspace_does_not_exist.aggr_does_not_exist");
+
+        assertInvalidThrowMessage("Aggregate 'keyspace_does_not_exist.aggr_does_not_exist(int, text)' doesn't exist",
+                                  InvalidRequestException.class,
+                                  "DROP AGGREGATE keyspace_does_not_exist.aggr_does_not_exist(int,text)");
 
         execute("DROP AGGREGATE IF EXISTS " + KEYSPACE + ".aggr_does_not_exist");
         execute("DROP AGGREGATE IF EXISTS " + KEYSPACE + ".aggr_does_not_exist(int,text)");
@@ -419,7 +427,7 @@ public class AggregationTest extends CQLTester
         schemaChange("CREATE OR REPLACE AGGREGATE " + a + "(double) " +
                      "SFUNC " + shortFunctionName(f) + " " +
                      "STYPE double " +
-                     "INITCOND 0");
+                     "INITCOND 1");
 
         assertLastSchemaChange(Event.SchemaChange.Change.UPDATED, Event.SchemaChange.Target.AGGREGATE,
                                KEYSPACE, parseFunctionName(a).name,
@@ -464,7 +472,7 @@ public class AggregationTest extends CQLTester
 
         // DROP AGGREGATE must not succeed against a scalar
         assertInvalidMessage("matches multiple function definitions", "DROP AGGREGATE " + f);
-        assertInvalidMessage("non existing", "DROP AGGREGATE " + f + "(double, double)");
+        assertInvalidMessage("doesn't exist", "DROP AGGREGATE " + f + "(double, double)");
 
         String a = createAggregate(KEYSPACE,
                                    "double",
@@ -481,7 +489,7 @@ public class AggregationTest extends CQLTester
 
         // DROP FUNCTION must not succeed against an aggregate
         assertInvalidMessage("matches multiple function definitions", "DROP FUNCTION " + a);
-        assertInvalidMessage("non existing function", "DROP FUNCTION " + a + "(double)");
+        assertInvalidMessage("doesn't exist", "DROP FUNCTION " + a + "(double)");
 
         // ambigious
         assertInvalidMessage("matches multiple function definitions", "DROP AGGREGATE " + a);
@@ -660,37 +668,37 @@ public class AggregationTest extends CQLTester
                                         "LANGUAGE java " +
                                         "AS 'return a.toString();'");
 
-        assertInvalidMessage("does not exist or is not a scalar function",
+        assertInvalidMessage("doesn't exist",
                              "CREATE AGGREGATE " + KEYSPACE + ".aggrInvalid(double)" +
                              "SFUNC " + shortFunctionName(fState) + " " +
                              "STYPE double " +
                              "FINALFUNC " + shortFunctionName(fFinal));
-        assertInvalidMessage("does not exist or is not a scalar function",
+        assertInvalidMessage("doesn't exist",
                              "CREATE AGGREGATE " + KEYSPACE + ".aggrInvalid(int)" +
                              "SFUNC " + shortFunctionName(fState) + " " +
                              "STYPE double " +
                              "FINALFUNC " + shortFunctionName(fFinal));
-        assertInvalidMessage("does not exist or is not a scalar function",
+        assertInvalidMessage("doesn't exist",
                              "CREATE AGGREGATE " + KEYSPACE + ".aggrInvalid(double)" +
                              "SFUNC " + shortFunctionName(fState) + " " +
                              "STYPE int " +
                              "FINALFUNC " + shortFunctionName(fFinal));
-        assertInvalidMessage("does not exist or is not a scalar function",
+        assertInvalidMessage("doesn't exist",
                              "CREATE AGGREGATE " + KEYSPACE + ".aggrInvalid(double)" +
                              "SFUNC " + shortFunctionName(fState) + " " +
                              "STYPE int");
-        assertInvalidMessage("does not exist or is not a scalar function",
+        assertInvalidMessage("doesn't exist",
                              "CREATE AGGREGATE " + KEYSPACE + ".aggrInvalid(int)" +
                              "SFUNC " + shortFunctionName(fState) + " " +
                              "STYPE double");
 
-        assertInvalidMessage("does not exist or is not a scalar function",
+        assertInvalidMessage("doesn't exist",
                              "CREATE AGGREGATE " + KEYSPACE + ".aggrInvalid(double)" +
                              "SFUNC " + shortFunctionName(fState2) + " " +
                              "STYPE double " +
                              "FINALFUNC " + shortFunctionName(fFinal));
 
-        assertInvalidMessage("does not exist or is not a scalar function",
+        assertInvalidMessage("doesn't exist",
                              "CREATE AGGREGATE " + KEYSPACE + ".aggrInvalid(double)" +
                              "SFUNC " + shortFunctionName(fState) + " " +
                              "STYPE double " +
@@ -716,13 +724,13 @@ public class AggregationTest extends CQLTester
                                        "LANGUAGE java " +
                                        "AS 'return a.toString();'");
 
-        assertInvalidMessage("does not exist or is not a scalar function",
+        assertInvalidMessage("doesn't exist",
                              "CREATE AGGREGATE " + KEYSPACE + ".aggrInvalid(int)" +
                              "SFUNC " + shortFunctionName(fState) + "_not_there " +
                              "STYPE int " +
                              "FINALFUNC " + shortFunctionName(fFinal));
 
-        assertInvalidMessage("does not exist or is not a scalar function",
+        assertInvalidMessage("doesn't exist",
                              "CREATE AGGREGATE " + KEYSPACE + ".aggrInvalid(int)" +
                              "SFUNC " + shortFunctionName(fState) + " " +
                              "STYPE int " +
@@ -805,7 +813,7 @@ public class AggregationTest extends CQLTester
     @Test
     public void testJavaAggregateWithoutStateOrFinal() throws Throwable
     {
-        assertInvalidMessage("does not exist or is not a scalar function",
+        assertInvalidMessage("doesn't exist",
                              "CREATE AGGREGATE " + KEYSPACE + ".jSumFooNE1(int) " +
                              "SFUNC jSumFooNEstate " +
                              "STYPE int");
@@ -818,7 +826,7 @@ public class AggregationTest extends CQLTester
                                   "LANGUAGE java " +
                                   "AS 'return Integer.valueOf(a + b);'");
 
-        assertInvalidMessage("does not exist or is not a scalar function",
+        assertInvalidMessage("doesn't exist",
                              "CREATE AGGREGATE " + KEYSPACE + ".jSumFooNE2(int) " +
                              "SFUNC " + shortFunctionName(f) + " " +
                              "STYPE int " +
@@ -1144,12 +1152,12 @@ public class AggregationTest extends CQLTester
                                    "SFUNC " + shortFunctionName(fState) + " " +
                                    "STYPE int ");
 
-        assertInvalidMessage("does not exist or is not a scalar function",
+        assertInvalidMessage("doesn't exist",
                              "CREATE AGGREGATE " + KEYSPACE + ".aggInv(int) " +
                              "SFUNC " + shortFunctionName(a) + " " +
                              "STYPE int ");
 
-        assertInvalidMessage("does not exist or is not a scalar function",
+        assertInvalidMessage("isn't a scalar function",
                              "CREATE AGGREGATE " + KEYSPACE + ".aggInv(int) " +
                              "SFUNC " + shortFunctionName(fState) + " " +
                              "STYPE int " +
@@ -1291,41 +1299,6 @@ public class AggregationTest extends CQLTester
     }
 
     @Test
-    public void testBrokenAggregate() throws Throwable
-    {
-        createTable("CREATE TABLE %s (key int primary key, val int)");
-        execute("INSERT INTO %s (key, val) VALUES (?, ?)", 1, 1);
-
-        String fState = createFunction(KEYSPACE,
-                                       "int, int",
-                                       "CREATE FUNCTION %s(a int, b int) " +
-                                       "CALLED ON NULL INPUT " +
-                                       "RETURNS int " +
-                                       "LANGUAGE javascript " +
-                                       "AS 'a + b;'");
-
-        String a = createAggregate(KEYSPACE,
-                                   "int",
-                                   "CREATE AGGREGATE %s(int) " +
-                                   "SFUNC " + shortFunctionName(fState) + " " +
-                                   "STYPE int ");
-
-        KeyspaceMetadata ksm = Schema.instance.getKeyspaceMetadata(keyspace());
-        UDAggregate f = (UDAggregate) ksm.functions.get(parseFunctionName(a)).iterator().next();
-
-        UDAggregate broken = UDAggregate.createBroken(f.name(),
-                                                      f.argTypes(),
-                                                      f.returnType(),
-                                                      null,
-                                                      new InvalidRequestException("foo bar is broken"));
-
-        Schema.instance.load(ksm.withSwapped(ksm.functions.without(f.name(), f.argTypes()).with(broken)));
-
-        assertInvalidThrowMessage("foo bar is broken", InvalidRequestException.class,
-                                  "SELECT " + a + "(val) FROM %s");
-    }
-
-    @Test
     public void testWrongStateType() throws Throwable
     {
         createTable("CREATE TABLE %s (key int primary key, val int)");
@@ -1450,7 +1423,7 @@ public class AggregationTest extends CQLTester
                                        "LANGUAGE java " +
                                        "AS 'return state;'");
 
-        assertInvalidMessage("The function state type should not be frozen",
+        assertInvalidMessage("cannot be frozen",
                              "CREATE AGGREGATE %s(set<int>) " +
                              "SFUNC " + parseFunctionName(fState).name + ' ' +
                              "STYPE frozen<set<int>> " +
@@ -1471,7 +1444,7 @@ public class AggregationTest extends CQLTester
         assertRows(execute("SELECT " + aggregation + "(b) FROM %s"),
                    row(set(7, 8, 9)));
 
-        assertInvalidMessage("The function arguments should not be frozen",
+        assertInvalidMessage("Argument 'frozen<set<int>>' cannot be frozen; remove frozen<> modifier from 'frozen<set<int>>'",
                              "DROP AGGREGATE %s (frozen<set<int>>);");
     }
 
@@ -1502,7 +1475,7 @@ public class AggregationTest extends CQLTester
                                        "LANGUAGE java " +
                                        "AS 'return state;'");
 
-        assertInvalidMessage("The function state type should not be frozen",
+        assertInvalidMessage("cannot be frozen",
                              "CREATE AGGREGATE %s(list<int>) " +
                              "SFUNC " + parseFunctionName(fState).name + ' ' +
                              "STYPE frozen<list<int>> " +
@@ -1520,7 +1493,7 @@ public class AggregationTest extends CQLTester
         assertRows(execute("SELECT " + aggregation + "(b) FROM %s"),
                    row(list(7, 8, 9)));
 
-        assertInvalidMessage("The function arguments should not be frozen",
+        assertInvalidMessage("Argument 'frozen<list<int>>' cannot be frozen; remove frozen<> modifier from 'frozen<list<int>>'",
                              "DROP AGGREGATE %s (frozen<list<int>>);");
     }
 
@@ -1551,7 +1524,7 @@ public class AggregationTest extends CQLTester
                                        "LANGUAGE java " +
                                        "AS 'return state;'");
 
-        assertInvalidMessage("The function state type should not be frozen",
+        assertInvalidMessage("cannot be frozen",
                              "CREATE AGGREGATE %s(map<int, int>) " +
                              "SFUNC " + parseFunctionName(fState).name + ' ' +
                              "STYPE frozen<map<int, int>> " +
@@ -1569,7 +1542,7 @@ public class AggregationTest extends CQLTester
         assertRows(execute("SELECT " + aggregation + "(b) FROM %s"),
                    row(map(7, 8, 9, 10)));
 
-        assertInvalidMessage("The function arguments should not be frozen",
+        assertInvalidMessage("Argument 'frozen<map<int, int>>' cannot be frozen; remove frozen<> modifier from 'frozen<map<int, int>>'",
                              "DROP AGGREGATE %s (frozen<map<int, int>>);");
     }
 
@@ -1600,7 +1573,7 @@ public class AggregationTest extends CQLTester
                                        "LANGUAGE java " +
                                        "AS 'return state;'");
 
-        assertInvalidMessage("The function state type should not be frozen",
+        assertInvalidMessage("cannot be frozen",
                              "CREATE AGGREGATE %s(tuple<int, int>) " +
                              "SFUNC " + parseFunctionName(fState).name + ' ' +
                              "STYPE frozen<tuple<int, int>> " +
@@ -1618,7 +1591,7 @@ public class AggregationTest extends CQLTester
         assertRows(execute("SELECT " + aggregation + "(b) FROM %s"),
                    row(tuple(7, 8)));
 
-        assertInvalidMessage("The function arguments should not be frozen",
+        assertInvalidMessage("Argument 'tuple<int, int>' cannot be frozen; remove frozen<> modifier from 'tuple<int, int>'",
                              "DROP AGGREGATE %s (frozen<tuple<int, int>>);");
     }
 
@@ -1650,7 +1623,7 @@ public class AggregationTest extends CQLTester
                                        "LANGUAGE java " +
                                        "AS 'return state;'");
 
-        assertInvalidMessage("The function state type should not be frozen",
+        assertInvalidMessage("cannot be frozen",
                              "CREATE AGGREGATE %s(" + myType + ") " +
                              "SFUNC " + parseFunctionName(fState).name + ' ' +
                              "STYPE frozen<" + myType + "> " +
@@ -1668,7 +1641,7 @@ public class AggregationTest extends CQLTester
         assertRows(execute("SELECT " + aggregation + "(b).f FROM %s"),
                    row(7));
 
-        assertInvalidMessage("The function arguments should not be frozen",
+        assertInvalidMessage(String.format("Argument 'frozen<%s>' cannot be frozen; remove frozen<> modifier from 'frozen<%s>'", myType, myType),
                              "DROP AGGREGATE %s (frozen<" + myType + ">);");
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/test/unit/org/apache/cassandra/cql3/validation/operations/AlterTest.java
----------------------------------------------------------------------
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 c6f255a..a792bcb 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/AlterTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/AlterTest.java
@@ -322,7 +322,7 @@ public class AlterTest extends CQLTester
         execute("insert into %s (k, v) VALUES (0, {'f'})");
         flush();
         execute("alter table %s drop v");
-        execute("alter table %s add v int");
+        execute("alter table %s add v1 int");
     }
 
     @Test
@@ -358,7 +358,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 = { 'sstable_compression' : 'LZ4Compressor', 'chunk_length_kb' : 64 };");
+        execute("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,
@@ -367,7 +367,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 = { 'sstable_compression' : 'LZ4Compressor', 'min_compress_ratio' : 2 };");
+        execute("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,
@@ -376,7 +376,7 @@ public class AlterTest extends CQLTester
                            currentTable()),
                    row(map("chunk_length_in_kb", "64", "class", "org.apache.cassandra.io.compress.LZ4Compressor", "min_compress_ratio", "2.0")));
 
-        execute("ALTER TABLE %s WITH compression = { 'sstable_compression' : 'LZ4Compressor', 'min_compress_ratio' : 1 };");
+        execute("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,
@@ -385,7 +385,7 @@ public class AlterTest extends CQLTester
                            currentTable()),
                    row(map("chunk_length_in_kb", "64", "class", "org.apache.cassandra.io.compress.LZ4Compressor", "min_compress_ratio", "1.0")));
 
-        execute("ALTER TABLE %s WITH compression = { 'sstable_compression' : 'LZ4Compressor', 'min_compress_ratio' : 0 };");
+        execute("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,
@@ -394,15 +394,6 @@ 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 = { 'sstable_compression' : '', 'chunk_length_kb' : 32 };");
-
-        assertRows(execute(format("SELECT compression FROM %s.%s WHERE keyspace_name = ? and table_name = ?;",
-                                  SchemaConstants.SCHEMA_KEYSPACE_NAME,
-                                  SchemaKeyspace.TABLES),
-                           KEYSPACE,
-                           currentTable()),
-                   row(map("enabled", "false")));
-
         execute("ALTER TABLE %s WITH compression = { 'class' : 'SnappyCompressor', 'chunk_length_in_kb' : 32 };");
         execute("ALTER TABLE %s WITH compression = { 'enabled' : 'false'};");
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/test/unit/org/apache/cassandra/cql3/validation/operations/CreateTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/CreateTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/CreateTest.java
index 96f88c3..184c5ad 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/CreateTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/CreateTest.java
@@ -25,22 +25,21 @@ import java.util.UUID;
 
 import org.junit.Test;
 
-import org.apache.cassandra.locator.InetAddressAndPort;
-import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.schema.Schema;
-import org.apache.cassandra.schema.SchemaConstants;
 import org.apache.cassandra.cql3.CQLTester;
 import org.apache.cassandra.cql3.Duration;
 import org.apache.cassandra.db.Mutation;
 import org.apache.cassandra.db.partitions.Partition;
 import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.exceptions.InvalidRequestException;
 import org.apache.cassandra.exceptions.SyntaxException;
+import org.apache.cassandra.io.util.DataOutputBuffer;
 import org.apache.cassandra.locator.AbstractEndpointSnitch;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.locator.IEndpointSnitch;
-import org.apache.cassandra.io.util.DataOutputBuffer;
 import org.apache.cassandra.schema.SchemaKeyspace;
 import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.schema.*;
 import org.apache.cassandra.triggers.ITrigger;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
@@ -91,14 +90,14 @@ public class CreateTest extends CQLTester
     @Test
     public void testCreateTableWithDurationColumns() throws Throwable
     {
-        assertInvalidMessage("duration type is not supported for PRIMARY KEY part a",
-                             "CREATE TABLE test (a duration PRIMARY KEY, b int);");
+        assertInvalidMessage("duration type is not supported for PRIMARY KEY column 'a'",
+                             "CREATE TABLE cql_test_keyspace.table0 (a duration PRIMARY KEY, b int);");
 
-        assertInvalidMessage("duration type is not supported for PRIMARY KEY part b",
-                             "CREATE TABLE test (a text, b duration, c duration, primary key (a, b));");
+        assertInvalidMessage("duration type is not supported for PRIMARY KEY column 'b'",
+                             "CREATE TABLE cql_test_keyspace.table0 (a text, b duration, c duration, primary key (a, b));");
 
-        assertInvalidMessage("duration type is not supported for PRIMARY KEY part b",
-                             "CREATE TABLE test (a text, b duration, c duration, primary key (a, b)) with clustering order by (b DESC);");
+        assertInvalidMessage("duration type is not supported for PRIMARY KEY column 'b'",
+                             "CREATE TABLE cql_test_keyspace.table0 (a text, b duration, c duration, primary key (a, b)) with clustering order by (b DESC);");
 
         createTable("CREATE TABLE %s (a int, b int, c duration, primary key (a, b));");
         execute("INSERT INTO %s (a, b, c) VALUES (1, 1, 1y2mo)");
@@ -178,25 +177,25 @@ public class CreateTest extends CQLTester
 
         // Test duration within Map
         assertInvalidMessage("Durations are not allowed as map keys: map<duration, text>",
-                             "CREATE TABLE test(pk int PRIMARY KEY, m map<duration, text>)");
+                             "CREATE TABLE cql_test_keyspace.table0(pk int PRIMARY KEY, m map<duration, text>)");
 
         createTable("CREATE TABLE %s(pk int PRIMARY KEY, m map<text, duration>)");
         execute("INSERT INTO %s (pk, m) VALUES (1, {'one month' : 1mo, '60 days' : 60d})");
         assertRows(execute("SELECT * FROM %s"),
                    row(1, map("one month", Duration.from("1mo"), "60 days", Duration.from("60d"))));
 
-        assertInvalidMessage("duration type is not supported for PRIMARY KEY part m",
-                "CREATE TABLE %s(m frozen<map<text, duration>> PRIMARY KEY, v int)");
+        assertInvalidMessage("duration type is not supported for PRIMARY KEY column 'm'",
+                "CREATE TABLE cql_test_keyspace.table0(m frozen<map<text, duration>> PRIMARY KEY, v int)");
 
-        assertInvalidMessage("duration type is not supported for PRIMARY KEY part m",
-                             "CREATE TABLE %s(pk int, m frozen<map<text, duration>>, v int, PRIMARY KEY (pk, m))");
+        assertInvalidMessage("duration type is not supported for PRIMARY KEY column 'm'",
+                             "CREATE TABLE cql_test_keyspace.table0(pk int, m frozen<map<text, duration>>, v int, PRIMARY KEY (pk, m))");
 
         // Test duration within Set
         assertInvalidMessage("Durations are not allowed inside sets: set<duration>",
-                             "CREATE TABLE %s(pk int PRIMARY KEY, s set<duration>)");
+                             "CREATE TABLE cql_test_keyspace.table0(pk int PRIMARY KEY, s set<duration>)");
 
         assertInvalidMessage("Durations are not allowed inside sets: frozen<set<duration>>",
-                             "CREATE TABLE %s(s frozen<set<duration>> PRIMARY KEY, v int)");
+                             "CREATE TABLE cql_test_keyspace.table0(s frozen<set<duration>> PRIMARY KEY, v int)");
 
         // Test duration within List
         createTable("CREATE TABLE %s(pk int PRIMARY KEY, l list<duration>)");
@@ -204,8 +203,8 @@ public class CreateTest extends CQLTester
         assertRows(execute("SELECT * FROM %s"),
                    row(1, list(Duration.from("1mo"), Duration.from("60d"))));
 
-        assertInvalidMessage("duration type is not supported for PRIMARY KEY part l",
-                             "CREATE TABLE %s(l frozen<list<duration>> PRIMARY KEY, v int)");
+        assertInvalidMessage("duration type is not supported for PRIMARY KEY column 'l'",
+                             "CREATE TABLE cql_test_keyspace.table0(l frozen<list<duration>> PRIMARY KEY, v int)");
 
         // Test duration within Tuple
         createTable("CREATE TABLE %s(pk int PRIMARY KEY, t tuple<int, duration>)");
@@ -213,8 +212,8 @@ public class CreateTest extends CQLTester
         assertRows(execute("SELECT * FROM %s"),
                    row(1, tuple(1, Duration.from("1mo"))));
 
-        assertInvalidMessage("duration type is not supported for PRIMARY KEY part t",
-                             "CREATE TABLE %s(t frozen<tuple<int, duration>> PRIMARY KEY, v int)");
+        assertInvalidMessage("duration type is not supported for PRIMARY KEY column 't'",
+                             "CREATE TABLE cql_test_keyspace.table0(t frozen<tuple<int, duration>> PRIMARY KEY, v int)");
 
         // Test duration within UDT
         String typename = createType("CREATE TYPE %s (a duration)");
@@ -224,12 +223,12 @@ public class CreateTest extends CQLTester
         assertRows(execute("SELECT * FROM %s"),
                    row(1, userType("a", Duration.from("1mo"))));
 
-        assertInvalidMessage("duration type is not supported for PRIMARY KEY part u",
-                             "CREATE TABLE %s(pk int, u frozen<" + myType + ">, v int, PRIMARY KEY(pk, u))");
+        assertInvalidMessage("duration type is not supported for PRIMARY KEY column 'u'",
+                             "CREATE TABLE cql_test_keyspace.table0(pk int, u frozen<" + myType + ">, v int, PRIMARY KEY(pk, u))");
 
         // Test duration with several level of depth
-        assertInvalidMessage("duration type is not supported for PRIMARY KEY part m",
-                "CREATE TABLE %s(pk int, m frozen<map<text, list<tuple<int, duration>>>>, v int, PRIMARY KEY (pk, m))");
+        assertInvalidMessage("duration type is not supported for PRIMARY KEY column 'm'",
+                "CREATE TABLE cql_test_keyspace.table0(pk int, m frozen<map<text, list<tuple<int, duration>>>>, v int, PRIMARY KEY (pk, m))");
     }
 
     private ByteBuffer duration(long months, long days, long nanoseconds) throws IOException
@@ -336,7 +335,7 @@ public class CreateTest extends CQLTester
     @Test
     public void testObsoleteTableProperties() throws Throwable
     {
-        assertInvalidThrow(SyntaxException.class, "CREATE TABLE test (foo text PRIMARY KEY, c int) WITH default_validation=timestamp");
+        assertInvalidThrow(SyntaxException.class, "CREATE TABLE cql_test_keyspace.table0 (foo text PRIMARY KEY, c int) WITH default_validation=timestamp");
 
         createTable("CREATE TABLE %s (foo text PRIMARY KEY, c int)");
         assertInvalidThrow(SyntaxException.class, "ALTER TABLE %s WITH default_validation=int");
@@ -357,7 +356,7 @@ public class CreateTest extends CQLTester
                      "CREATE KEYSPACE My_much_much_too_long_identifier_that_should_not_work WITH replication = { 'class' : 'SimpleStrategy', 'replication_factor' : 1 }");
 
         execute("DROP KEYSPACE testXYZ");
-        assertInvalidThrow(ConfigurationException.class, "DROP KEYSPACE non_existing");
+        assertInvalidThrow(InvalidRequestException.class, "DROP KEYSPACE non_existing");
 
         execute("CREATE KEYSPACE testXYZ WITH replication = { 'class' : 'SimpleStrategy', 'replication_factor' : 1 }");
 
@@ -419,7 +418,15 @@ public class CreateTest extends CQLTester
         String table4 = createTableName();
 
         // repeated column
-        assertInvalidMessage("Multiple definition of identifier k", String.format("CREATE TABLE %s (k int PRIMARY KEY, c int, k text)", table4));
+        assertInvalidMessage("Duplicate column 'k' declaration for table", String.format("CREATE TABLE %s (k int PRIMARY KEY, c int, k text)", table4));
+
+        // compact storage limitations
+        assertInvalidThrow(SyntaxException.class,
+                           String.format("CREATE TABLE %s (k int, name, int, c1 int, c2 int, PRIMARY KEY(k, name)) WITH COMPACT STORAGE", table4));
+
+        execute(String.format("DROP TABLE %s.%s", keyspace(), table1));
+
+        createTable(String.format("CREATE TABLE %s.%s ( k int PRIMARY KEY, c1 int, c2 int, ) ", keyspace(), table1));
     }
 
     /**
@@ -682,7 +689,7 @@ public class CreateTest extends CQLTester
     @Test
     public void compactTableTest() throws Throwable
     {
-        assertInvalidMessage("Compact tables are not allowed in Cassandra starting with 4.0 version.",
+        assertInvalidMessage("COMPACT STORAGE tables are not allowed starting with version 4.0",
                              "CREATE TABLE compact_table_create (id text PRIMARY KEY, content text) WITH COMPACT STORAGE;");
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/test/unit/org/apache/cassandra/cql3/validation/operations/DropTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/DropTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/DropTest.java
index 692eb45..90130ad 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/DropTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/DropTest.java
@@ -27,8 +27,8 @@ public class DropTest extends CQLTester
     @Test
     public void testNonExistingOnes() throws Throwable
     {
-        assertInvalidMessage("Cannot drop non existing table", "DROP TABLE " + KEYSPACE + ".table_does_not_exist");
-        assertInvalidMessage("Cannot drop table in unknown keyspace", "DROP TABLE keyspace_does_not_exist.table_does_not_exist");
+        assertInvalidMessage(String.format("Table '%s.table_does_not_exist' doesn't exist", KEYSPACE),  "DROP TABLE " + KEYSPACE + ".table_does_not_exist");
+        assertInvalidMessage("Table 'keyspace_does_not_exist.table_does_not_exist' doesn't exist", "DROP TABLE keyspace_does_not_exist.table_does_not_exist");
 
         execute("DROP TABLE IF EXISTS " + KEYSPACE + ".table_does_not_exist");
         execute("DROP TABLE IF EXISTS keyspace_does_not_exist.table_does_not_exist");

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/test/unit/org/apache/cassandra/cql3/validation/operations/InsertUpdateIfConditionTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/InsertUpdateIfConditionTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/InsertUpdateIfConditionTest.java
index b40a13e..edfe57a 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/InsertUpdateIfConditionTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/InsertUpdateIfConditionTest.java
@@ -1414,10 +1414,8 @@ public class InsertUpdateIfConditionTest extends CQLTester
     {
         String tableName = createTable("CREATE TABLE %s (id text PRIMARY KEY, value1 blob, value2 blob)with comment = 'foo'");
 
-        execute("use " + KEYSPACE);
-
         // try dropping when doesn't exist
-        schemaChange("DROP INDEX IF EXISTS myindex");
+        schemaChange(format("DROP INDEX IF EXISTS %s.myindex", KEYSPACE));
 
         // create and confirm
         createIndex("CREATE INDEX IF NOT EXISTS myindex ON %s (value1)");
@@ -1428,7 +1426,7 @@ public class InsertUpdateIfConditionTest extends CQLTester
         execute("CREATE INDEX IF NOT EXISTS myindex ON %s (value1)");
 
         // drop and confirm
-        execute("DROP INDEX IF EXISTS myindex");
+        execute(format("DROP INDEX IF EXISTS %s.myindex", KEYSPACE));
 
         Object[][] rows = getRows(execute("select index_name from system.\"IndexInfo\" where table_name = ?", tableName));
         assertEquals(0, rows.length);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/test/unit/org/apache/cassandra/cql3/validation/operations/SelectSingleColumnRelationTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/SelectSingleColumnRelationTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/SelectSingleColumnRelationTest.java
index 09fd464..3795ce5 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/SelectSingleColumnRelationTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/SelectSingleColumnRelationTest.java
@@ -206,7 +206,7 @@ public class SelectSingleColumnRelationTest extends CQLTester
     @Test
     public void testClusteringColumnRelationsWithClusteringOrder() throws Throwable
     {
-        createTable("CREATE TABLE %s (a text, b int, c int, d int, primary key(a, b, c)) WITH CLUSTERING ORDER BY (b DESC);");
+        createTable("CREATE TABLE %s (a text, b int, c int, d int, primary key(a, b, c)) WITH CLUSTERING ORDER BY (b DESC, c ASC);");
         execute("insert into %s (a, b, c, d) values (?, ?, ?, ?)", "first", 1, 5, 1);
         execute("insert into %s (a, b, c, d) values (?, ?, ?, ?)", "first", 2, 6, 2);
         execute("insert into %s (a, b, c, d) values (?, ?, ?, ?)", "first", 3, 7, 3);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/test/unit/org/apache/cassandra/cql3/validation/operations/SelectTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/SelectTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/SelectTest.java
index 4f12b2b..54e079e 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/SelectTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/SelectTest.java
@@ -1859,7 +1859,7 @@ public class SelectTest extends CQLTester
         });
 
         // test clutering order
-        createTable("CREATE TABLE %s (a int, b int, c int, d int, e int, PRIMARY KEY ((a, b), c, d)) WITH CLUSTERING ORDER BY (c DESC)");
+        createTable("CREATE TABLE %s (a int, b int, c int, d int, e int, PRIMARY KEY ((a, b), c, d)) WITH CLUSTERING ORDER BY (c DESC, d ASC)");
 
         execute("INSERT INTO %s (a,b,c,d,e) VALUES (11, 11, 13, 14, 15)");
         execute("INSERT INTO %s (a,b,c,d,e) VALUES (11, 11, 14, 17, 18)");
@@ -2415,7 +2415,7 @@ public class SelectTest extends CQLTester
                    row("a", 3));
 
         // compound, first column DESC order
-        createTable("CREATE TABLE %s (a text, b int, c int, PRIMARY KEY (a, b, c)) WITH CLUSTERING ORDER BY (b DESC)");
+        createTable("CREATE TABLE %s (a text, b int, c int, PRIMARY KEY (a, b, c)) WITH CLUSTERING ORDER BY (b DESC, c ASC)");
 
         execute("INSERT INTO %s (a, b, c) VALUES ('a', 2, 4)");
         execute("INSERT INTO %s (a, b, c) VALUES ('a', 3, 5)");

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/test/unit/org/apache/cassandra/db/DirectoriesTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/DirectoriesTest.java b/test/unit/org/apache/cassandra/db/DirectoriesTest.java
index bbbb4e6..fa23560 100644
--- a/test/unit/org/apache/cassandra/db/DirectoriesTest.java
+++ b/test/unit/org/apache/cassandra/db/DirectoriesTest.java
@@ -35,7 +35,7 @@ import org.apache.cassandra.schema.Indexes;
 import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.config.Config.DiskFailurePolicy;
 import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.cql3.statements.IndexTarget;
+import org.apache.cassandra.cql3.statements.schema.IndexTarget;
 import org.apache.cassandra.db.Directories.DataDirectory;
 import org.apache.cassandra.db.marshal.UTF8Type;
 import org.apache.cassandra.index.internal.CassandraIndex;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/test/unit/org/apache/cassandra/db/RangeTombstoneTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/RangeTombstoneTest.java b/test/unit/org/apache/cassandra/db/RangeTombstoneTest.java
index b2c1542..3d1d003 100644
--- a/test/unit/org/apache/cassandra/db/RangeTombstoneTest.java
+++ b/test/unit/org/apache/cassandra/db/RangeTombstoneTest.java
@@ -32,7 +32,7 @@ import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.UpdateBuilder;
 import org.apache.cassandra.Util;
 import org.apache.cassandra.schema.ColumnMetadata;
-import org.apache.cassandra.cql3.statements.IndexTarget;
+import org.apache.cassandra.cql3.statements.schema.IndexTarget;
 import org.apache.cassandra.db.compaction.CompactionManager;
 import org.apache.cassandra.db.filter.ColumnFilter;
 import org.apache.cassandra.db.marshal.Int32Type;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/test/unit/org/apache/cassandra/db/RowIndexEntryTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/RowIndexEntryTest.java b/test/unit/org/apache/cassandra/db/RowIndexEntryTest.java
index e563070..d1c4527 100644
--- a/test/unit/org/apache/cassandra/db/RowIndexEntryTest.java
+++ b/test/unit/org/apache/cassandra/db/RowIndexEntryTest.java
@@ -33,7 +33,7 @@ import org.junit.Test;
 
 import org.apache.cassandra.Util;
 import org.apache.cassandra.cache.IMeasurableMemory;
-import org.apache.cassandra.cql3.statements.CreateTableStatement;
+import org.apache.cassandra.cql3.statements.schema.CreateTableStatement;
 import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.cql3.CQLTester;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/test/unit/org/apache/cassandra/db/SecondaryIndexTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/SecondaryIndexTest.java b/test/unit/org/apache/cassandra/db/SecondaryIndexTest.java
index 38b0a39..5dadc11 100644
--- a/test/unit/org/apache/cassandra/db/SecondaryIndexTest.java
+++ b/test/unit/org/apache/cassandra/db/SecondaryIndexTest.java
@@ -32,7 +32,7 @@ import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
 import org.apache.cassandra.schema.ColumnMetadata;
 import org.apache.cassandra.cql3.Operator;
-import org.apache.cassandra.cql3.statements.IndexTarget;
+import org.apache.cassandra.cql3.statements.schema.IndexTarget;
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.db.partitions.*;
 import org.apache.cassandra.db.rows.Row;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/test/unit/org/apache/cassandra/db/TableCQLHelperTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/TableCQLHelperTest.java b/test/unit/org/apache/cassandra/db/TableCQLHelperTest.java
index 701afbb..32fa4e4 100644
--- a/test/unit/org/apache/cassandra/db/TableCQLHelperTest.java
+++ b/test/unit/org/apache/cassandra/db/TableCQLHelperTest.java
@@ -32,7 +32,7 @@ import org.junit.Test;
 
 import org.apache.cassandra.*;
 import org.apache.cassandra.cql3.*;
-import org.apache.cassandra.cql3.statements.*;
+import org.apache.cassandra.cql3.statements.schema.IndexTarget;
 import org.apache.cassandra.db.marshal.*;
 import org.apache.cassandra.exceptions.*;
 import org.apache.cassandra.index.sasi.*;
@@ -406,7 +406,7 @@ public class TableCQLHelperTest extends CQLTester
         String tableName = createTable("CREATE TABLE IF NOT EXISTS %s (" +
                                        "pk1 varint," +
                                        "ck1 varint," +
-                                       "reg1 " + typeB + "," +
+                                       "reg1 frozen<" + typeB + ">," +
                                        "reg2 varint," +
                                        "PRIMARY KEY (pk1, ck1));");
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/test/unit/org/apache/cassandra/db/compaction/AbstractPendingRepairTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/compaction/AbstractPendingRepairTest.java b/test/unit/org/apache/cassandra/db/compaction/AbstractPendingRepairTest.java
index 3e38dfc..a320248 100644
--- a/test/unit/org/apache/cassandra/db/compaction/AbstractPendingRepairTest.java
+++ b/test/unit/org/apache/cassandra/db/compaction/AbstractPendingRepairTest.java
@@ -29,13 +29,11 @@ import org.junit.BeforeClass;
 import org.junit.Ignore;
 
 import org.apache.cassandra.SchemaLoader;
-import org.apache.cassandra.cql3.statements.CreateTableStatement;
-import org.apache.cassandra.locator.InetAddressAndPort;
-import org.apache.cassandra.schema.TableMetadata;
-import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.cql3.QueryProcessor;
+import org.apache.cassandra.cql3.statements.schema.CreateTableStatement;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.net.IMessageSink;
 import org.apache.cassandra.net.MessageIn;
 import org.apache.cassandra.net.MessageOut;
@@ -43,6 +41,8 @@ import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.repair.AbstractRepairTest;
 import org.apache.cassandra.repair.consistent.LocalSessionAccessor;
 import org.apache.cassandra.schema.KeyspaceParams;
+import org.apache.cassandra.schema.Schema;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.service.ActiveRepairService;
 
 @Ignore

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/test/unit/org/apache/cassandra/db/compaction/CompactionTaskTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/compaction/CompactionTaskTest.java b/test/unit/org/apache/cassandra/db/compaction/CompactionTaskTest.java
index 4640248..599fc74 100644
--- a/test/unit/org/apache/cassandra/db/compaction/CompactionTaskTest.java
+++ b/test/unit/org/apache/cassandra/db/compaction/CompactionTaskTest.java
@@ -32,7 +32,7 @@ import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.cql3.QueryProcessor;
-import org.apache.cassandra.cql3.statements.CreateTableStatement;
+import org.apache.cassandra.cql3.statements.schema.CreateTableStatement;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
 import org.apache.cassandra.io.sstable.format.SSTableReader;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/test/unit/org/apache/cassandra/db/compaction/CompactionsPurgeTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/compaction/CompactionsPurgeTest.java b/test/unit/org/apache/cassandra/db/compaction/CompactionsPurgeTest.java
index fc193e8..4573485 100644
--- a/test/unit/org/apache/cassandra/db/compaction/CompactionsPurgeTest.java
+++ b/test/unit/org/apache/cassandra/db/compaction/CompactionsPurgeTest.java
@@ -27,9 +27,9 @@ import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
-import org.apache.cassandra.cql3.statements.CreateTableStatement;
 import org.apache.cassandra.cql3.QueryProcessor;
 import org.apache.cassandra.cql3.UntypedResultSet;
+import org.apache.cassandra.cql3.statements.schema.CreateTableStatement;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.partitions.ImmutableBTreePartition;
 import org.apache.cassandra.db.rows.Row;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/test/unit/org/apache/cassandra/db/repair/CompactionManagerGetSSTablesForValidationTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/repair/CompactionManagerGetSSTablesForValidationTest.java b/test/unit/org/apache/cassandra/db/repair/CompactionManagerGetSSTablesForValidationTest.java
index 365ad7e..76ebfd8 100644
--- a/test/unit/org/apache/cassandra/db/repair/CompactionManagerGetSSTablesForValidationTest.java
+++ b/test/unit/org/apache/cassandra/db/repair/CompactionManagerGetSSTablesForValidationTest.java
@@ -31,9 +31,7 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
-import org.apache.cassandra.cql3.statements.CreateTableStatement;
-import org.apache.cassandra.db.compaction.CompactionManager;
-import org.apache.cassandra.locator.InetAddressAndPort;
+import org.apache.cassandra.cql3.statements.schema.CreateTableStatement;
 import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.schema.Schema;
@@ -42,6 +40,7 @@ import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.streaming.PreviewKind;
 import org.apache.cassandra.repair.RepairJobDesc;
 import org.apache.cassandra.repair.Validator;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/test/unit/org/apache/cassandra/db/repair/PendingAntiCompactionTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/repair/PendingAntiCompactionTest.java b/test/unit/org/apache/cassandra/db/repair/PendingAntiCompactionTest.java
index 269a725..447d504 100644
--- a/test/unit/org/apache/cassandra/db/repair/PendingAntiCompactionTest.java
+++ b/test/unit/org/apache/cassandra/db/repair/PendingAntiCompactionTest.java
@@ -41,21 +41,21 @@ import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.cql3.statements.CreateTableStatement;
-import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.db.compaction.CompactionManager;
 import org.apache.cassandra.schema.TableId;
 import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.cql3.QueryProcessor;
+import org.apache.cassandra.cql3.statements.schema.CreateTableStatement;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.dht.ByteOrderedPartitioner;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
-import org.apache.cassandra.streaming.PreviewKind;
+import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.schema.KeyspaceParams;
 import org.apache.cassandra.service.ActiveRepairService;
+import org.apache.cassandra.streaming.PreviewKind;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.UUIDGen;
 import org.apache.cassandra.utils.concurrent.Transactional;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/test/unit/org/apache/cassandra/db/streaming/CassandraStreamHeaderTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/streaming/CassandraStreamHeaderTest.java b/test/unit/org/apache/cassandra/db/streaming/CassandraStreamHeaderTest.java
index 289bb0f..061a4b2 100644
--- a/test/unit/org/apache/cassandra/db/streaming/CassandraStreamHeaderTest.java
+++ b/test/unit/org/apache/cassandra/db/streaming/CassandraStreamHeaderTest.java
@@ -22,7 +22,7 @@ import java.util.ArrayList;
 
 import org.junit.Test;
 
-import org.apache.cassandra.cql3.statements.CreateTableStatement;
+import org.apache.cassandra.cql3.statements.schema.CreateTableStatement;
 import org.apache.cassandra.db.SerializationHeader;
 import org.apache.cassandra.io.compress.CompressionMetadata;
 import org.apache.cassandra.io.sstable.format.SSTableFormat;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/test/unit/org/apache/cassandra/db/streaming/CassandraStreamManagerTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/streaming/CassandraStreamManagerTest.java b/test/unit/org/apache/cassandra/db/streaming/CassandraStreamManagerTest.java
index 80e4bfb..86018af 100644
--- a/test/unit/org/apache/cassandra/db/streaming/CassandraStreamManagerTest.java
+++ b/test/unit/org/apache/cassandra/db/streaming/CassandraStreamManagerTest.java
@@ -42,7 +42,7 @@ import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.concurrent.NamedThreadFactory;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.cql3.QueryProcessor;
-import org.apache.cassandra.cql3.statements.CreateTableStatement;
+import org.apache.cassandra.cql3.statements.schema.CreateTableStatement;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.dht.IPartitioner;
 import org.apache.cassandra.dht.Range;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/207c80c1/test/unit/org/apache/cassandra/hints/HintWriteTTLTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/hints/HintWriteTTLTest.java b/test/unit/org/apache/cassandra/hints/HintWriteTTLTest.java
index b06187d..21dbd7e 100644
--- a/test/unit/org/apache/cassandra/hints/HintWriteTTLTest.java
+++ b/test/unit/org/apache/cassandra/hints/HintWriteTTLTest.java
@@ -32,7 +32,7 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
-import org.apache.cassandra.cql3.statements.CreateTableStatement;
+import org.apache.cassandra.cql3.statements.schema.CreateTableStatement;
 import org.apache.cassandra.db.DecoratedKey;
 import org.apache.cassandra.db.Mutation;
 import org.apache.cassandra.db.partitions.PartitionUpdate;


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