You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by sn...@apache.org on 2015/02/15 20:07:47 UTC

cassandra git commit: Add an FunctionExecutionException to the protocol

Repository: cassandra
Updated Branches:
  refs/heads/trunk f61bc44a7 -> e3c018628


Add an FunctionExecutionException to the protocol

Patch by Robert Stupp; Reviewed by Aleksey Yeschenko for CASSANDRA-8528


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

Branch: refs/heads/trunk
Commit: e3c01862881362bba2b39ec01fe25e4945eafa36
Parents: f61bc44
Author: Robert Stupp <sn...@snazy.de>
Authored: Sun Feb 15 20:06:29 2015 +0100
Committer: Robert Stupp <sn...@snazy.de>
Committed: Sun Feb 15 20:06:29 2015 +0100

----------------------------------------------------------------------
 doc/native_protocol_v4.spec                     |  10 +-
 .../cql3/functions/JavaSourceUDFFactory.java    |   4 +-
 .../cql3/functions/ScriptBasedUDF.java          |   3 +-
 .../selection/AbstractFunctionSelector.java     |   2 +-
 .../cassandra/cql3/selection/Selection.java     |   2 +-
 .../statements/CreateAggregateStatement.java    |   2 +-
 .../cql3/statements/DropFunctionStatement.java  |   2 +-
 .../cassandra/exceptions/ExceptionCode.java     |   3 +-
 .../exceptions/FunctionExecutionException.java  |  47 +++++
 .../transport/messages/ErrorMessage.java        |  33 ++-
 .../apache/cassandra/cql3/AggregationTest.java  | 199 ++++++++++++-------
 .../org/apache/cassandra/cql3/CQLTester.java    |  32 ++-
 test/unit/org/apache/cassandra/cql3/UFTest.java | 166 +++++++++++-----
 13 files changed, 358 insertions(+), 147 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/e3c01862/doc/native_protocol_v4.spec
----------------------------------------------------------------------
diff --git a/doc/native_protocol_v4.spec b/doc/native_protocol_v4.spec
index 58896cf..5032bb5 100644
--- a/doc/native_protocol_v4.spec
+++ b/doc/native_protocol_v4.spec
@@ -1006,6 +1006,14 @@ Table of Contents
                 <data_present> is a single byte. If its value is 0, it means
                                the replica that was asked for data had not
                                responded. Otherwise, the value is != 0.
+    0x1400    Function_failure: A (user defined) function failed during execution.
+              The rest of the ERROR message body will be
+                <keyspace><function><arg_types>
+              where:
+                <keyspace> is the keyspace [string] of the failed function
+                <function> is the name [string] of the failed function
+                <arg_types> [string list] one string for each argument type (as CQL type) of the failed function
+
     0x2000    Syntax_error: The submitted query has a syntax error.
     0x2100    Unauthorized: The logged user doesn't have the right to perform
               the query.
@@ -1031,4 +1039,4 @@ Table of Contents
   * The format of "SCHEMA_CHANGE" events (Section 4.2.6) (and implicitly "Schema_change" results (Section 4.2.5.5))
     has been modified, and now includes changes related to user defined functions and user defined aggregates.
   * Read_failure error code was added.
-
+  * Function_failure error code was added.

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e3c01862/src/java/org/apache/cassandra/cql3/functions/JavaSourceUDFFactory.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/functions/JavaSourceUDFFactory.java b/src/java/org/apache/cassandra/cql3/functions/JavaSourceUDFFactory.java
index e4e6a55..4bce2f0 100644
--- a/src/java/org/apache/cassandra/cql3/functions/JavaSourceUDFFactory.java
+++ b/src/java/org/apache/cassandra/cql3/functions/JavaSourceUDFFactory.java
@@ -205,7 +205,7 @@ public final class JavaSourceUDFFactory
      *         logger.error("Invocation of function '{}' failed", this, t);
      *         if (t instanceof VirtualMachineError)
      *             throw (VirtualMachineError)t;
-     *         throw new org.apache.cassandra.exceptions.InvalidRequestException("Invocation of function '" + this + "' failed: " + t);
+     *         throw org.apache.cassandra.exceptions.FunctionExecutionException.build(this, t);
      *     }
      * }
      * </pre></code>
@@ -248,7 +248,7 @@ public final class JavaSourceUDFFactory
                     // handle OutOfMemoryError and other fatals not here!
                     "    if (t instanceof VirtualMachineError)\n" +
                     "      throw (VirtualMachineError)t;\n" +
-                    "    throw new org.apache.cassandra.exceptions.InvalidRequestException(\"Invocation of function '\" + this + \"' failed: \" + t);\n" +
+                    "    throw org.apache.cassandra.exceptions.FunctionExecutionException.build(this, t);\n" +
                     "  }\n" +
                     "}");
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e3c01862/src/java/org/apache/cassandra/cql3/functions/ScriptBasedUDF.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/functions/ScriptBasedUDF.java b/src/java/org/apache/cassandra/cql3/functions/ScriptBasedUDF.java
index 059a612..06452e6 100644
--- a/src/java/org/apache/cassandra/cql3/functions/ScriptBasedUDF.java
+++ b/src/java/org/apache/cassandra/cql3/functions/ScriptBasedUDF.java
@@ -35,6 +35,7 @@ import javax.script.SimpleBindings;
 
 import org.apache.cassandra.cql3.ColumnIdentifier;
 import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.exceptions.FunctionExecutionException;
 import org.apache.cassandra.exceptions.InvalidRequestException;
 
 public class ScriptBasedUDF extends UDFunction
@@ -139,7 +140,7 @@ public class ScriptBasedUDF extends UDFunction
         catch (RuntimeException | ScriptException e)
         {
             logger.info("Execution of UDF '{}' failed", name, e);
-            throw new InvalidRequestException("Execution of user-defined function '" + name + "' failed: " + e);
+            throw FunctionExecutionException.create(this, e);
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e3c01862/src/java/org/apache/cassandra/cql3/selection/AbstractFunctionSelector.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/selection/AbstractFunctionSelector.java b/src/java/org/apache/cassandra/cql3/selection/AbstractFunctionSelector.java
index 2bf169d..d6a0c71 100644
--- a/src/java/org/apache/cassandra/cql3/selection/AbstractFunctionSelector.java
+++ b/src/java/org/apache/cassandra/cql3/selection/AbstractFunctionSelector.java
@@ -48,7 +48,7 @@ abstract class AbstractFunctionSelector<T extends Function> extends Selector
         else
         {
             if (factories.doesAggregation() && !factories.containsOnlyAggregateFunctions())
-                throw new InvalidRequestException(String.format("the %s function arguments must be either all aggregates or all none aggregates",
+                throw new InvalidRequestException(String.format("arguments of function %s must be either all aggregates or no aggregates",
                                                                 fun.name()));
         }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e3c01862/src/java/org/apache/cassandra/cql3/selection/Selection.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/selection/Selection.java b/src/java/org/apache/cassandra/cql3/selection/Selection.java
index 58e994a..5d3a125 100644
--- a/src/java/org/apache/cassandra/cql3/selection/Selection.java
+++ b/src/java/org/apache/cassandra/cql3/selection/Selection.java
@@ -446,7 +446,7 @@ public abstract class Selection
             this.factories = factories;
 
             if (factories.doesAggregation() && !factories.containsOnlyAggregateFunctions())
-                throw new InvalidRequestException("the select clause must either contains only aggregates or none");
+                throw new InvalidRequestException("the select clause must either contain only aggregates or no aggregate");
         }
 
         @Override

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e3c01862/src/java/org/apache/cassandra/cql3/statements/CreateAggregateStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/CreateAggregateStatement.java b/src/java/org/apache/cassandra/cql3/statements/CreateAggregateStatement.java
index e135ffe..df7e87e 100644
--- a/src/java/org/apache/cassandra/cql3/statements/CreateAggregateStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/CreateAggregateStatement.java
@@ -132,7 +132,7 @@ public final class CreateAggregateStatement extends SchemaAlteringStatement
             finalFuncName = new FunctionName(functionName.keyspace, finalFunc);
             f = Functions.find(finalFuncName, Collections.<AbstractType<?>>singletonList(stateType));
             if (!(f instanceof ScalarFunction))
-                throw new InvalidRequestException("Final function " + finalFuncName + "(" + stateTypeRaw + ") does not exist");
+                throw new InvalidRequestException("Final function " + finalFuncName + "(" + stateTypeRaw + ") does not exist or is not a scalar function");
             fFinal = (ScalarFunction) f;
             returnType = fFinal.returnType();
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e3c01862/src/java/org/apache/cassandra/cql3/statements/DropFunctionStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/DropFunctionStatement.java b/src/java/org/apache/cassandra/cql3/statements/DropFunctionStatement.java
index 083db45..8863ffe 100644
--- a/src/java/org/apache/cassandra/cql3/statements/DropFunctionStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/DropFunctionStatement.java
@@ -136,7 +136,7 @@ public final class DropFunctionStatement extends SchemaAlteringStatement
 
         List<Function> references = Functions.getReferencesTo(old);
         if (!references.isEmpty())
-            throw new InvalidRequestException(String.format("Function '%s' still referenced by %s", functionName, references));
+            throw new InvalidRequestException(String.format("Function '%s' still referenced by %s", old, references));
 
         this.old = old;
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e3c01862/src/java/org/apache/cassandra/exceptions/ExceptionCode.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/exceptions/ExceptionCode.java b/src/java/org/apache/cassandra/exceptions/ExceptionCode.java
index 7fcb2d2..80cd4df 100644
--- a/src/java/org/apache/cassandra/exceptions/ExceptionCode.java
+++ b/src/java/org/apache/cassandra/exceptions/ExceptionCode.java
@@ -40,6 +40,7 @@ public enum ExceptionCode
     WRITE_TIMEOUT   (0x1100),
     READ_TIMEOUT    (0x1200),
     READ_FAILURE    (0x1300),
+    FUNCTION_FAILURE(0x1400),
 
     // 2xx: problem validating the request
     SYNTAX_ERROR    (0x2000),
@@ -50,7 +51,7 @@ public enum ExceptionCode
     UNPREPARED      (0x2500);
 
     public final int value;
-    private static final Map<Integer, ExceptionCode> valueToCode = new HashMap<Integer, ExceptionCode>(ExceptionCode.values().length);
+    private static final Map<Integer, ExceptionCode> valueToCode = new HashMap<>(ExceptionCode.values().length);
     static
     {
         for (ExceptionCode code : ExceptionCode.values())

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e3c01862/src/java/org/apache/cassandra/exceptions/FunctionExecutionException.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/exceptions/FunctionExecutionException.java b/src/java/org/apache/cassandra/exceptions/FunctionExecutionException.java
new file mode 100644
index 0000000..9c8d7ae
--- /dev/null
+++ b/src/java/org/apache/cassandra/exceptions/FunctionExecutionException.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.exceptions;
+
+import java.util.List;
+
+import org.apache.cassandra.cql3.functions.Function;
+import org.apache.cassandra.cql3.functions.FunctionName;
+import org.apache.cassandra.db.marshal.AbstractType;
+
+public class FunctionExecutionException extends CassandraException
+{
+    public final FunctionName functionName;
+    public final List<String> argTypes;
+    public final String detail;
+
+    public static FunctionExecutionException create(Function function, Throwable cause)
+    {
+        List<String> cqlTypes = AbstractType.asCQLTypeStringList(function.argTypes());
+        FunctionExecutionException fee = new FunctionExecutionException(function.name(), cqlTypes, cause.toString());
+        fee.initCause(cause);
+        return fee;
+    }
+
+    public FunctionExecutionException(FunctionName functionName, List<String> argTypes, String detail)
+    {
+        super(ExceptionCode.FUNCTION_FAILURE, "execution of '" + functionName + argTypes + "' failed: " + detail);
+        this.functionName = functionName;
+        this.argTypes = argTypes;
+        this.detail = detail;
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e3c01862/src/java/org/apache/cassandra/transport/messages/ErrorMessage.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/transport/messages/ErrorMessage.java b/src/java/org/apache/cassandra/transport/messages/ErrorMessage.java
index 3097c5b..d30e4ef 100644
--- a/src/java/org/apache/cassandra/transport/messages/ErrorMessage.java
+++ b/src/java/org/apache/cassandra/transport/messages/ErrorMessage.java
@@ -17,12 +17,15 @@
  */
 package org.apache.cassandra.transport.messages;
 
+import java.util.List;
+
 import io.netty.buffer.ByteBuf;
 import io.netty.handler.codec.CodecException;
 import com.google.common.base.Predicate;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.cassandra.cql3.functions.FunctionName;
 import org.apache.cassandra.db.ConsistencyLevel;
 import org.apache.cassandra.db.WriteType;
 import org.apache.cassandra.exceptions.*;
@@ -98,6 +101,12 @@ public class ErrorMessage extends Message.Response
                         te = new ReadTimeoutException(cl, received, blockFor, dataPresent != 0);
                     }
                     break;
+                case FUNCTION_FAILURE:
+                    String fKeyspace = CBUtil.readString(body);
+                    String fName = CBUtil.readString(body);
+                    List<String> argTypes = CBUtil.readStringList(body);
+                    te = new FunctionExecutionException(new FunctionName(fKeyspace, fName), argTypes, msg);
+                    break;
                 case UNPREPARED:
                     {
                         MD5Digest id = MD5Digest.wrap(CBUtil.readBytes(body));
@@ -166,6 +175,12 @@ public class ErrorMessage extends Message.Response
                     else
                         dest.writeByte((byte)(((ReadTimeoutException)rte).dataPresent ? 1 : 0));
                     break;
+                case FUNCTION_FAILURE:
+                    FunctionExecutionException fee = (FunctionExecutionException)msg.error;
+                    CBUtil.writeString(fee.functionName.keyspace, dest);
+                    CBUtil.writeString(fee.functionName.name, dest);
+                    CBUtil.writeStringList(fee.argTypes, dest);
+                    break;
                 case UNPREPARED:
                     PreparedQueryNotFoundException pqnfe = (PreparedQueryNotFoundException)err;
                     CBUtil.writeBytes(pqnfe.id.bytes, dest);
@@ -201,6 +216,12 @@ public class ErrorMessage extends Message.Response
                     size += CBUtil.sizeOfConsistencyLevel(rte.consistency) + 8;
                     size += isWrite ? CBUtil.sizeOfString(((WriteTimeoutException)rte).writeType.toString()) : 1;
                     break;
+                case FUNCTION_FAILURE:
+                    FunctionExecutionException fee = (FunctionExecutionException)msg.error;
+                    size += CBUtil.sizeOfString(fee.functionName.keyspace);
+                    size += CBUtil.sizeOfString(fee.functionName.name);
+                    size += CBUtil.sizeOfStringList(fee.argTypes);
+                    break;
                 case UNPREPARED:
                     PreparedQueryNotFoundException pqnfe = (PreparedQueryNotFoundException)err;
                     size += CBUtil.sizeOfBytes(pqnfe.id.bytes);
@@ -217,10 +238,16 @@ public class ErrorMessage extends Message.Response
 
     private static TransportException getBackwardsCompatibleException(ErrorMessage msg, int version)
     {
-        if (msg.error.code() == ExceptionCode.READ_FAILURE && version < Server.VERSION_4)
+        if (version < Server.VERSION_4)
         {
-            ReadFailureException rfe = (ReadFailureException) msg.error;
-            return new ReadTimeoutException(rfe.consistency, rfe.received, rfe.blockFor, rfe.dataPresent);
+            switch (msg.error.code())
+            {
+                case READ_FAILURE:
+                    ReadFailureException rfe = (ReadFailureException) msg.error;
+                    return new ReadTimeoutException(rfe.consistency, rfe.received, rfe.blockFor, rfe.dataPresent);
+                case FUNCTION_FAILURE:
+                    return new InvalidRequestException(msg.toString());
+            }
         }
 
         return msg.error;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e3c01862/test/unit/org/apache/cassandra/cql3/AggregationTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/AggregationTest.java b/test/unit/org/apache/cassandra/cql3/AggregationTest.java
index 1ddd1f1..86bd8f2 100644
--- a/test/unit/org/apache/cassandra/cql3/AggregationTest.java
+++ b/test/unit/org/apache/cassandra/cql3/AggregationTest.java
@@ -27,6 +27,10 @@ import org.apache.commons.lang3.time.DateUtils;
 import org.junit.Assert;
 import org.junit.Test;
 
+import org.apache.cassandra.cql3.functions.Functions;
+import org.apache.cassandra.cql3.functions.UDAggregate;
+import org.apache.cassandra.exceptions.FunctionExecutionException;
+import org.apache.cassandra.exceptions.InvalidRequestException;
 import org.apache.cassandra.service.ClientState;
 import org.apache.cassandra.transport.Event;
 import org.apache.cassandra.transport.messages.ResultMessage;
@@ -88,9 +92,9 @@ public class AggregationTest extends CQLTester
         execute("INSERT INTO %s (a, b, c) VALUES (1, 3, 8)");
 
         assertInvalidSyntax("SELECT max(b), max(c) FROM %s WHERE max(a) = 1");
-        assertInvalid("SELECT max(b), c FROM %s");
-        assertInvalid("SELECT b, max(c) FROM %s");
-        assertInvalid("SELECT max(sum(c)) FROM %s");
+        assertInvalidMessage("only aggregates or no aggregate", "SELECT max(b), c FROM %s");
+        assertInvalidMessage("only aggregates or no aggregate", "SELECT b, max(c) FROM %s");
+        assertInvalidMessage("aggregate functions cannot be used as arguments of aggregate functions", "SELECT max(sum(c)) FROM %s");
         assertInvalidSyntax("SELECT COUNT(2) FROM %s");
     }
 
@@ -129,8 +133,8 @@ public class AggregationTest extends CQLTester
         assertRows(execute("SELECT " + copySign + "(max(c), min(c)) FROM %s"), row(-1.4));
         assertRows(execute("SELECT " + copySign + "(c, d) FROM %s"), row(1.2), row(-1.3), row(1.4));
         assertRows(execute("SELECT max(" + copySign + "(c, d)) FROM %s"), row(1.4));
-        assertInvalid("SELECT " + copySign + "(c, max(c)) FROM %s");
-        assertInvalid("SELECT " + copySign + "(max(c), c) FROM %s");
+        assertInvalidMessage("must be either all aggregates or no aggregates", "SELECT " + copySign + "(c, max(c)) FROM %s");
+        assertInvalidMessage("must be either all aggregates or no aggregates", "SELECT " + copySign + "(max(c), c) FROM %s");
     }
 
     @Test
@@ -203,8 +207,8 @@ public class AggregationTest extends CQLTester
                                "AS '\"string\";';");
 
         // DROP AGGREGATE must not succeed against a scalar
-        assertInvalid("DROP AGGREGATE " + f);
-        assertInvalid("DROP AGGREGATE " + f + "(double, double)");
+        assertInvalidMessage("matches multiple function definitions", "DROP AGGREGATE " + f);
+        assertInvalidMessage("non existing", "DROP AGGREGATE " + f + "(double, double)");
 
         String a = createAggregate(KEYSPACE,
                                    "double",
@@ -218,12 +222,12 @@ public class AggregationTest extends CQLTester
                                 "STYPE int");
 
         // DROP FUNCTION must not succeed against an aggregate
-        assertInvalid("DROP FUNCTION " + a);
-        assertInvalid("DROP FUNCTION " + a + "(double)");
+        assertInvalidMessage("matches multiple function definitions", "DROP FUNCTION " + a);
+        assertInvalidMessage("non existing function", "DROP FUNCTION " + a + "(double)");
 
         // ambigious
-        assertInvalid("DROP AGGREGATE " + a);
-        assertInvalid("DROP AGGREGATE IF EXISTS " + a);
+        assertInvalidMessage("matches multiple function definitions", "DROP AGGREGATE " + a);
+        assertInvalidMessage("matches multiple function definitions", "DROP AGGREGATE IF EXISTS " + a);
 
         execute("DROP AGGREGATE IF EXISTS " + KEYSPACE + ".non_existing");
         execute("DROP AGGREGATE IF EXISTS " + a + "(int, text)");
@@ -250,7 +254,7 @@ public class AggregationTest extends CQLTester
                                    "STYPE double");
 
         // DROP FUNCTION must not succeed because the function is still referenced by the aggregate
-        assertInvalid("DROP FUNCTION " + f);
+        assertInvalidMessage("still referenced by", "DROP FUNCTION " + f);
 
         execute("DROP AGGREGATE " + a + "(double)");
     }
@@ -289,7 +293,7 @@ public class AggregationTest extends CQLTester
 
         execute("DROP AGGREGATE " + a + "(int)");
 
-        assertInvalid("SELECT " + a + "(b) FROM %s");
+        assertInvalidMessage("Unknown function", "SELECT " + a + "(b) FROM %s");
     }
 
     @Test
@@ -327,7 +331,7 @@ public class AggregationTest extends CQLTester
 
         execute("DROP AGGREGATE " + a + "(int)");
 
-        assertInvalid("SELECT " + a + "(b) FROM %s");
+        assertInvalidMessage("Unknown function", "SELECT " + a + "(b) FROM %s");
     }
 
     @Test
@@ -347,11 +351,12 @@ public class AggregationTest extends CQLTester
                                        "LANGUAGE java " +
                                        "AS 'return a.toString();'");
 
-        assertInvalid("CREATE AGGREGATE " + KEYSPACE + ".aggrInvalid(int)" +
-                      "SFUNC " + shortFunctionName(fState) + " " +
-                      "STYPE int " +
-                      "FINALFUNC " + shortFunctionName(fFinal) + " " +
-                      "INITCOND 'foobar'");
+        assertInvalidMessage("Invalid STRING constant (foobar)",
+                             "CREATE AGGREGATE " + KEYSPACE + ".aggrInvalid(int)" +
+                             "SFUNC " + shortFunctionName(fState) + " " +
+                             "STYPE int " +
+                             "FINALFUNC " + shortFunctionName(fFinal) + " " +
+                             "INITCOND 'foobar'");
     }
 
     @Test
@@ -385,34 +390,41 @@ public class AggregationTest extends CQLTester
                                         "LANGUAGE java " +
                                         "AS 'return a.toString();'");
 
-        assertInvalid("CREATE AGGREGATE " + KEYSPACE + ".aggrInvalid(double)" +
-                      "SFUNC " + shortFunctionName(fState) + " " +
-                      "STYPE double " +
-                      "FINALFUNC " + shortFunctionName(fFinal));
-        assertInvalid("CREATE AGGREGATE " + KEYSPACE + ".aggrInvalid(int)" +
-                      "SFUNC " + shortFunctionName(fState) + " " +
-                      "STYPE double " +
-                      "FINALFUNC " + shortFunctionName(fFinal));
-        assertInvalid("CREATE AGGREGATE " + KEYSPACE + ".aggrInvalid(double)" +
-                      "SFUNC " + shortFunctionName(fState) + " " +
-                      "STYPE int " +
-                      "FINALFUNC " + shortFunctionName(fFinal));
-        assertInvalid("CREATE AGGREGATE " + KEYSPACE + ".aggrInvalid(double)" +
-                      "SFUNC " + shortFunctionName(fState) + " " +
-                      "STYPE int");
-        assertInvalid("CREATE AGGREGATE " + KEYSPACE + ".aggrInvalid(int)" +
-                      "SFUNC " + shortFunctionName(fState) + " " +
-                      "STYPE double");
-
-        assertInvalid("CREATE AGGREGATE " + KEYSPACE + ".aggrInvalid(double)" +
-                      "SFUNC " + shortFunctionName(fState2) + " " +
-                      "STYPE double " +
-                      "FINALFUNC " + shortFunctionName(fFinal));
-
-        assertInvalid("CREATE AGGREGATE " + KEYSPACE + ".aggrInvalid(double)" +
-                      "SFUNC " + shortFunctionName(fState) + " " +
-                      "STYPE double " +
-                      "FINALFUNC " + shortFunctionName(fFinal2));
+        assertInvalidMessage("does not exist or is not a scalar function",
+                             "CREATE AGGREGATE " + KEYSPACE + ".aggrInvalid(double)" +
+                             "SFUNC " + shortFunctionName(fState) + " " +
+                             "STYPE double " +
+                             "FINALFUNC " + shortFunctionName(fFinal));
+        assertInvalidMessage("does not exist or is not a scalar function",
+                             "CREATE AGGREGATE " + KEYSPACE + ".aggrInvalid(int)" +
+                             "SFUNC " + shortFunctionName(fState) + " " +
+                             "STYPE double " +
+                             "FINALFUNC " + shortFunctionName(fFinal));
+        assertInvalidMessage("does not exist or is not a scalar function",
+                             "CREATE AGGREGATE " + KEYSPACE + ".aggrInvalid(double)" +
+                             "SFUNC " + shortFunctionName(fState) + " " +
+                             "STYPE int " +
+                             "FINALFUNC " + shortFunctionName(fFinal));
+        assertInvalidMessage("does not exist or is not a scalar function",
+                             "CREATE AGGREGATE " + KEYSPACE + ".aggrInvalid(double)" +
+                             "SFUNC " + shortFunctionName(fState) + " " +
+                             "STYPE int");
+        assertInvalidMessage("does not exist or is not a scalar function",
+                             "CREATE AGGREGATE " + KEYSPACE + ".aggrInvalid(int)" +
+                             "SFUNC " + shortFunctionName(fState) + " " +
+                             "STYPE double");
+
+        assertInvalidMessage("does not exist or is not a scalar function",
+                             "CREATE AGGREGATE " + KEYSPACE + ".aggrInvalid(double)" +
+                             "SFUNC " + shortFunctionName(fState2) + " " +
+                             "STYPE double " +
+                             "FINALFUNC " + shortFunctionName(fFinal));
+
+        assertInvalidMessage("does not exist or is not a scalar function",
+                             "CREATE AGGREGATE " + KEYSPACE + ".aggrInvalid(double)" +
+                             "SFUNC " + shortFunctionName(fState) + " " +
+                             "STYPE double " +
+                             "FINALFUNC " + shortFunctionName(fFinal2));
     }
 
     @Test
@@ -432,15 +444,17 @@ public class AggregationTest extends CQLTester
                                        "LANGUAGE java " +
                                        "AS 'return a.toString();'");
 
-        assertInvalid("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",
+                             "CREATE AGGREGATE " + KEYSPACE + ".aggrInvalid(int)" +
+                             "SFUNC " + shortFunctionName(fState) + "_not_there " +
+                             "STYPE int " +
+                             "FINALFUNC " + shortFunctionName(fFinal));
 
-        assertInvalid("CREATE AGGREGATE " + KEYSPACE + ".aggrInvalid(int)" +
-                      "SFUNC " + shortFunctionName(fState) + " " +
-                      "STYPE int " +
-                      "FINALFUNC " + shortFunctionName(fFinal) + "_not_there");
+        assertInvalidMessage("does not exist or is not a scalar function",
+                             "CREATE AGGREGATE " + KEYSPACE + ".aggrInvalid(int)" +
+                             "SFUNC " + shortFunctionName(fState) + " " +
+                             "STYPE int " +
+                             "FINALFUNC " + shortFunctionName(fFinal) + "_not_there");
 
         execute("CREATE AGGREGATE " + KEYSPACE + ".aggrInvalid(int)" +
                 "SFUNC " + shortFunctionName(fState) + " " +
@@ -507,17 +521,18 @@ public class AggregationTest extends CQLTester
                                     "FINALFUNC " + shortFunctionName(fFinalOK) + " " +
                                     "INITCOND null");
 
-        assertInvalid("SELECT " + a0 + "(b) FROM %s");
-        assertInvalid("SELECT " + a1 + "(b) FROM %s");
+        assertInvalidThrowMessage("java.lang.RuntimeException", FunctionExecutionException.class, "SELECT " + a0 + "(b) FROM %s");
+        assertInvalidThrowMessage("java.lang.RuntimeException", FunctionExecutionException.class, "SELECT " + a1 + "(b) FROM %s");
         assertRows(execute("SELECT " + a2 + "(b) FROM %s"), row("foobar"));
     }
 
     @Test
     public void testJavaAggregateWithoutStateOrFinal() throws Throwable
     {
-        assertInvalid("CREATE AGGREGATE " + KEYSPACE + ".jSumFooNE1(int) " +
-                      "SFUNC jSumFooNEstate " +
-                      "STYPE int");
+        assertInvalidMessage("does not exist or is not a scalar function",
+                             "CREATE AGGREGATE " + KEYSPACE + ".jSumFooNE1(int) " +
+                             "SFUNC jSumFooNEstate " +
+                             "STYPE int");
 
         String f = createFunction(KEYSPACE,
                                   "int, int",
@@ -526,10 +541,11 @@ public class AggregationTest extends CQLTester
                                   "LANGUAGE java " +
                                   "AS 'return Integer.valueOf((a!=null?a.intValue():0) + b.intValue());'");
 
-        assertInvalid("CREATE AGGREGATE " + KEYSPACE + ".jSumFooNE2(int) " +
-                      "SFUNC " + shortFunctionName(f) + " " +
-                      "STYPE int " +
-                      "FINALFUNC jSumFooNEfinal");
+        assertInvalidMessage("does not exist or is not a scalar function",
+                             "CREATE AGGREGATE " + KEYSPACE + ".jSumFooNE2(int) " +
+                             "SFUNC " + shortFunctionName(f) + " " +
+                             "STYPE int " +
+                             "FINALFUNC jSumFooNEfinal");
 
         execute("DROP FUNCTION " + f + "(int, int)");
     }
@@ -572,7 +588,7 @@ public class AggregationTest extends CQLTester
         execute("DROP FUNCTION " + fFinal + "(int)");
         execute("DROP FUNCTION " + fState + "(int, int)");
 
-        assertInvalid("SELECT " + a + "(b) FROM %s");
+        assertInvalidMessage("Unknown function", "SELECT " + a + "(b) FROM %s");
     }
 
     @Test
@@ -603,7 +619,7 @@ public class AggregationTest extends CQLTester
 
         execute("DROP FUNCTION " + fState + "(int, int)");
 
-        assertInvalid("SELECT " + a + "(b) FROM %s");
+        assertInvalidMessage("Unknown function", "SELECT " + a + "(b) FROM %s");
     }
 
     @Test
@@ -691,7 +707,7 @@ public class AggregationTest extends CQLTester
         execute("DROP FUNCTION " + fFinal + "(int)");
         execute("DROP FUNCTION " + fState + "(int, int)");
 
-        assertInvalid("SELECT " + a + "(b) FROM %s");
+        assertInvalidMessage("Unknown function", "SELECT " + a + "(b) FROM %s");
     }
 
     @Test
@@ -722,7 +738,7 @@ public class AggregationTest extends CQLTester
 
         execute("DROP FUNCTION " + fState + "(int, int)");
 
-        assertInvalid("SELECT " + a + "(b) FROM %s");
+        assertInvalidMessage("Unknown function", "SELECT " + a + "(b) FROM %s");
     }
 
     @Test
@@ -785,18 +801,49 @@ public class AggregationTest extends CQLTester
                                        "AS 'a + b;'");
 
         String a = createAggregate(KEYSPACE,
-                                   "int, int",
+                                   "int",
                                    "CREATE AGGREGATE %s(int) " +
                                    "SFUNC " + shortFunctionName(fState) + " " +
                                    "STYPE int ");
 
-        assertInvalid("CREATE AGGREGATE " + KEYSPACE + ".aggInv(int) " +
-                      "SFUNC " + shortFunctionName(a) + " " +
-                      "STYPE int ");
+        assertInvalidMessage("does not exist or is not a scalar function",
+                             "CREATE AGGREGATE " + KEYSPACE + ".aggInv(int) " +
+                             "SFUNC " + shortFunctionName(a) + " " +
+                             "STYPE int ");
 
-        assertInvalid("CREATE AGGREGATE " + KEYSPACE + ".aggInv(int) " +
-                      "SFUNC " + shortFunctionName(fState) + " " +
-                      "STYPE int " +
-                      "FINALFUNC " + shortFunctionName(a));
+        assertInvalidMessage("does not exist or is not a scalar function",
+                             "CREATE AGGREGATE " + KEYSPACE + ".aggInv(int) " +
+                             "SFUNC " + shortFunctionName(fState) + " " +
+                             "STYPE int " +
+                             "FINALFUNC " + shortFunctionName(a));
     }
+
+    @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) " +
+                                       "RETURNS int " +
+                                       "LANGUAGE javascript " +
+                                       "AS 'a + b;'");
+
+        String a = createAggregate(KEYSPACE,
+                                   "int",
+                                   "CREATE AGGREGATE %s(int) " +
+                                   "SFUNC " + shortFunctionName(fState) + " " +
+                                   "STYPE int ");
+
+        UDAggregate f = (UDAggregate) Functions.find(parseFunctionName(a)).get(0);
+
+        Functions.replaceFunction(UDAggregate.createBroken(f.name(), f.argTypes(), f.returnType(),
+                                                           null, new InvalidRequestException("foo bar is broken")));
+
+        assertInvalidThrowMessage("foo bar is broken", InvalidRequestException.class,
+                                  "SELECT " + a + "(val) FROM %s");
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e3c01862/test/unit/org/apache/cassandra/cql3/CQLTester.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/CQLTester.java b/test/unit/org/apache/cassandra/cql3/CQLTester.java
index efda704..1baebb6 100644
--- a/test/unit/org/apache/cassandra/cql3/CQLTester.java
+++ b/test/unit/org/apache/cassandra/cql3/CQLTester.java
@@ -619,7 +619,7 @@ public abstract class CQLTester
 
     protected void assertEmpty(UntypedResultSet result) throws Throwable
     {
-        if (result != null && result.size() != 0)
+        if (result != null && !result.isEmpty())
             throw new AssertionError(String.format("Expected empty result but got %d rows", result.size()));
     }
 
@@ -630,6 +630,16 @@ public abstract class CQLTester
 
     protected void assertInvalidMessage(String errorMessage, String query, Object... values) throws Throwable
     {
+        assertInvalidThrowMessage(errorMessage, null, query, values);
+    }
+
+    protected void assertInvalidThrow(Class<? extends Throwable> exception, String query, Object... values) throws Throwable
+    {
+        assertInvalidThrowMessage(null, exception, query, values);
+    }
+
+    protected void assertInvalidThrowMessage(String errorMessage, Class<? extends Throwable> exception, String query, Object... values) throws Throwable
+    {
         try
         {
             execute(query, values);
@@ -638,8 +648,14 @@ public abstract class CQLTester
                      : replaceValues(query, values);
             Assert.fail("Query should be invalid but no error was thrown. Query is: " + q);
         }
-        catch (InvalidRequestException e)
+        catch (CassandraException e)
         {
+            if (exception != null && !exception.isAssignableFrom(e.getClass()))
+            {
+                Assert.fail("Query should be invalid but wrong error was thrown. " +
+                            "Expected: " + exception.getName() + ", got: " + e.getClass().getName() + ". " +
+                            "Query is: " + queryInfo(query, values));
+            }
             if (errorMessage != null)
             {
                 assertMessageContains(errorMessage, e);
@@ -647,6 +663,13 @@ public abstract class CQLTester
         }
     }
 
+    private static String queryInfo(String query, Object[] values)
+    {
+        return USE_PREPARED_VALUES
+               ? query + " (values: " + formatAllValues(values) + ")"
+               : replaceValues(query, values);
+    }
+
     protected void assertInvalidSyntax(String query, Object... values) throws Throwable
     {
         assertInvalidSyntaxMessage(null, query, values);
@@ -657,10 +680,7 @@ public abstract class CQLTester
         try
         {
             execute(query, values);
-            String q = USE_PREPARED_VALUES
-                     ? query + " (values: " + formatAllValues(values) + ")"
-                     : replaceValues(query, values);
-            Assert.fail("Query should have invalid syntax but no error was thrown. Query is: " + q);
+            Assert.fail("Query should have invalid syntax but no error was thrown. Query is: " + queryInfo(query, values));
         }
         catch (SyntaxException e)
         {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e3c01862/test/unit/org/apache/cassandra/cql3/UFTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/UFTest.java b/test/unit/org/apache/cassandra/cql3/UFTest.java
index ea1b2da..84a7dd9 100644
--- a/test/unit/org/apache/cassandra/cql3/UFTest.java
+++ b/test/unit/org/apache/cassandra/cql3/UFTest.java
@@ -22,11 +22,14 @@ import java.math.BigInteger;
 import java.util.*;
 
 import org.junit.Assert;
+import org.junit.Ignore;
 import org.junit.Test;
 
 import com.datastax.driver.core.*;
 import org.apache.cassandra.cql3.functions.FunctionName;
 import org.apache.cassandra.cql3.functions.Functions;
+import org.apache.cassandra.cql3.functions.UDFunction;
+import org.apache.cassandra.exceptions.FunctionExecutionException;
 import org.apache.cassandra.exceptions.InvalidRequestException;
 import org.apache.cassandra.service.ClientState;
 import org.apache.cassandra.transport.Event;
@@ -159,7 +162,7 @@ public class UFTest extends CQLTester
                                      "LANGUAGE java " +
                                      "AS 'return Double.valueOf(Math.sin(input.doubleValue()));'");
         // check we can't recreate the same function
-        assertInvalid("CREATE FUNCTION " + fSin + " ( input double ) RETURNS double LANGUAGE java AS 'return Double.valueOf(Math.sin(input.doubleValue()));'");
+        assertInvalidMessage("already exists", "CREATE FUNCTION " + fSin + " ( input double ) RETURNS double LANGUAGE java AS 'return Double.valueOf(Math.sin(input.doubleValue()));'");
         // but that it doesn't comply with "IF NOT EXISTS"
         execute("CREATE FUNCTION IF NOT EXISTS " + fSin + " ( input double ) RETURNS double LANGUAGE java AS 'return Double.valueOf(Math.sin(input.doubleValue()));'");
 
@@ -171,7 +174,8 @@ public class UFTest extends CQLTester
         );
 
         // Replace the method with incompatible return type
-        assertInvalid("CREATE OR REPLACE FUNCTION " + fSin + " ( input double ) RETURNS text LANGUAGE java AS 'return Double.valueOf(42d);'");
+        assertInvalidMessage("the new return type text is not compatible with the return type double of existing function",
+                             "CREATE OR REPLACE FUNCTION " + fSin + " ( input double ) RETURNS text LANGUAGE java AS 'return Double.valueOf(42d);'");
         // proper replacement
         execute("CREATE OR REPLACE FUNCTION " + fSin + " ( input double ) RETURNS double LANGUAGE java AS 'return Double.valueOf(42d);'");
 
@@ -199,16 +203,16 @@ public class UFTest extends CQLTester
         execute("DROP FUNCTION " + fSin2);
 
         // Drop unexisting function
-        assertInvalid("DROP FUNCTION " + fSin);
+        assertInvalidMessage("Cannot drop non existing function", "DROP FUNCTION " + fSin);
         // but don't complain with "IF EXISTS"
         execute("DROP FUNCTION IF EXISTS " + fSin);
 
         // can't drop native functions
-        assertInvalid("DROP FUNCTION dateof");
-        assertInvalid("DROP FUNCTION uuid");
+        assertInvalidMessage("system keyspace is not user-modifiable", "DROP FUNCTION dateof");
+        assertInvalidMessage("system keyspace is not user-modifiable", "DROP FUNCTION uuid");
 
         // sin() no longer exists
-        assertInvalid("SELECT key, sin(d) FROM %s");
+        assertInvalidMessage("Unknown function", "SELECT key, sin(d) FROM %s");
     }
 
     @Test
@@ -254,7 +258,8 @@ public class UFTest extends CQLTester
                                "CREATE OR REPLACE FUNCTION %s(v ascii) RETURNS text LANGUAGE java AS 'return \"f1\";'");
 
         // text == varchar, so this should be considered as a duplicate
-        assertInvalid("CREATE FUNCTION " + fOverload + "(v varchar) RETURNS text LANGUAGE java AS 'return \"f1\";'");
+        assertInvalidMessage("already exists",
+                             "CREATE FUNCTION " + fOverload + "(v varchar) RETURNS text LANGUAGE java AS 'return \"f1\";'");
 
         assertRows(execute("SELECT " + fOverload + "(k), " + fOverload + "(v), " + fOverload + "(k, k) FROM %s"),
             row("f1", "f2", "f3")
@@ -262,7 +267,7 @@ public class UFTest extends CQLTester
 
         forcePreparedValues();
         // This shouldn't work if we use preparation since there no way to know which overload to use
-        assertInvalid("SELECT v FROM %s WHERE k = " + fOverload + "(?)", "foo");
+        assertInvalidMessage("Ambiguous call to function", "SELECT v FROM %s WHERE k = " + fOverload + "(?)", "foo");
         stopForcingPreparedValues();
 
         // but those should since we specifically cast
@@ -273,17 +278,17 @@ public class UFTest extends CQLTester
         assertEmpty(execute("SELECT v FROM %s WHERE k = " + fOverload + "((varchar)?)", "foo"));
 
         // no such functions exist...
-        assertInvalid("DROP FUNCTION " + fOverload + "(boolean)");
-        assertInvalid("DROP FUNCTION " + fOverload + "(bigint)");
+        assertInvalidMessage("non existing function", "DROP FUNCTION " + fOverload + "(boolean)");
+        assertInvalidMessage("non existing function", "DROP FUNCTION " + fOverload + "(bigint)");
 
         // 'overloaded' has multiple overloads - so it has to fail (CASSANDRA-7812)
-        assertInvalid("DROP FUNCTION " + fOverload);
+        assertInvalidMessage("matches multiple function definitions", "DROP FUNCTION " + fOverload);
         execute("DROP FUNCTION " + fOverload + "(varchar)");
-        assertInvalid("SELECT v FROM %s WHERE k = " + fOverload + "((text)?)", "foo");
+        assertInvalidMessage("none of its type signatures match", "SELECT v FROM %s WHERE k = " + fOverload + "((text)?)", "foo");
         execute("DROP FUNCTION " + fOverload + "(text, text)");
-        assertInvalid("SELECT v FROM %s WHERE k = " + fOverload + "((text)?,(text)?)", "foo", "bar");
+        assertInvalidMessage("none of its type signatures match", "SELECT v FROM %s WHERE k = " + fOverload + "((text)?,(text)?)", "foo", "bar");
         execute("DROP FUNCTION " + fOverload + "(ascii)");
-        assertInvalid("SELECT v FROM %s WHERE k = " + fOverload + "((ascii)?)", "foo");
+        assertInvalidMessage("cannot be passed as argument 0 of function", "SELECT v FROM %s WHERE k = " + fOverload + "((ascii)?)", "foo");
         // single-int-overload must still work
         assertRows(execute("SELECT v FROM %s WHERE k = " + fOverload + "((int)?)", 3), row(1));
         // overloaded has just one overload now - so the following DROP FUNCTION is not ambigious (CASSANDRA-7812)
@@ -392,8 +397,9 @@ public class UFTest extends CQLTester
     @Test
     public void testJavaFunctionInvalidReturn() throws Throwable
     {
-        assertInvalid("CREATE OR REPLACE FUNCTION jfir(val double) RETURNS double LANGUAGE JAVA\n" +
-                      "AS 'return Long.valueOf(1L);';");
+        assertInvalidMessage("system keyspace is not user-modifiable",
+                             "CREATE OR REPLACE FUNCTION jfir(val double) RETURNS double LANGUAGE JAVA\n" +
+                             "AS 'return Long.valueOf(1L);';");
     }
 
     @Test
@@ -409,7 +415,8 @@ public class UFTest extends CQLTester
         execute("INSERT INTO %s (key, val) VALUES (?, ?)", 1, 1L);
         execute("INSERT INTO %s (key, val) VALUES (?, ?)", 2, 2L);
         execute("INSERT INTO %s (key, val) VALUES (?, ?)", 3, 3L);
-        assertInvalid("SELECT key, val, " + fName + "(val) FROM %s");
+        assertInvalidMessage("val cannot be passed as argument 0 of function",
+                             "SELECT key, val, " + fName + "(val) FROM %s");
     }
 
     @Test
@@ -462,7 +469,8 @@ public class UFTest extends CQLTester
         execute("INSERT INTO %s (key, val) VALUES (?, ?)", 1, 1d);
         execute("INSERT INTO %s (key, val) VALUES (?, ?)", 2, 2d);
         execute("INSERT INTO %s (key, val) VALUES (?, ?)", 3, 3d);
-        assertInvalid("SELECT key, val, " + parseFunctionName(fName).name + "(val) FROM %s");
+        assertInvalidMessage("Unknown function",
+                             "SELECT key, val, " + parseFunctionName(fName).name + "(val) FROM %s");
 
         execute("INSERT INTO " + KEYSPACE_PER_TEST + ".second_tab (key, val) VALUES (?, ?)", 1, 1d);
         execute("INSERT INTO " + KEYSPACE_PER_TEST + ".second_tab (key, val) VALUES (?, ?)", 2, 2d);
@@ -507,25 +515,32 @@ public class UFTest extends CQLTester
         execute("CREATE OR REPLACE FUNCTION " + KEYSPACE + ".dateof(val timeuuid) RETURNS timestamp LANGUAGE JAVA\n" +
                 "AS 'return null;';");
 
-        assertInvalid("CREATE OR REPLACE FUNCTION system.jnft(val double) RETURNS double LANGUAGE JAVA\n" +
-                      "AS 'return null;';");
-        assertInvalid("CREATE OR REPLACE FUNCTION system.dateof(val timeuuid) RETURNS timestamp LANGUAGE JAVA\n" +
-                      "AS 'return null;';");
-        assertInvalid("DROP FUNCTION system.now");
+        assertInvalidMessage("system keyspace is not user-modifiable",
+                             "CREATE OR REPLACE FUNCTION system.jnft(val double) RETURNS double LANGUAGE JAVA\n" +
+                             "AS 'return null;';");
+        assertInvalidMessage("system keyspace is not user-modifiable",
+                             "CREATE OR REPLACE FUNCTION system.dateof(val timeuuid) RETURNS timestamp LANGUAGE JAVA\n" +
+                             "AS 'return null;';");
+        assertInvalidMessage("system keyspace is not user-modifiable",
+                             "DROP FUNCTION system.now");
 
         // KS for executeInternal() is system
-        assertInvalid("CREATE OR REPLACE FUNCTION jnft(val double) RETURNS double LANGUAGE JAVA\n" +
-                      "AS 'return null;';");
-        assertInvalid("CREATE OR REPLACE FUNCTION dateof(val timeuuid) RETURNS timestamp LANGUAGE JAVA\n" +
-                      "AS 'return null;';");
-        assertInvalid("DROP FUNCTION now");
+        assertInvalidMessage("system keyspace is not user-modifiable",
+                             "CREATE OR REPLACE FUNCTION jnft(val double) RETURNS double LANGUAGE JAVA\n" +
+                             "AS 'return null;';");
+        assertInvalidMessage("system keyspace is not user-modifiable",
+                             "CREATE OR REPLACE FUNCTION dateof(val timeuuid) RETURNS timestamp LANGUAGE JAVA\n" +
+                             "AS 'return null;';");
+        assertInvalidMessage("system keyspace is not user-modifiable",
+                             "DROP FUNCTION now");
     }
 
     @Test
     public void testFunctionNonExistingKeyspace() throws Throwable
     {
-        assertInvalid("CREATE OR REPLACE FUNCTION this_ks_does_not_exist.jnft(val double) RETURNS double LANGUAGE JAVA\n" +
-                      "AS 'return null;';");
+        assertInvalidMessage("to non existing keyspace",
+                             "CREATE OR REPLACE FUNCTION this_ks_does_not_exist.jnft(val double) RETURNS double LANGUAGE JAVA\n" +
+                             "AS 'return null;';");
     }
 
     @Test
@@ -533,8 +548,9 @@ public class UFTest extends CQLTester
     {
         dropPerTestKeyspace();
 
-        assertInvalid("CREATE OR REPLACE FUNCTION " + KEYSPACE_PER_TEST + ".jnft(val double) RETURNS double LANGUAGE JAVA\n" +
-                      "AS 'return null;';");
+        assertInvalidMessage("to non existing keyspace",
+                             "CREATE OR REPLACE FUNCTION " + KEYSPACE_PER_TEST + ".jnft(val double) RETURNS double LANGUAGE JAVA\n" +
+                             "AS 'return null;';");
     }
 
     @Test
@@ -593,8 +609,9 @@ public class UFTest extends CQLTester
         execute("INSERT INTO %s (key, val) VALUES (?, ?)", 2, 2d);
         execute("INSERT INTO %s (key, val) VALUES (?, ?)", 3, 3d);
 
-        // function throws a RuntimeException which is wrapped by InvalidRequestException
-        assertInvalid("SELECT key, val, " + fName + "(val) FROM %s");
+        // function throws a RuntimeException which is wrapped by FunctionExecutionException
+        assertInvalidThrowMessage("java.lang.RuntimeException: oh no", FunctionExecutionException.class,
+                                  "SELECT key, val, " + fName + "(val) FROM %s");
     }
 
     @Test
@@ -991,12 +1008,12 @@ public class UFTest extends CQLTester
         Assert.assertNotNull(QueryProcessor.instance.getPrepared(prepared.statementId));
 
         // UT still referenced by table
-        assertInvalid("DROP TYPE " + type);
+        assertInvalidMessage("Cannot drop user type", "DROP TYPE " + type);
 
         execute("DROP TABLE %s");
 
         // UT still referenced by UDF
-        assertInvalid("DROP TYPE " + type);
+        assertInvalidMessage("as it is still used by function", "DROP TYPE " + type);
 
         Assert.assertNull(QueryProcessor.instance.getPrepared(prepared.statementId));
 
@@ -1023,7 +1040,8 @@ public class UFTest extends CQLTester
 
         execute("ALTER TYPE " + type + " RENAME txt TO str");
 
-        assertInvalid("SELECT " + fName + "(udt) FROM %s WHERE key = 1");
+        assertInvalidMessage("txt is not a field defined in this UDT",
+                             "SELECT " + fName + "(udt) FROM %s WHERE key = 1");
 
         execute("ALTER TYPE " + type + " RENAME str TO txt");
 
@@ -1393,11 +1411,11 @@ public class UFTest extends CQLTester
                                "AS $$" +
                                "        st.iterator().next().getString(\"txt\");$$;");
         createFunctionOverload(fName, "map<int, frozen<" + type + ">>",
-                       "CREATE FUNCTION %s( mp map<int, frozen<" + type + ">> ) " +
-                       "RETURNS text " +
-                       "LANGUAGE javascript\n" +
-                       "AS $$" +
-                       "        mp.get(java.lang.Integer.valueOf(3)).getString(\"txt\");$$;");
+                               "CREATE FUNCTION %s( mp map<int, frozen<" + type + ">> ) " +
+                               "RETURNS text " +
+                               "LANGUAGE javascript\n" +
+                               "AS $$" +
+                               "        mp.get(java.lang.Integer.valueOf(3)).getString(\"txt\");$$;");
 
         execute("INSERT INTO %s (key, lst, st, mp) values (1, " +
                 // list<frozen<UDT>>
@@ -1467,7 +1485,7 @@ public class UFTest extends CQLTester
 
         execute("INSERT INTO %s (key, val) VALUES (?, ?)", 1, 1d);
         // throws IRE with ClassCastException
-        assertInvalid("SELECT key, val, " + fName + "(val) FROM %s");
+        assertInvalidMessage("Invalid value for CQL type double", "SELECT key, val, " + fName + "(val) FROM %s");
     }
 
     @Test
@@ -1483,31 +1501,35 @@ public class UFTest extends CQLTester
 
         execute("INSERT INTO %s (key, val) VALUES (?, ?)", 1, 1d);
         // throws IRE with ScriptException
-        assertInvalid("SELECT key, val, " + fName + "(val) FROM %s");
+        assertInvalidThrowMessage("fool", FunctionExecutionException.class,
+                                  "SELECT key, val, " + fName + "(val) FROM %s");
     }
 
     @Test
     public void testDuplicateArgNames() throws Throwable
     {
-        assertInvalid("CREATE OR REPLACE FUNCTION " + KEYSPACE + ".scrinv(val double, val text) " +
-                      "RETURNS text LANGUAGE javascript\n" +
-                      "AS '\"foo bar\";';");
+        assertInvalidMessage("duplicate argument names for given function",
+                             "CREATE OR REPLACE FUNCTION " + KEYSPACE + ".scrinv(val double, val text) " +
+                             "RETURNS text LANGUAGE javascript\n" +
+                             "AS '\"foo bar\";';");
     }
 
     @Test
     public void testJavascriptCompileFailure() throws Throwable
     {
-        assertInvalid("CREATE OR REPLACE FUNCTION " + KEYSPACE + ".scrinv(val double) " +
-                      "RETURNS double LANGUAGE javascript\n" +
-                      "AS 'foo bar';");
+        assertInvalidMessage("Failed to compile function 'cql_test_keyspace.scrinv'",
+                             "CREATE OR REPLACE FUNCTION " + KEYSPACE + ".scrinv(val double) " +
+                             "RETURNS double LANGUAGE javascript\n" +
+                             "AS 'foo bar';");
     }
 
     @Test
     public void testScriptInvalidLanguage() throws Throwable
     {
-        assertInvalid("CREATE OR REPLACE FUNCTION " + KEYSPACE + ".scrinv(val double) " +
-                      "RETURNS double LANGUAGE artificial_intelligence\n" +
-                      "AS 'question for 42?';");
+        assertInvalidMessage("Invalid language 'artificial_intelligence' for function 'cql_test_keyspace.scrinv'",
+                             "CREATE OR REPLACE FUNCTION " + KEYSPACE + ".scrinv(val double) " +
+                             "RETURNS double LANGUAGE artificial_intelligence\n" +
+                             "AS 'question for 42?';");
     }
 
     @Test
@@ -1580,4 +1602,42 @@ public class UFTest extends CQLTester
                        row(1, expected1, expected2));
         }
     }
+
+    @Test
+    public void testBrokenFunction() throws Throwable
+    {
+        createTable("CREATE TABLE %s (key int primary key, dval double)");
+        execute("INSERT INTO %s (key, dval) VALUES (?, ?)", 1, 1d);
+
+        String fName = createFunction(KEYSPACE_PER_TEST, "double",
+                                      "CREATE OR REPLACE FUNCTION %s(val double) RETURNS double LANGUAGE JAVA\n" +
+                                      "AS 'throw new RuntimeException();';");
+
+        UDFunction f = (UDFunction) Functions.find(parseFunctionName(fName)).get(0);
+
+        Functions.replaceFunction(UDFunction.createBrokenFunction(f.name(), f.argNames(), f.argTypes(), f.returnType(),
+                                                                  "java", f.body(), new InvalidRequestException("foo bar is broken")));
+
+        assertInvalidThrowMessage("foo bar is broken", InvalidRequestException.class,
+                                  "SELECT key, " + fName + "(dval) FROM %s");
+    }
+
+    @Test
+    @Ignore("implement this unit test when Java Driver can handle new ExceptionCode.")
+    public void testFunctionExecutionExceptionNet() throws Throwable
+    {
+        createTable("CREATE TABLE %s (key int primary key, dval double)");
+        execute("INSERT INTO %s (key, dval) VALUES (?, ?)", 1, 1d);
+
+        String fName = createFunction(KEYSPACE_PER_TEST, "double",
+                                      "CREATE OR REPLACE FUNCTION %s(val double) RETURNS double LANGUAGE JAVA\n" +
+                                      "AS 'throw new RuntimeException()';");
+
+        for (int version = Server.VERSION_2; version <= Server.CURRENT_VERSION; version++)
+        {
+            // TODO replace with appropiate code
+            assertRowsNet(version,
+                          executeNet(version, "SELECT " + fName + "(dval) FROM %s WHERE key = 1"));
+        }
+    }
 }