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 2016/12/07 12:38:50 UTC

[15/25] cassandra git commit: Merge branch 'cassandra-2.2' into cassandra-3.0

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3aefe356/test/unit/org/apache/cassandra/cql3/validation/entities/UFScriptTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/cql3/validation/entities/UFScriptTest.java
index 0000000,d3050a5..af3c894
mode 000000,100644..100644
--- a/test/unit/org/apache/cassandra/cql3/validation/entities/UFScriptTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/UFScriptTest.java
@@@ -1,0 -1,504 +1,428 @@@
+ /*
+  * 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.cql3.validation.entities;
+ 
+ import java.math.BigDecimal;
+ import java.math.BigInteger;
+ import java.util.Arrays;
+ import java.util.Date;
+ import java.util.List;
+ import java.util.Map;
+ import java.util.Set;
+ import java.util.TreeMap;
+ import java.util.TreeSet;
+ import java.util.UUID;
+ 
+ import org.junit.Assert;
 -import org.junit.BeforeClass;
+ import org.junit.Test;
+ 
 -import com.datastax.driver.core.DataType;
 -import com.datastax.driver.core.TupleType;
 -import com.datastax.driver.core.TupleValue;
+ import org.apache.cassandra.config.DatabaseDescriptor;
+ import org.apache.cassandra.cql3.CQLTester;
+ import org.apache.cassandra.cql3.UntypedResultSet;
+ import org.apache.cassandra.cql3.functions.FunctionName;
 -import org.apache.cassandra.dht.ByteOrderedPartitioner;
+ import org.apache.cassandra.exceptions.FunctionExecutionException;
 -import org.apache.cassandra.transport.Server;
+ import org.apache.cassandra.utils.UUIDGen;
+ 
+ public class UFScriptTest extends CQLTester
+ {
 -    @BeforeClass
 -    public static void setUp()
 -    {
 -        DatabaseDescriptor.setPartitioner(ByteOrderedPartitioner.instance);
 -    }
++    // Just JavaScript UDFs to check how UDF - especially security/class-loading/sandboxing stuff -
++    // behaves, if no Java UDF has been executed before.
++
++    // Do not add any other test here - especially none using Java UDFs
+ 
+     @Test
+     public void testJavascriptSimpleCollections() throws Throwable
+     {
+         createTable("CREATE TABLE %s (key int primary key, lst list<double>, st set<text>, mp map<int, boolean>)");
+ 
+         String fName1 = createFunction(KEYSPACE_PER_TEST, "list<double>",
+                                        "CREATE FUNCTION %s( lst list<double> ) " +
+                                        "RETURNS NULL ON NULL INPUT " +
+                                        "RETURNS list<double> " +
+                                        "LANGUAGE javascript\n" +
+                                        "AS 'lst;';");
+         String fName2 = createFunction(KEYSPACE_PER_TEST, "set<text>",
+                                        "CREATE FUNCTION %s( st set<text> ) " +
+                                        "RETURNS NULL ON NULL INPUT " +
+                                        "RETURNS set<text> " +
+                                        "LANGUAGE javascript\n" +
+                                        "AS 'st;';");
+         String fName3 = createFunction(KEYSPACE_PER_TEST, "map<int, boolean>",
+                                        "CREATE FUNCTION %s( mp map<int, boolean> ) " +
+                                        "RETURNS NULL ON NULL INPUT " +
+                                        "RETURNS map<int, boolean> " +
+                                        "LANGUAGE javascript\n" +
+                                        "AS 'mp;';");
+ 
+         List<Double> list = Arrays.asList(1d, 2d, 3d);
+         Set<String> set = new TreeSet<>(Arrays.asList("one", "three", "two"));
+         Map<Integer, Boolean> map = new TreeMap<>();
+         map.put(1, true);
+         map.put(2, false);
+         map.put(3, true);
+ 
+         execute("INSERT INTO %s (key, lst, st, mp) VALUES (1, ?, ?, ?)", list, set, map);
+ 
+         assertRows(execute("SELECT lst, st, mp FROM %s WHERE key = 1"),
+                    row(list, set, map));
+ 
+         assertRows(execute("SELECT " + fName1 + "(lst), " + fName2 + "(st), " + fName3 + "(mp) FROM %s WHERE key = 1"),
+                    row(list, set, map));
+ 
 -        for (int version = Server.VERSION_2; version <= maxProtocolVersion; version++)
++        for (int version : PROTOCOL_VERSIONS)
+             assertRowsNet(version,
+                           executeNet(version, "SELECT " + fName1 + "(lst), " + fName2 + "(st), " + fName3 + "(mp) FROM %s WHERE key = 1"),
+                           row(list, set, map));
+     }
+ 
+     @Test
+     public void testJavascriptTupleType() throws Throwable
+     {
+         createTable("CREATE TABLE %s (key int primary key, tup frozen<tuple<double, text, int, boolean>>)");
+ 
+         String fName = createFunction(KEYSPACE_PER_TEST, "tuple<double, text, int, boolean>",
+                                       "CREATE FUNCTION %s( tup tuple<double, text, int, boolean> ) " +
+                                       "RETURNS NULL ON NULL INPUT " +
+                                       "RETURNS tuple<double, text, int, boolean> " +
+                                       "LANGUAGE javascript\n" +
+                                       "AS $$tup;$$;");
+ 
+         Object t = tuple(1d, "foo", 2, true);
+ 
+         execute("INSERT INTO %s (key, tup) VALUES (1, ?)", t);
+ 
+         assertRows(execute("SELECT tup FROM %s WHERE key = 1"),
+                    row(t));
+ 
+         assertRows(execute("SELECT " + fName + "(tup) FROM %s WHERE key = 1"),
+                    row(t));
+     }
+ 
+     @Test
 -    public void testJavascriptTupleTypeCollection() throws Throwable
 -    {
 -        String tupleTypeDef = "tuple<double, list<double>, set<text>, map<int, boolean>>";
 -        createTable("CREATE TABLE %s (key int primary key, tup frozen<" + tupleTypeDef + ">)");
 -
 -        String fTup1 = createFunction(KEYSPACE_PER_TEST, tupleTypeDef,
 -                                      "CREATE FUNCTION %s( tup " + tupleTypeDef + " ) " +
 -                                      "RETURNS NULL ON NULL INPUT " +
 -                                      "RETURNS tuple<double, list<double>, set<text>, map<int, boolean>> " +
 -                                      "LANGUAGE javascript\n" +
 -                                      "AS $$" +
 -                                      "       tup;$$;");
 -        String fTup2 = createFunction(KEYSPACE_PER_TEST, tupleTypeDef,
 -                                      "CREATE FUNCTION %s( tup " + tupleTypeDef + " ) " +
 -                                      "RETURNS NULL ON NULL INPUT " +
 -                                      "RETURNS double " +
 -                                      "LANGUAGE javascript\n" +
 -                                      "AS $$" +
 -                                      "       tup.getDouble(0);$$;");
 -        String fTup3 = createFunction(KEYSPACE_PER_TEST, tupleTypeDef,
 -                                      "CREATE FUNCTION %s( tup " + tupleTypeDef + " ) " +
 -                                      "RETURNS NULL ON NULL INPUT " +
 -                                      "RETURNS list<double> " +
 -                                      "LANGUAGE javascript\n" +
 -                                      "AS $$" +
 -                                      "       tup.getList(1, java.lang.Class.forName(\"java.lang.Double\"));$$;");
 -        String fTup4 = createFunction(KEYSPACE_PER_TEST, tupleTypeDef,
 -                                      "CREATE FUNCTION %s( tup " + tupleTypeDef + " ) " +
 -                                      "RETURNS NULL ON NULL INPUT " +
 -                                      "RETURNS set<text> " +
 -                                      "LANGUAGE javascript\n" +
 -                                      "AS $$" +
 -                                      "       tup.getSet(2, java.lang.Class.forName(\"java.lang.String\"));$$;");
 -        String fTup5 = createFunction(KEYSPACE_PER_TEST, tupleTypeDef,
 -                                      "CREATE FUNCTION %s( tup " + tupleTypeDef + " ) " +
 -                                      "RETURNS NULL ON NULL INPUT " +
 -                                      "RETURNS map<int, boolean> " +
 -                                      "LANGUAGE javascript\n" +
 -                                      "AS $$" +
 -                                      "       tup.getMap(3, java.lang.Class.forName(\"java.lang.Integer\"), java.lang.Class.forName(\"java.lang.Boolean\"));$$;");
 -
 -        List<Double> list = Arrays.asList(1d, 2d, 3d);
 -        Set<String> set = new TreeSet<>(Arrays.asList("one", "three", "two"));
 -        Map<Integer, Boolean> map = new TreeMap<>();
 -        map.put(1, true);
 -        map.put(2, false);
 -        map.put(3, true);
 -
 -        Object t = tuple(1d, list, set, map);
 -
 -        execute("INSERT INTO %s (key, tup) VALUES (1, ?)", t);
 -
 -        assertRows(execute("SELECT " + fTup1 + "(tup) FROM %s WHERE key = 1"),
 -                   row(t));
 -        assertRows(execute("SELECT " + fTup2 + "(tup) FROM %s WHERE key = 1"),
 -                   row(1d));
 -        assertRows(execute("SELECT " + fTup3 + "(tup) FROM %s WHERE key = 1"),
 -                   row(list));
 -        assertRows(execute("SELECT " + fTup4 + "(tup) FROM %s WHERE key = 1"),
 -                   row(set));
 -        assertRows(execute("SELECT " + fTup5 + "(tup) FROM %s WHERE key = 1"),
 -                   row(map));
 -
 -        // same test - but via native protocol
 -        TupleType tType = TupleType.of(DataType.cdouble(),
 -                                       DataType.list(DataType.cdouble()),
 -                                       DataType.set(DataType.text()),
 -                                       DataType.map(DataType.cint(), DataType.cboolean()));
 -        TupleValue tup = tType.newValue(1d, list, set, map);
 -        for (int version = Server.VERSION_2; version <= maxProtocolVersion; version++)
 -        {
 -            assertRowsNet(version,
 -                          executeNet(version, "SELECT " + fTup1 + "(tup) FROM %s WHERE key = 1"),
 -                          row(tup));
 -            assertRowsNet(version,
 -                          executeNet(version, "SELECT " + fTup2 + "(tup) FROM %s WHERE key = 1"),
 -                          row(1d));
 -            assertRowsNet(version,
 -                          executeNet(version, "SELECT " + fTup3 + "(tup) FROM %s WHERE key = 1"),
 -                          row(list));
 -            assertRowsNet(version,
 -                          executeNet(version, "SELECT " + fTup4 + "(tup) FROM %s WHERE key = 1"),
 -                          row(set));
 -            assertRowsNet(version,
 -                          executeNet(version, "SELECT " + fTup5 + "(tup) FROM %s WHERE key = 1"),
 -                          row(map));
 -        }
 -    }
 -
 -    @Test
+     public void testJavascriptUserType() throws Throwable
+     {
+         String type = createType("CREATE TYPE %s (txt text, i int)");
+ 
+         createTable("CREATE TABLE %s (key int primary key, udt frozen<" + type + ">)");
+ 
+         String fUdt1 = createFunction(KEYSPACE, type,
+                                       "CREATE FUNCTION %s( udt " + type + " ) " +
+                                       "RETURNS NULL ON NULL INPUT " +
 -                                      "RETURNS " + type + " " +
++                                      "RETURNS " + type + ' ' +
+                                       "LANGUAGE javascript\n" +
+                                       "AS $$" +
+                                       "     udt;$$;");
+         String fUdt2 = createFunction(KEYSPACE, type,
+                                       "CREATE FUNCTION %s( udt " + type + " ) " +
+                                       "RETURNS NULL ON NULL INPUT " +
+                                       "RETURNS text " +
+                                       "LANGUAGE javascript\n" +
+                                       "AS $$" +
+                                       "     udt.getString(\"txt\");$$;");
+         String fUdt3 = createFunction(KEYSPACE, type,
+                                       "CREATE FUNCTION %s( udt " + type + " ) " +
+                                       "RETURNS NULL ON NULL INPUT " +
+                                       "RETURNS int " +
+                                       "LANGUAGE javascript\n" +
+                                       "AS $$" +
+                                       "     udt.getInt(\"i\");$$;");
+ 
+         execute("INSERT INTO %s (key, udt) VALUES (1, {txt: 'one', i:1})");
+ 
+         UntypedResultSet rows = execute("SELECT " + fUdt1 + "(udt) FROM %s WHERE key = 1");
+         Assert.assertEquals(1, rows.size());
+         assertRows(execute("SELECT " + fUdt2 + "(udt) FROM %s WHERE key = 1"),
+                    row("one"));
+         assertRows(execute("SELECT " + fUdt3 + "(udt) FROM %s WHERE key = 1"),
+                    row(1));
+     }
+ 
+     @Test
+     public void testJavascriptUTCollections() throws Throwable
+     {
+         String type = createType("CREATE TYPE %s (txt text, i int)");
+ 
+         createTable(String.format("CREATE TABLE %%s " +
+                                   "(key int primary key, lst list<frozen<%s>>, st set<frozen<%s>>, mp map<int, frozen<%s>>)",
+                                   type, type, type));
+ 
+         String fName = createFunction(KEYSPACE, "list<frozen<" + type + ">>",
+                                       "CREATE FUNCTION %s( lst list<frozen<" + type + ">> ) " +
+                                       "RETURNS NULL ON NULL INPUT " +
+                                       "RETURNS text " +
+                                       "LANGUAGE javascript\n" +
+                                       "AS $$" +
+                                       "        lst.get(1).getString(\"txt\");$$;");
+         createFunctionOverload(fName, "set<frozen<" + type + ">>",
+                                "CREATE FUNCTION %s( st set<frozen<" + type + ">> ) " +
+                                "RETURNS NULL ON NULL INPUT " +
+                                "RETURNS text " +
+                                "LANGUAGE javascript\n" +
+                                "AS $$" +
+                                "        st.iterator().next().getString(\"txt\");$$;");
+         createFunctionOverload(fName, "map<int, frozen<" + type + ">>",
+                                "CREATE FUNCTION %s( mp map<int, frozen<" + type + ">> ) " +
+                                "RETURNS NULL ON NULL INPUT " +
+                                "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>>
+                 "[ {txt: 'one', i:1}, {txt: 'three', i:1}, {txt: 'one', i:1} ] , " +
+                 // set<frozen<UDT>>
+                 "{ {txt: 'one', i:1}, {txt: 'three', i:3}, {txt: 'two', i:2} }, " +
+                 // map<int, frozen<UDT>>
+                 "{ 1: {txt: 'one', i:1}, 2: {txt: 'one', i:3}, 3: {txt: 'two', i:2} })");
+ 
+         assertRows(execute("SELECT " + fName + "(lst) FROM %s WHERE key = 1"),
+                    row("three"));
+         assertRows(execute("SELECT " + fName + "(st) FROM %s WHERE key = 1"),
+                    row("one"));
+         assertRows(execute("SELECT " + fName + "(mp) FROM %s WHERE key = 1"),
+                    row("two"));
+ 
+         String cqlSelect = "SELECT " + fName + "(lst), " + fName + "(st), " + fName + "(mp) FROM %s WHERE key = 1";
+         assertRows(execute(cqlSelect),
+                    row("three", "one", "two"));
+ 
+         // same test - but via native protocol
 -        for (int version = Server.VERSION_2; version <= maxProtocolVersion; version++)
++        for (int version : PROTOCOL_VERSIONS)
+             assertRowsNet(version,
+                           executeNet(version, cqlSelect),
+                           row("three", "one", "two"));
+     }
+ 
+     @Test
+     public void testJavascriptFunction() throws Throwable
+     {
+         createTable("CREATE TABLE %s (key int primary key, val double)");
+ 
+         String functionBody = '\n' +
+                               "  Math.sin(val);\n";
+ 
+         String fName = createFunction(KEYSPACE, "double",
+                                       "CREATE OR REPLACE FUNCTION %s(val double) " +
+                                       "RETURNS NULL ON NULL INPUT " +
+                                       "RETURNS double " +
+                                       "LANGUAGE javascript\n" +
+                                       "AS '" + functionBody + "';");
+ 
+         FunctionName fNameName = parseFunctionName(fName);
+ 
 -        assertRows(execute("SELECT language, body FROM system.schema_functions WHERE keyspace_name=? AND function_name=?",
++        assertRows(execute("SELECT language, body FROM system_schema.functions WHERE keyspace_name=? AND function_name=?",
+                            fNameName.keyspace, fNameName.name),
+                    row("javascript", functionBody));
+ 
+         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);
+         assertRows(execute("SELECT key, val, " + fName + "(val) FROM %s"),
+                    row(1, 1d, Math.sin(1d)),
+                    row(2, 2d, Math.sin(2d)),
+                    row(3, 3d, Math.sin(3d))
+         );
+     }
+ 
+     @Test
+     public void testJavascriptBadReturnType() throws Throwable
+     {
+         createTable("CREATE TABLE %s (key int primary key, val double)");
+ 
+         String fName = createFunction(KEYSPACE, "double",
+                                       "CREATE OR REPLACE FUNCTION %s(val double) " +
+                                       "RETURNS NULL ON NULL INPUT " +
+                                       "RETURNS double " +
+                                       "LANGUAGE javascript\n" +
+                                       "AS '\"string\";';");
+ 
+         execute("INSERT INTO %s (key, val) VALUES (?, ?)", 1, 1d);
+         // throws IRE with ClassCastException
+         assertInvalidMessage("Invalid value for CQL type double", "SELECT key, val, " + fName + "(val) FROM %s");
+     }
+ 
+     @Test
+     public void testJavascriptThrow() throws Throwable
+     {
+         createTable("CREATE TABLE %s (key int primary key, val double)");
+ 
+         String fName = createFunction(KEYSPACE, "double",
+                                       "CREATE OR REPLACE FUNCTION %s(val double) " +
+                                       "RETURNS NULL ON NULL INPUT " +
+                                       "RETURNS double " +
+                                       "LANGUAGE javascript\n" +
+                                       "AS 'throw \"fool\";';");
+ 
+         execute("INSERT INTO %s (key, val) VALUES (?, ?)", 1, 1d);
+         // throws IRE with ScriptException
+         assertInvalidThrowMessage("fool", FunctionExecutionException.class,
+                                   "SELECT key, val, " + fName + "(val) FROM %s");
+     }
+ 
+     @Test
 -    public void testJavascriptCompileFailure() throws Throwable
 -    {
 -        assertInvalidMessage("Failed to compile function 'cql_test_keyspace.scrinv'",
 -                             "CREATE OR REPLACE FUNCTION " + KEYSPACE + ".scrinv(val double) " +
 -                             "RETURNS NULL ON NULL INPUT " +
 -                             "RETURNS double " +
 -                             "LANGUAGE javascript\n" +
 -                             "AS 'foo bar';");
 -    }
 -
 -    @Test
 -    public void testScriptInvalidLanguage() throws Throwable
 -    {
 -        assertInvalidMessage("Invalid language 'artificial_intelligence' for function 'cql_test_keyspace.scrinv'",
 -                             "CREATE OR REPLACE FUNCTION " + KEYSPACE + ".scrinv(val double) " +
 -                             "RETURNS NULL ON NULL INPUT " +
 -                             "RETURNS double " +
 -                             "LANGUAGE artificial_intelligence\n" +
 -                             "AS 'question for 42?';");
 -    }
 -
 -    @Test
+     public void testScriptReturnTypeCasting() throws Throwable
+     {
+         createTable("CREATE TABLE %s (key int primary key, val double)");
+         execute("INSERT INTO %s (key, val) VALUES (?, ?)", 1, 1d);
+ 
+         Object[][] variations = {
+         new Object[]    {   "true",     "boolean",  true    },
+         new Object[]    {   "false",    "boolean",  false   },
+         new Object[]    {   "100",      "tinyint",  (byte)100 },
+         new Object[]    {   "100.",     "tinyint",  (byte)100 },
+         new Object[]    {   "100",      "smallint", (short)100 },
+         new Object[]    {   "100.",     "smallint", (short)100 },
+         new Object[]    {   "100",      "int",      100     },
+         new Object[]    {   "100.",     "int",      100     },
+         new Object[]    {   "100",      "double",   100d    },
+         new Object[]    {   "100.",     "double",   100d    },
+         new Object[]    {   "100",      "bigint",   100L    },
+         new Object[]    {   "100.",     "bigint",   100L    },
+         new Object[]    { "100", "varint", BigInteger.valueOf(100L)    },
+         new Object[]    {   "100.",     "varint",   BigInteger.valueOf(100L)    },
+         new Object[]    { "parseInt(\"100\");", "decimal", BigDecimal.valueOf(100d)    },
+         new Object[]    {   "100.",     "decimal",  BigDecimal.valueOf(100d)    },
+         };
+ 
+         for (Object[] variation : variations)
+         {
+             Object functionBody = variation[0];
+             Object returnType = variation[1];
+             Object expectedResult = variation[2];
+ 
+             String fName = createFunction(KEYSPACE, "double",
+                                           "CREATE OR REPLACE FUNCTION %s(val double) " +
+                                           "RETURNS NULL ON NULL INPUT " +
+                                           "RETURNS " +returnType + ' ' +
+                                           "LANGUAGE javascript " +
+                                           "AS '" + functionBody + ";';");
+             assertRows(execute("SELECT key, val, " + fName + "(val) FROM %s"),
+                        row(1, 1d, expectedResult));
+         }
+     }
+ 
+     @Test
+     public void testScriptParamReturnTypes() throws Throwable
+     {
+         UUID ruuid = UUID.randomUUID();
+         UUID tuuid = UUIDGen.getTimeUUID();
+ 
+         createTable("CREATE TABLE %s (key int primary key, " +
+                     "tival tinyint, sival smallint, ival int, lval bigint, fval float, dval double, vval varint, ddval decimal, " +
+                     "timval time, dtval date, tsval timestamp, uval uuid, tuval timeuuid)");
+         execute("INSERT INTO %s (key, tival, sival, ival, lval, fval, dval, vval, ddval, timval, dtval, tsval, uval, tuval) VALUES " +
+                 "(?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?)", 1,
+                 (byte)1, (short)1, 1, 1L, 1f, 1d, BigInteger.valueOf(1L), BigDecimal.valueOf(1d), 1L, Integer.MAX_VALUE, new Date(1), ruuid, tuuid);
+ 
+         Object[][] variations = {
+         new Object[] {  "tinyint",  "tival",    (byte)1,                (byte)2  },
+         new Object[] {  "smallint", "sival",    (short)1,               (short)2  },
+         new Object[] {  "int",      "ival",     1,                      2  },
+         new Object[] {  "bigint",   "lval",     1L,                     2L  },
+         new Object[] {  "float",    "fval",     1f,                     2f  },
+         new Object[] {  "double",   "dval",     1d,                     2d  },
+         new Object[] {  "varint",   "vval",     BigInteger.valueOf(1L), BigInteger.valueOf(2L)  },
+         new Object[] {  "decimal",  "ddval",    BigDecimal.valueOf(1d), BigDecimal.valueOf(2d)  },
+         new Object[] {  "time",     "timval",   1L,                     2L  },
+         };
+ 
+         for (Object[] variation : variations)
+         {
+             Object type = variation[0];
+             Object col = variation[1];
+             Object expected1 = variation[2];
+             Object expected2 = variation[3];
+             String fName = createFunction(KEYSPACE, type.toString(),
+                                           "CREATE OR REPLACE FUNCTION %s(val " + type + ") " +
+                                           "RETURNS NULL ON NULL INPUT " +
+                                           "RETURNS " + type + ' ' +
+                                           "LANGUAGE javascript " +
+                                           "AS 'val+1;';");
+             assertRows(execute("SELECT key, " + col + ", " + fName + '(' + col + ") FROM %s"),
+                        row(1, expected1, expected2));
+         }
+ 
+         variations = new Object[][] {
+         new Object[] {  "timestamp","tsval",    new Date(1),            new Date(1)  },
+         new Object[] {  "uuid",     "uval",     ruuid,                  ruuid  },
+         new Object[] {  "timeuuid", "tuval",    tuuid,                  tuuid  },
+         new Object[] {  "date",     "dtval",    Integer.MAX_VALUE,      Integer.MAX_VALUE },
+         };
+ 
+         for (Object[] variation : variations)
+         {
+             Object type = variation[0];
+             Object col = variation[1];
+             Object expected1 = variation[2];
+             Object expected2 = variation[3];
+             String fName = createFunction(KEYSPACE, type.toString(),
+                                           "CREATE OR REPLACE FUNCTION %s(val " + type + ") " +
+                                           "RETURNS NULL ON NULL INPUT " +
+                                           "RETURNS " + type + ' ' +
+                                           "LANGUAGE javascript " +
+                                           "AS 'val;';");
+             assertRows(execute("SELECT key, " + col + ", " + fName + '(' + col + ") FROM %s"),
+                        row(1, expected1, expected2));
+         }
+     }
++
++    @Test
++    public void testJavascriptDisabled() throws Throwable
++    {
++        createTable("CREATE TABLE %s (key int primary key, val double)");
++
++        DatabaseDescriptor.enableScriptedUserDefinedFunctions(false);
++        try
++        {
++            assertInvalid("double",
++                          "CREATE OR REPLACE FUNCTION " + KEYSPACE + ".assertNotEnabled(val double) " +
++                          "RETURNS NULL ON NULL INPUT " +
++                          "RETURNS double " +
++                          "LANGUAGE javascript\n" +
++                          "AS 'Math.sin(val);';");
++        }
++        finally
++        {
++            DatabaseDescriptor.enableScriptedUserDefinedFunctions(true);
++        }
++    }
++
++    @Test
++    public void testJavascriptCompileFailure() throws Throwable
++    {
++        assertInvalidMessage("Failed to compile function 'cql_test_keyspace.scrinv'",
++                             "CREATE OR REPLACE FUNCTION " + KEYSPACE + ".scrinv(val double) " +
++                             "RETURNS NULL ON NULL INPUT " +
++                             "RETURNS double " +
++                             "LANGUAGE javascript\n" +
++                             "AS 'foo bar';");
++    }
++
++    @Test
++    public void testScriptInvalidLanguage() throws Throwable
++    {
++        assertInvalidMessage("Invalid language 'artificial_intelligence' for function 'cql_test_keyspace.scrinv'",
++                             "CREATE OR REPLACE FUNCTION " + KEYSPACE + ".scrinv(val double) " +
++                             "RETURNS NULL ON NULL INPUT " +
++                             "RETURNS double " +
++                             "LANGUAGE artificial_intelligence\n" +
++                             "AS 'question for 42?';");
++    }
+ }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3aefe356/test/unit/org/apache/cassandra/cql3/validation/entities/UFSecurityTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/cql3/validation/entities/UFSecurityTest.java
index 0000000,0000000..5c7ca2b
new file mode 100644
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/UFSecurityTest.java
@@@ -1,0 -1,0 +1,258 @@@
++/*
++ * 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.cql3.validation.entities;
++
++import java.security.AccessControlException;
++import java.util.List;
++
++import org.junit.Assert;
++import org.junit.Test;
++
++import org.apache.cassandra.config.Config;
++import org.apache.cassandra.config.DatabaseDescriptor;
++import org.apache.cassandra.cql3.CQLTester;
++import org.apache.cassandra.cql3.functions.UDHelper;
++import org.apache.cassandra.exceptions.FunctionExecutionException;
++import org.apache.cassandra.service.ClientWarn;
++
++public class UFSecurityTest extends CQLTester
++{
++    @Test
++    public void testSecurityPermissions() throws Throwable
++    {
++        createTable("CREATE TABLE %s (key int primary key, dval double)");
++        execute("INSERT INTO %s (key, dval) VALUES (?, ?)", 1, 1d);
++
++        // Java UDFs
++
++        try
++        {
++            String fName = createFunction(KEYSPACE_PER_TEST, "double",
++                                          "CREATE OR REPLACE FUNCTION %s(val double) " +
++                                          "RETURNS NULL ON NULL INPUT " +
++                                          "RETURNS double " +
++                                          "LANGUAGE JAVA\n" +
++                                          "AS 'System.getProperty(\"foo.bar.baz\"); return 0d;';");
++            execute("SELECT " + fName + "(dval) FROM %s WHERE key=1");
++            Assert.fail();
++        }
++        catch (FunctionExecutionException e)
++        {
++            assertAccessControlException("System.getProperty(\"foo.bar.baz\"); return 0d;", e);
++        }
++
++        String[][] typesAndSources =
++        {
++        {"", "try { Class.forName(\"" + UDHelper.class.getName() + "\"); } catch (Exception e) { throw new RuntimeException(e); } return 0d;"},
++        {"sun.misc.Unsafe",         "sun.misc.Unsafe.getUnsafe(); return 0d;"},
++        {"",                        "try { Class.forName(\"sun.misc.Unsafe\"); } catch (Exception e) { throw new RuntimeException(e); } return 0d;"},
++        {"java.nio.file.FileSystems", "try {" +
++                                      "     java.nio.file.FileSystems.getDefault(); return 0d;" +
++                                      "} catch (Exception t) {" +
++                                      "     throw new RuntimeException(t);" +
++                                      '}'},
++        {"java.nio.channels.FileChannel", "try {" +
++                                          "     java.nio.channels.FileChannel.open(java.nio.file.FileSystems.getDefault().getPath(\"/etc/passwd\")).close(); return 0d;" +
++                                          "} catch (Exception t) {" +
++                                          "     throw new RuntimeException(t);" +
++                                          '}'},
++        {"java.nio.channels.SocketChannel", "try {" +
++                                            "     java.nio.channels.SocketChannel.open().close(); return 0d;" +
++                                            "} catch (Exception t) {" +
++                                            "     throw new RuntimeException(t);" +
++                                            '}'},
++        {"java.io.FileInputStream", "try {" +
++                                    "     new java.io.FileInputStream(\"./foobar\").close(); return 0d;" +
++                                    "} catch (Exception t) {" +
++                                    "     throw new RuntimeException(t);" +
++                                    '}'},
++        {"java.lang.Runtime",       "try {" +
++                                    "     java.lang.Runtime.getRuntime(); return 0d;" +
++                                    "} catch (Exception t) {" +
++                                    "     throw new RuntimeException(t);" +
++                                    '}'},
++        {"org.apache.cassandra.service.StorageService",
++         "try {" +
++         "     org.apache.cassandra.service.StorageService v = org.apache.cassandra.service.StorageService.instance; v.isShutdown(); return 0d;" +
++         "} catch (Exception t) {" +
++         "     throw new RuntimeException(t);" +
++         '}'},
++        {"java.net.ServerSocket",   "try {" +
++                                    "     new java.net.ServerSocket().bind(); return 0d;" +
++                                    "} catch (Exception t) {" +
++                                    "     throw new RuntimeException(t);" +
++                                    '}'},
++        {"java.io.FileOutputStream","try {" +
++                                    "     new java.io.FileOutputStream(\".foo\"); return 0d;" +
++                                    "} catch (Exception t) {" +
++                                    "     throw new RuntimeException(t);" +
++                                    '}'},
++        {"java.lang.Runtime",       "try {" +
++                                    "     java.lang.Runtime.getRuntime().exec(\"/tmp/foo\"); return 0d;" +
++                                    "} catch (Exception t) {" +
++                                    "     throw new RuntimeException(t);" +
++                                    '}'}
++        };
++
++        for (String[] typeAndSource : typesAndSources)
++        {
++            assertInvalidMessage(typeAndSource[0] + " cannot be resolved",
++                                 "CREATE OR REPLACE FUNCTION " + KEYSPACE + ".invalid_class_access(val double) " +
++                                 "RETURNS NULL ON NULL INPUT " +
++                                 "RETURNS double " +
++                                 "LANGUAGE JAVA\n" +
++                                 "AS '" + typeAndSource[1] + "';");
++        }
++
++        // JavaScript UDFs
++
++        try
++        {
++            String fName = createFunction(KEYSPACE_PER_TEST, "double",
++                                          "CREATE OR REPLACE FUNCTION %s(val double) " +
++                                          "RETURNS NULL ON NULL INPUT " +
++                                          "RETURNS double " +
++                                          "LANGUAGE javascript\n" +
++                                          "AS 'org.apache.cassandra.service.StorageService.instance.isShutdown(); 0;';");
++            execute("SELECT " + fName + "(dval) FROM %s WHERE key=1");
++            Assert.fail("Javascript security check failed");
++        }
++        catch (FunctionExecutionException e)
++        {
++            assertAccessControlException("", e);
++        }
++
++        String[] javascript =
++        {
++        "java.lang.management.ManagmentFactory.getThreadMXBean(); 0;",
++        "new java.io.FileInputStream(\"/tmp/foo\"); 0;",
++        "new java.io.FileOutputStream(\"/tmp/foo\"); 0;",
++        "java.nio.file.FileSystems.getDefault().createFileExclusively(\"./foo_bar_baz\"); 0;",
++        "java.nio.channels.FileChannel.open(java.nio.file.FileSystems.getDefault().getPath(\"/etc/passwd\")); 0;",
++        "java.nio.channels.SocketChannel.open(); 0;",
++        "new java.net.ServerSocket().bind(null); 0;",
++        "var thread = new java.lang.Thread(); thread.start(); 0;",
++        "java.lang.System.getProperty(\"foo.bar.baz\"); 0;",
++        "java.lang.Class.forName(\"java.lang.System\"); 0;",
++        "java.lang.Runtime.getRuntime().exec(\"/tmp/foo\"); 0;",
++        "java.lang.Runtime.getRuntime().loadLibrary(\"foobar\"); 0;",
++        "java.lang.Runtime.getRuntime().loadLibrary(\"foobar\"); 0;",
++        // TODO these (ugly) calls are still possible - these can consume CPU (as one could do with an evil loop, too)
++//        "java.lang.Runtime.getRuntime().traceMethodCalls(true); 0;",
++//        "java.lang.Runtime.getRuntime().gc(); 0;",
++//        "java.lang.Runtime.getRuntime(); 0;",
++        };
++
++        for (String script : javascript)
++        {
++            try
++            {
++                String fName = createFunction(KEYSPACE_PER_TEST, "double",
++                                              "CREATE OR REPLACE FUNCTION %s(val double) " +
++                                              "RETURNS NULL ON NULL INPUT " +
++                                              "RETURNS double " +
++                                              "LANGUAGE javascript\n" +
++                                              "AS '" + script + "';");
++                execute("SELECT " + fName + "(dval) FROM %s WHERE key=1");
++                Assert.fail("Javascript security check failed: " + script);
++            }
++            catch (FunctionExecutionException e)
++            {
++                assertAccessControlException(script, e);
++            }
++        }
++    }
++
++    private static void assertAccessControlException(String script, FunctionExecutionException e)
++    {
++        for (Throwable t = e; t != null && t != t.getCause(); t = t.getCause())
++            if (t instanceof AccessControlException)
++                return;
++        Assert.fail("no AccessControlException for " + script + " (got " + e + ')');
++    }
++
++    @Test
++    public void testAmokUDF() throws Throwable
++    {
++        createTable("CREATE TABLE %s (key int primary key, dval double)");
++        execute("INSERT INTO %s (key, dval) VALUES (?, ?)", 1, 1d);
++
++        long udfWarnTimeout = DatabaseDescriptor.getUserDefinedFunctionWarnTimeout();
++        long udfFailTimeout = DatabaseDescriptor.getUserDefinedFunctionFailTimeout();
++        int maxTries = 5;
++        for (int i = 1; i <= maxTries; i++)
++        {
++            try
++            {
++                // short timeout
++                DatabaseDescriptor.setUserDefinedFunctionWarnTimeout(10);
++                DatabaseDescriptor.setUserDefinedFunctionFailTimeout(250);
++                // don't kill the unit test... - default policy is "die"
++                DatabaseDescriptor.setUserFunctionTimeoutPolicy(Config.UserFunctionTimeoutPolicy.ignore);
++
++                ClientWarn.instance.captureWarnings();
++                String fName = createFunction(KEYSPACE_PER_TEST, "double",
++                                              "CREATE OR REPLACE FUNCTION %s(val double) " +
++                                              "RETURNS NULL ON NULL INPUT " +
++                                              "RETURNS double " +
++                                              "LANGUAGE JAVA\n" +
++                                              "AS 'long t=System.currentTimeMillis()+110; while (t>System.currentTimeMillis()) { }; return 0d;'");
++                execute("SELECT " + fName + "(dval) FROM %s WHERE key=1");
++                List<String> warnings = ClientWarn.instance.getWarnings();
++                Assert.assertNotNull(warnings);
++                Assert.assertFalse(warnings.isEmpty());
++                ClientWarn.instance.resetWarnings();
++
++                // Java UDF
++
++                fName = createFunction(KEYSPACE_PER_TEST, "double",
++                                       "CREATE OR REPLACE FUNCTION %s(val double) " +
++                                       "RETURNS NULL ON NULL INPUT " +
++                                       "RETURNS double " +
++                                       "LANGUAGE JAVA\n" +
++                                       "AS 'long t=System.currentTimeMillis()+500; while (t>System.currentTimeMillis()) { }; return 0d;';");
++                assertInvalidMessage("ran longer than 250ms", "SELECT " + fName + "(dval) FROM %s WHERE key=1");
++
++                // Javascript UDF
++
++                fName = createFunction(KEYSPACE_PER_TEST, "double",
++                                       "CREATE OR REPLACE FUNCTION %s(val double) " +
++                                       "RETURNS NULL ON NULL INPUT " +
++                                       "RETURNS double " +
++                                       "LANGUAGE JAVASCRIPT\n" +
++                                       "AS 'var t=java.lang.System.currentTimeMillis()+500; while (t>java.lang.System.currentTimeMillis()) { }; 0;';");
++                assertInvalidMessage("ran longer than 250ms", "SELECT " + fName + "(dval) FROM %s WHERE key=1");
++
++                return;
++            }
++            catch (Error | RuntimeException e)
++            {
++                if (i == maxTries)
++                    throw e;
++            }
++            finally
++            {
++                // reset to defaults
++                DatabaseDescriptor.setUserDefinedFunctionWarnTimeout(udfWarnTimeout);
++                DatabaseDescriptor.setUserDefinedFunctionFailTimeout(udfFailTimeout);
++            }
++        }
++    }
++
++}