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/11/23 16:53:49 UTC

[2/2] cassandra git commit: Unable to create a function with argument of type Inet

Unable to create a function with argument of type Inet

patch by Robert Stupp; reviewed by T Jake Luciani for CASSANDRA-10741

Merge branch 'cassandra-2.2' into cassandra-3.0


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

Branch: refs/heads/cassandra-3.0
Commit: 174a0aa7d1f643cd27f15c8255627fdfdca1ae22
Parents: 6ac831f d425fe4
Author: Robert Stupp <sn...@snazy.de>
Authored: Mon Nov 23 16:47:52 2015 +0100
Committer: Robert Stupp <sn...@snazy.de>
Committed: Mon Nov 23 16:47:52 2015 +0100

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../cql3/functions/JavaBasedUDFunction.java     | 15 +++++
 .../cql3/functions/UDFByteCodeVerifier.java     | 21 ++++++-
 .../cassandra/cql3/functions/UDFunction.java    |  6 ++
 .../cql3/validation/entities/UFTest.java        | 42 ++++++++++++-
 .../validation/entities/UFVerifierTest.java     | 62 +++++++++++++++-----
 6 files changed, 128 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/174a0aa7/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index e6f5406,146a0ce..4182cc1
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,11 -1,4 +1,12 @@@
 -2.2.4
 +3.0.1
++ * Unable to create a function with argument of type Inet (CASSANDRA-10741)
 + * Fix backward incompatibiliy in CqlInputFormat (CASSANDRA-10717)
 + * Correctly preserve deletion info on updated rows when notifying indexers
 +   of single-row deletions (CASSANDRA-10694)
 + * Notify indexers of partition delete during cleanup (CASSANDRA-10685)
 + * Keep the file open in trySkipCache (CASSANDRA-10669)
 + * Updated trigger example (CASSANDRA-10257)
 +Merged from 2.2:
   * Don't do anticompaction after subrange repair (CASSANDRA-10422)
   * Fix SimpleDateType type compatibility (CASSANDRA-10027)
   * (Hadoop) fix splits calculation (CASSANDRA-10640)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/174a0aa7/src/java/org/apache/cassandra/cql3/functions/JavaBasedUDFunction.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/functions/JavaBasedUDFunction.java
index 0c1ee64,0000000..c61e72e
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/cql3/functions/JavaBasedUDFunction.java
+++ b/src/java/org/apache/cassandra/cql3/functions/JavaBasedUDFunction.java
@@@ -1,639 -1,0 +1,654 @@@
 +/*
 + * 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.functions;
 +
 +import java.io.ByteArrayOutputStream;
 +import java.io.IOException;
 +import java.io.InputStream;
 +import java.lang.invoke.MethodHandle;
 +import java.lang.invoke.MethodHandles;
 +import java.lang.invoke.MethodType;
 +import java.lang.reflect.InvocationTargetException;
 +import java.lang.reflect.Method;
 +import java.net.*;
 +import java.nio.ByteBuffer;
 +import java.security.*;
 +import java.security.cert.Certificate;
 +import java.util.*;
 +import java.util.concurrent.ConcurrentHashMap;
 +import java.util.concurrent.ExecutorService;
 +import java.util.concurrent.ThreadLocalRandom;
 +import java.util.concurrent.atomic.AtomicInteger;
 +
 +import com.google.common.io.ByteStreams;
 +import org.slf4j.Logger;
 +import org.slf4j.LoggerFactory;
 +
 +import com.datastax.driver.core.DataType;
 +import org.apache.cassandra.concurrent.NamedThreadFactory;
 +import org.apache.cassandra.cql3.ColumnIdentifier;
 +import org.apache.cassandra.db.marshal.AbstractType;
 +import org.apache.cassandra.exceptions.InvalidRequestException;
 +import org.apache.cassandra.utils.FBUtilities;
 +import org.eclipse.jdt.core.compiler.IProblem;
 +import org.eclipse.jdt.internal.compiler.*;
 +import org.eclipse.jdt.internal.compiler.Compiler;
 +import org.eclipse.jdt.internal.compiler.classfmt.ClassFileReader;
 +import org.eclipse.jdt.internal.compiler.classfmt.ClassFormatException;
 +import org.eclipse.jdt.internal.compiler.env.ICompilationUnit;
 +import org.eclipse.jdt.internal.compiler.env.INameEnvironment;
 +import org.eclipse.jdt.internal.compiler.env.NameEnvironmentAnswer;
 +import org.eclipse.jdt.internal.compiler.impl.CompilerOptions;
 +import org.eclipse.jdt.internal.compiler.problem.DefaultProblemFactory;
 +
 +final class JavaBasedUDFunction extends UDFunction
 +{
 +    private static final String BASE_PACKAGE = "org.apache.cassandra.cql3.udf.gen";
 +
 +    static final Logger logger = LoggerFactory.getLogger(JavaBasedUDFunction.class);
 +
 +    private static final AtomicInteger classSequence = new AtomicInteger();
 +
 +    // use a JVM standard ExecutorService as DebuggableThreadPoolExecutor references internal
 +    // classes, which triggers AccessControlException from the UDF sandbox
 +    private static final UDFExecutorService executor =
 +        new UDFExecutorService(new NamedThreadFactory("UserDefinedFunctions",
 +                                                      Thread.MIN_PRIORITY,
 +                                                      udfClassLoader,
 +                                                      new SecurityThreadGroup("UserDefinedFunctions", null, UDFunction::initializeThread)),
 +                               "userfunction");
 +
 +    private static final EcjTargetClassLoader targetClassLoader = new EcjTargetClassLoader();
 +
 +    private static final UDFByteCodeVerifier udfByteCodeVerifier = new UDFByteCodeVerifier();
 +
 +    private static final ProtectionDomain protectionDomain;
 +
 +    private static final IErrorHandlingPolicy errorHandlingPolicy = DefaultErrorHandlingPolicies.proceedWithAllProblems();
 +    private static final IProblemFactory problemFactory = new DefaultProblemFactory(Locale.ENGLISH);
 +    private static final CompilerOptions compilerOptions;
 +
 +    /**
 +     * Poor man's template - just a text file splitted at '#' chars.
 +     * Each string at an even index is a constant string (just copied),
 +     * each string at an odd index is an 'instruction'.
 +     */
 +    private static final String[] javaSourceTemplate;
 +
 +    static
 +    {
 +        udfByteCodeVerifier.addDisallowedMethodCall("java/lang/Class", "forName");
 +        udfByteCodeVerifier.addDisallowedMethodCall("java/lang/Class", "getClassLoader");
 +        udfByteCodeVerifier.addDisallowedMethodCall("java/lang/Class", "getResource");
 +        udfByteCodeVerifier.addDisallowedMethodCall("java/lang/Class", "getResourceAsStream");
 +        udfByteCodeVerifier.addDisallowedMethodCall("java/lang/ClassLoader", "clearAssertionStatus");
 +        udfByteCodeVerifier.addDisallowedMethodCall("java/lang/ClassLoader", "getResource");
 +        udfByteCodeVerifier.addDisallowedMethodCall("java/lang/ClassLoader", "getResourceAsStream");
 +        udfByteCodeVerifier.addDisallowedMethodCall("java/lang/ClassLoader", "getResources");
 +        udfByteCodeVerifier.addDisallowedMethodCall("java/lang/ClassLoader", "getSystemClassLoader");
 +        udfByteCodeVerifier.addDisallowedMethodCall("java/lang/ClassLoader", "getSystemResource");
 +        udfByteCodeVerifier.addDisallowedMethodCall("java/lang/ClassLoader", "getSystemResourceAsStream");
 +        udfByteCodeVerifier.addDisallowedMethodCall("java/lang/ClassLoader", "getSystemResources");
 +        udfByteCodeVerifier.addDisallowedMethodCall("java/lang/ClassLoader", "loadClass");
 +        udfByteCodeVerifier.addDisallowedMethodCall("java/lang/ClassLoader", "setClassAssertionStatus");
 +        udfByteCodeVerifier.addDisallowedMethodCall("java/lang/ClassLoader", "setDefaultAssertionStatus");
 +        udfByteCodeVerifier.addDisallowedMethodCall("java/lang/ClassLoader", "setPackageAssertionStatus");
 +        udfByteCodeVerifier.addDisallowedMethodCall("java/nio/ByteBuffer", "allocateDirect");
++        for (String ia : new String[]{"java/net/InetAddress", "java/net/Inet4Address", "java/net/Inet6Address"})
++        {
++            // static method, probably performing DNS lookups (despite SecurityManager)
++            udfByteCodeVerifier.addDisallowedMethodCall(ia, "getByAddress");
++            udfByteCodeVerifier.addDisallowedMethodCall(ia, "getAllByName");
++            udfByteCodeVerifier.addDisallowedMethodCall(ia, "getByName");
++            udfByteCodeVerifier.addDisallowedMethodCall(ia, "getLocalHost");
++            // instance methods, probably performing DNS lookups (despite SecurityManager)
++            udfByteCodeVerifier.addDisallowedMethodCall(ia, "getHostName");
++            udfByteCodeVerifier.addDisallowedMethodCall(ia, "getCanonicalHostName");
++            // ICMP PING
++            udfByteCodeVerifier.addDisallowedMethodCall(ia, "isReachable");
++        }
++        udfByteCodeVerifier.addDisallowedClass("java/net/NetworkInterface");
++        udfByteCodeVerifier.addDisallowedClass("java/net/SocketException");
 +
 +        Map<String, String> settings = new HashMap<>();
 +        settings.put(CompilerOptions.OPTION_LineNumberAttribute,
 +                     CompilerOptions.GENERATE);
 +        settings.put(CompilerOptions.OPTION_SourceFileAttribute,
 +                     CompilerOptions.DISABLED);
 +        settings.put(CompilerOptions.OPTION_ReportDeprecation,
 +                     CompilerOptions.IGNORE);
 +        settings.put(CompilerOptions.OPTION_Source,
 +                     CompilerOptions.VERSION_1_8);
 +        settings.put(CompilerOptions.OPTION_TargetPlatform,
 +                     CompilerOptions.VERSION_1_8);
 +
 +        compilerOptions = new CompilerOptions(settings);
 +        compilerOptions.parseLiteralExpressionsAsConstants = true;
 +
 +        try (InputStream input = JavaBasedUDFunction.class.getResource("JavaSourceUDF.txt").openConnection().getInputStream())
 +        {
 +            ByteArrayOutputStream output = new ByteArrayOutputStream();
 +            FBUtilities.copy(input, output, Long.MAX_VALUE);
 +            String template = output.toString();
 +
 +            StringTokenizer st = new StringTokenizer(template, "#");
 +            javaSourceTemplate = new String[st.countTokens()];
 +            for (int i = 0; st.hasMoreElements(); i++)
 +                javaSourceTemplate[i] = st.nextToken();
 +        }
 +        catch (IOException e)
 +        {
 +            throw new RuntimeException(e);
 +        }
 +
 +        CodeSource codeSource;
 +        try
 +        {
 +            codeSource = new CodeSource(new URL("udf", "localhost", 0, "/java", new URLStreamHandler()
 +            {
 +                protected URLConnection openConnection(URL u)
 +                {
 +                    return null;
 +                }
 +            }), (Certificate[])null);
 +        }
 +        catch (MalformedURLException e)
 +        {
 +            throw new RuntimeException(e);
 +        }
 +
 +        protectionDomain = new ProtectionDomain(codeSource, ThreadAwareSecurityManager.noPermissions, targetClassLoader, null);
 +    }
 +
 +    private final JavaUDF javaUDF;
 +
 +    JavaBasedUDFunction(FunctionName name, List<ColumnIdentifier> argNames, List<AbstractType<?>> argTypes,
 +                        AbstractType<?> returnType, boolean calledOnNullInput, String body)
 +    {
 +        super(name, argNames, argTypes, UDHelper.driverTypes(argTypes),
 +              returnType, UDHelper.driverType(returnType), calledOnNullInput, "java", body);
 +
 +        // javaParamTypes is just the Java representation for argTypes resp. argDataTypes
 +        Class<?>[] javaParamTypes = UDHelper.javaTypes(argDataTypes, calledOnNullInput);
 +        // javaReturnType is just the Java representation for returnType resp. returnDataType
 +        Class<?> javaReturnType = UDHelper.asJavaClass(returnDataType);
 +
 +        // put each UDF in a separate package to prevent cross-UDF code access
 +        String pkgName = BASE_PACKAGE + '.' + generateClassName(name, 'p');
 +        String clsName = generateClassName(name, 'C');
 +
 +        String executeInternalName = generateClassName(name, 'x');
 +
 +        StringBuilder javaSourceBuilder = new StringBuilder();
 +        int lineOffset = 1;
 +        for (int i = 0; i < javaSourceTemplate.length; i++)
 +        {
 +            String s = javaSourceTemplate[i];
 +
 +            // strings at odd indexes are 'instructions'
 +            if ((i & 1) == 1)
 +            {
 +                switch (s)
 +                {
 +                    case "package_name":
 +                        s = pkgName;
 +                        break;
 +                    case "class_name":
 +                        s = clsName;
 +                        break;
 +                    case "body":
 +                        lineOffset = countNewlines(javaSourceBuilder);
 +                        s = body;
 +                        break;
 +                    case "arguments":
 +                        s = generateArguments(javaParamTypes, argNames);
 +                        break;
 +                    case "argument_list":
 +                        s = generateArgumentList(javaParamTypes, argNames);
 +                        break;
 +                    case "return_type":
 +                        s = javaSourceName(javaReturnType);
 +                        break;
 +                    case "execute_internal_name":
 +                        s = executeInternalName;
 +                        break;
 +                }
 +            }
 +
 +            javaSourceBuilder.append(s);
 +        }
 +
 +        String targetClassName = pkgName + '.' + clsName;
 +
 +        String javaSource = javaSourceBuilder.toString();
 +
 +        logger.trace("Compiling Java source UDF '{}' as class '{}' using source:\n{}", name, targetClassName, javaSource);
 +
 +        try
 +        {
 +            EcjCompilationUnit compilationUnit = new EcjCompilationUnit(javaSource, targetClassName);
 +
 +            org.eclipse.jdt.internal.compiler.Compiler compiler = new Compiler(compilationUnit,
 +                                                                               errorHandlingPolicy,
 +                                                                               compilerOptions,
 +                                                                               compilationUnit,
 +                                                                               problemFactory);
 +            compiler.compile(new ICompilationUnit[]{ compilationUnit });
 +
 +            if (compilationUnit.problemList != null && !compilationUnit.problemList.isEmpty())
 +            {
 +                boolean fullSource = false;
 +                StringBuilder problems = new StringBuilder();
 +                for (IProblem problem : compilationUnit.problemList)
 +                {
 +                    long ln = problem.getSourceLineNumber() - lineOffset;
 +                    if (ln < 1L)
 +                    {
 +                        if (problem.isError())
 +                        {
 +                            // if generated source around UDF source provided by the user is buggy,
 +                            // this code is appended.
 +                            problems.append("GENERATED SOURCE ERROR: line ")
 +                                    .append(problem.getSourceLineNumber())
 +                                    .append(" (in generated source): ")
 +                                    .append(problem.getMessage())
 +                                    .append('\n');
 +                            fullSource = true;
 +                        }
 +                    }
 +                    else
 +                    {
 +                        problems.append("Line ")
 +                                .append(Long.toString(ln))
 +                                .append(": ")
 +                                .append(problem.getMessage())
 +                                .append('\n');
 +                    }
 +                }
 +
 +                if (fullSource)
 +                    throw new InvalidRequestException("Java source compilation failed:\n" + problems + "\n generated source:\n" + javaSource);
 +                else
 +                    throw new InvalidRequestException("Java source compilation failed:\n" + problems);
 +            }
 +
 +            // Verify the UDF bytecode against use of probably dangerous code
 +            Set<String> errors = udfByteCodeVerifier.verify(targetClassLoader.classData(targetClassName));
 +            String validDeclare = "not allowed method declared: " + executeInternalName + '(';
 +            String validCall = "call to " + targetClassName.replace('.', '/') + '.' + executeInternalName + "()";
 +            for (Iterator<String> i = errors.iterator(); i.hasNext();)
 +            {
 +                String error = i.next();
 +                // we generate a random name of the private, internal execute method, which is detected by the byte-code verifier
 +                if (error.startsWith(validDeclare) || error.equals(validCall))
 +                {
 +                    i.remove();
 +                }
 +            }
 +            if (!errors.isEmpty())
 +                throw new InvalidRequestException("Java UDF validation failed: " + errors);
 +
 +            // Load the class and create a new instance of it
 +            Thread thread = Thread.currentThread();
 +            ClassLoader orig = thread.getContextClassLoader();
 +            try
 +            {
 +                thread.setContextClassLoader(UDFunction.udfClassLoader);
 +                // Execute UDF intiialization from UDF class loader
 +
 +                Class cls = Class.forName(targetClassName, false, targetClassLoader);
 +
 +                // Count only non-synthetic methods, so code coverage instrumentation doesn't cause a miscount
 +                int nonSyntheticMethodCount = 0;
 +                for (Method m : cls.getDeclaredMethods())
 +                {
 +                    if (!m.isSynthetic())
 +                    {
 +                        nonSyntheticMethodCount += 1;
 +                    }
 +                }
 +
 +                if (nonSyntheticMethodCount != 2 || cls.getDeclaredConstructors().length != 1)
 +                    throw new InvalidRequestException("Check your source to not define additional Java methods or constructors");
 +                MethodType methodType = MethodType.methodType(void.class)
 +                                                  .appendParameterTypes(DataType.class, DataType[].class);
 +                MethodHandle ctor = MethodHandles.lookup().findConstructor(cls, methodType);
 +                this.javaUDF = (JavaUDF) ctor.invokeWithArguments(returnDataType, argDataTypes);
 +            }
 +            finally
 +            {
 +                thread.setContextClassLoader(orig);
 +            }
 +        }
 +        catch (InvocationTargetException e)
 +        {
 +            // in case of an ITE, use the cause
 +            throw new InvalidRequestException(String.format("Could not compile function '%s' from Java source: %s", name, e.getCause()));
 +        }
 +        catch (VirtualMachineError e)
 +        {
 +            throw e;
 +        }
 +        catch (Throwable e)
 +        {
 +            throw new InvalidRequestException(String.format("Could not compile function '%s' from Java source: %s", name, e));
 +        }
 +    }
 +
 +    protected ExecutorService executor()
 +    {
 +        return executor;
 +    }
 +
 +    protected ByteBuffer executeUserDefined(int protocolVersion, List<ByteBuffer> params)
 +    {
 +        return javaUDF.executeImpl(protocolVersion, params);
 +    }
 +
 +
 +    private static int countNewlines(StringBuilder javaSource)
 +    {
 +        int ln = 0;
 +        for (int i = 0; i < javaSource.length(); i++)
 +            if (javaSource.charAt(i) == '\n')
 +                ln++;
 +        return ln;
 +    }
 +
 +    private static String generateClassName(FunctionName name, char prefix)
 +    {
 +        String qualifiedName = name.toString();
 +
 +        StringBuilder sb = new StringBuilder(qualifiedName.length() + 10);
 +        sb.append(prefix);
 +        for (int i = 0; i < qualifiedName.length(); i++)
 +        {
 +            char c = qualifiedName.charAt(i);
 +            if (Character.isJavaIdentifierPart(c))
 +                sb.append(c);
 +            else
 +                sb.append(Integer.toHexString(((short)c)&0xffff));
 +        }
 +        sb.append('_')
 +          .append(ThreadLocalRandom.current().nextInt() & 0xffffff)
 +          .append('_')
 +          .append(classSequence.incrementAndGet());
 +        return sb.toString();
 +    }
 +
 +    private static String javaSourceName(Class<?> type)
 +    {
 +        String n = type.getName();
 +        return n.startsWith("java.lang.") ? type.getSimpleName() : n;
 +    }
 +
 +    private static String generateArgumentList(Class<?>[] paramTypes, List<ColumnIdentifier> argNames)
 +    {
 +        // initial builder size can just be a guess (prevent temp object allocations)
 +        StringBuilder code = new StringBuilder(32 * paramTypes.length);
 +        for (int i = 0; i < paramTypes.length; i++)
 +        {
 +            if (i > 0)
 +                code.append(", ");
 +            code.append(javaSourceName(paramTypes[i]))
 +                .append(' ')
 +                .append(argNames.get(i));
 +        }
 +        return code.toString();
 +    }
 +
 +    private static String generateArguments(Class<?>[] paramTypes, List<ColumnIdentifier> argNames)
 +    {
 +        StringBuilder code = new StringBuilder(64 * paramTypes.length);
 +        for (int i = 0; i < paramTypes.length; i++)
 +        {
 +            if (i > 0)
 +                code.append(",\n");
 +
 +            if (logger.isTraceEnabled())
 +                code.append("            /* parameter '").append(argNames.get(i)).append("' */\n");
 +
 +            code
 +                // cast to Java type
 +                .append("            (").append(javaSourceName(paramTypes[i])).append(") ")
 +                // generate object representation of input parameter (call UDFunction.compose)
 +                .append(composeMethod(paramTypes[i])).append("(protocolVersion, ").append(i).append(", params.get(").append(i).append("))");
 +        }
 +        return code.toString();
 +    }
 +
 +    private static String composeMethod(Class<?> type)
 +    {
 +        return (type.isPrimitive()) ? ("super.compose_" + type.getName()) : "super.compose";
 +    }
 +
 +    // Java source UDFs are a very simple compilation task, which allows us to let one class implement
 +    // all interfaces required by ECJ.
 +    static final class EcjCompilationUnit implements ICompilationUnit, ICompilerRequestor, INameEnvironment
 +    {
 +        List<IProblem> problemList;
 +        private final String className;
 +        private final char[] sourceCode;
 +
 +        EcjCompilationUnit(String sourceCode, String className)
 +        {
 +            this.className = className;
 +            this.sourceCode = sourceCode.toCharArray();
 +        }
 +
 +        // ICompilationUnit
 +
 +        @Override
 +        public char[] getFileName()
 +        {
 +            return sourceCode;
 +        }
 +
 +        @Override
 +        public char[] getContents()
 +        {
 +            return sourceCode;
 +        }
 +
 +        @Override
 +        public char[] getMainTypeName()
 +        {
 +            int dot = className.lastIndexOf('.');
 +            return ((dot > 0) ? className.substring(dot + 1) : className).toCharArray();
 +        }
 +
 +        @Override
 +        public char[][] getPackageName()
 +        {
 +            StringTokenizer izer = new StringTokenizer(className, ".");
 +            char[][] result = new char[izer.countTokens() - 1][];
 +            for (int i = 0; i < result.length; i++)
 +                result[i] = izer.nextToken().toCharArray();
 +            return result;
 +        }
 +
 +        @Override
 +        public boolean ignoreOptionalProblems()
 +        {
 +            return false;
 +        }
 +
 +        // ICompilerRequestor
 +
 +        @Override
 +        public void acceptResult(CompilationResult result)
 +        {
 +            if (result.hasErrors())
 +            {
 +                IProblem[] problems = result.getProblems();
 +                if (problemList == null)
 +                    problemList = new ArrayList<>(problems.length);
 +                Collections.addAll(problemList, problems);
 +            }
 +            else
 +            {
 +                ClassFile[] classFiles = result.getClassFiles();
 +                for (ClassFile classFile : classFiles)
 +                    targetClassLoader.addClass(className, classFile.getBytes());
 +            }
 +        }
 +
 +        // INameEnvironment
 +
 +        @Override
 +        public NameEnvironmentAnswer findType(char[][] compoundTypeName)
 +        {
 +            StringBuilder result = new StringBuilder();
 +            for (int i = 0; i < compoundTypeName.length; i++)
 +            {
 +                if (i > 0)
 +                    result.append('.');
 +                result.append(compoundTypeName[i]);
 +            }
 +            return findType(result.toString());
 +        }
 +
 +        @Override
 +        public NameEnvironmentAnswer findType(char[] typeName, char[][] packageName)
 +        {
 +            StringBuilder result = new StringBuilder();
 +            int i = 0;
 +            for (; i < packageName.length; i++)
 +            {
 +                if (i > 0)
 +                    result.append('.');
 +                result.append(packageName[i]);
 +            }
 +            if (i > 0)
 +                result.append('.');
 +            result.append(typeName);
 +            return findType(result.toString());
 +        }
 +
 +        private NameEnvironmentAnswer findType(String className)
 +        {
 +            if (className.equals(this.className))
 +            {
 +                return new NameEnvironmentAnswer(this, null);
 +            }
 +
 +            String resourceName = className.replace('.', '/') + ".class";
 +
 +            try (InputStream is = UDFunction.udfClassLoader.getResourceAsStream(resourceName))
 +            {
 +                if (is != null)
 +                {
 +                    byte[] classBytes = ByteStreams.toByteArray(is);
 +                    char[] fileName = className.toCharArray();
 +                    ClassFileReader classFileReader = new ClassFileReader(classBytes, fileName, true);
 +                    return new NameEnvironmentAnswer(classFileReader, null);
 +                }
 +            }
 +            catch (IOException | ClassFormatException exc)
 +            {
 +                throw new RuntimeException(exc);
 +            }
 +            return null;
 +        }
 +
 +        private boolean isPackage(String result)
 +        {
 +            if (result.equals(this.className))
 +                return false;
 +            String resourceName = result.replace('.', '/') + ".class";
 +            try (InputStream is = UDFunction.udfClassLoader.getResourceAsStream(resourceName))
 +            {
 +                return is == null;
 +            }
 +            catch (IOException e)
 +            {
 +                // we are here, since close on is failed. That means it was not null
 +                return false;
 +            }
 +        }
 +
 +        @Override
 +        public boolean isPackage(char[][] parentPackageName, char[] packageName)
 +        {
 +            StringBuilder result = new StringBuilder();
 +            int i = 0;
 +            if (parentPackageName != null)
 +                for (; i < parentPackageName.length; i++)
 +                {
 +                    if (i > 0)
 +                        result.append('.');
 +                    result.append(parentPackageName[i]);
 +                }
 +
 +            if (Character.isUpperCase(packageName[0]) && !isPackage(result.toString()))
 +                return false;
 +            if (i > 0)
 +                result.append('.');
 +            result.append(packageName);
 +
 +            return isPackage(result.toString());
 +        }
 +
 +        @Override
 +        public void cleanup()
 +        {
 +        }
 +    }
 +
 +    static final class EcjTargetClassLoader extends SecureClassLoader
 +    {
 +        EcjTargetClassLoader()
 +        {
 +            super(UDFunction.udfClassLoader);
 +        }
 +
 +        // This map is usually empty.
 +        // It only contains data *during* UDF compilation but not during runtime.
 +        //
 +        // addClass() is invoked by ECJ after successful compilation of the generated Java source.
 +        // loadClass(targetClassName) is invoked by buildUDF() after ECJ returned from successful compilation.
 +        //
 +        private final Map<String, byte[]> classes = new ConcurrentHashMap<>();
 +
 +        void addClass(String className, byte[] classData)
 +        {
 +            classes.put(className, classData);
 +        }
 +
 +        byte[] classData(String className)
 +        {
 +            return classes.get(className);
 +        }
 +
 +        protected Class<?> findClass(String name) throws ClassNotFoundException
 +        {
 +            // remove the class binary - it's only used once - so it's wasting heap
 +            byte[] classData = classes.remove(name);
 +
 +            if (classData != null)
 +                return defineClass(name, classData, 0, classData.length, protectionDomain);
 +
 +            return getParent().loadClass(name);
 +        }
 +
 +        protected PermissionCollection getPermissions(CodeSource codesource)
 +        {
 +            return ThreadAwareSecurityManager.noPermissions;
 +        }
 +    }}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/174a0aa7/src/java/org/apache/cassandra/cql3/functions/UDFByteCodeVerifier.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/functions/UDFByteCodeVerifier.java
index 6e5978c,0000000..59cef70
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/cql3/functions/UDFByteCodeVerifier.java
+++ b/src/java/org/apache/cassandra/cql3/functions/UDFByteCodeVerifier.java
@@@ -1,215 -1,0 +1,232 @@@
 +/*
 + * 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.functions;
 +
 +import java.util.ArrayList;
 +import java.util.Collection;
++import java.util.HashSet;
 +import java.util.List;
 +import java.util.Set;
 +import java.util.TreeSet;
 +
 +import com.google.common.collect.HashMultimap;
 +import com.google.common.collect.Multimap;
 +
 +import org.objectweb.asm.ClassReader;
 +import org.objectweb.asm.ClassVisitor;
 +import org.objectweb.asm.FieldVisitor;
 +import org.objectweb.asm.Handle;
 +import org.objectweb.asm.MethodVisitor;
 +import org.objectweb.asm.Opcodes;
 +
 +/**
 + * Verifies Java UDF byte code.
 + * Checks for disallowed method calls (e.g. {@code Object.finalize()}),
 + * additional code in the constructor,
 + * use of {@code synchronized} blocks,
 + * too many methods.
 + */
 +public final class UDFByteCodeVerifier
 +{
 +
 +    public static final String JAVA_UDF_NAME = JavaUDF.class.getName().replace('.', '/');
 +    public static final String OBJECT_NAME = Object.class.getName().replace('.', '/');
 +    public static final String CTOR_SIG = "(Lcom/datastax/driver/core/DataType;[Lcom/datastax/driver/core/DataType;)V";
 +
++    private final Set<String> disallowedClasses = new HashSet<>();
 +    private final Multimap<String, String> disallowedMethodCalls = HashMultimap.create();
 +    private final List<String> disallowedPackages = new ArrayList<>();
 +
 +    public UDFByteCodeVerifier()
 +    {
 +        addDisallowedMethodCall(OBJECT_NAME, "clone");
 +        addDisallowedMethodCall(OBJECT_NAME, "finalize");
 +        addDisallowedMethodCall(OBJECT_NAME, "notify");
 +        addDisallowedMethodCall(OBJECT_NAME, "notifyAll");
 +        addDisallowedMethodCall(OBJECT_NAME, "wait");
 +    }
 +
++    public UDFByteCodeVerifier addDisallowedClass(String clazz)
++    {
++        disallowedClasses.add(clazz);
++        return this;
++    }
++
 +    public UDFByteCodeVerifier addDisallowedMethodCall(String clazz, String method)
 +    {
 +        disallowedMethodCalls.put(clazz, method);
 +        return this;
 +    }
 +
 +    public UDFByteCodeVerifier addDisallowedPackage(String pkg)
 +    {
 +        disallowedPackages.add(pkg);
 +        return this;
 +    }
 +
 +    public Set<String> verify(byte[] bytes)
 +    {
 +        Set<String> errors = new TreeSet<>(); // it's a TreeSet for unit tests
 +        ClassVisitor classVisitor = new ClassVisitor(Opcodes.ASM5)
 +        {
 +            public FieldVisitor visitField(int access, String name, String desc, String signature, Object value)
 +            {
 +                errors.add("field declared: " + name);
 +                return null;
 +            }
 +
 +            public MethodVisitor visitMethod(int access, String name, String desc, String signature, String[] exceptions)
 +            {
 +                if ("<init>".equals(name) && CTOR_SIG.equals(desc))
 +                {
 +                    if (Opcodes.ACC_PUBLIC != access)
 +                        errors.add("constructor not public");
 +                    // allowed constructor - JavaUDF(DataType returnDataType, DataType[] argDataTypes)
 +                    return new ConstructorVisitor(errors);
 +                }
 +                if ("executeImpl".equals(name) && "(ILjava/util/List;)Ljava/nio/ByteBuffer;".equals(desc))
 +                {
 +                    if (Opcodes.ACC_PROTECTED != access)
 +                        errors.add("executeImpl not protected");
 +                    // the executeImpl method - ByteBuffer executeImpl(int protocolVersion, List<ByteBuffer> params)
 +                    return new ExecuteImplVisitor(errors);
 +                }
 +                if ("<clinit>".equals(name))
 +                {
 +                    errors.add("static initializer declared");
 +                }
 +                else
 +                {
 +                    errors.add("not allowed method declared: " + name + desc);
 +                    return new ExecuteImplVisitor(errors);
 +                }
 +                return null;
 +            }
 +
 +            public void visit(int version, int access, String name, String signature, String superName, String[] interfaces)
 +            {
 +                if (!JAVA_UDF_NAME.equals(superName))
 +                {
 +                    errors.add("class does not extend " + JavaUDF.class.getName());
 +                }
 +                if (access != (Opcodes.ACC_PUBLIC | Opcodes.ACC_FINAL | Opcodes.ACC_SUPER))
 +                {
 +                    errors.add("class not public final");
 +                }
 +                super.visit(version, access, name, signature, superName, interfaces);
 +            }
 +
 +            public void visitInnerClass(String name, String outerName, String innerName, int access)
 +            {
 +                errors.add("class declared as inner class");
 +                super.visitInnerClass(name, outerName, innerName, access);
 +            }
 +        };
 +
 +        ClassReader classReader = new ClassReader(bytes);
 +        classReader.accept(classVisitor, ClassReader.SKIP_DEBUG);
 +
 +        return errors;
 +    }
 +
 +    private class ExecuteImplVisitor extends MethodVisitor
 +    {
 +        private final Set<String> errors;
 +
 +        ExecuteImplVisitor(Set<String> errors)
 +        {
 +            super(Opcodes.ASM5);
 +            this.errors = errors;
 +        }
 +
 +        public void visitMethodInsn(int opcode, String owner, String name, String desc, boolean itf)
 +        {
++            if (disallowedClasses.contains(owner))
++            {
++                errorDisallowed(owner, name);
++            }
 +            Collection<String> disallowed = disallowedMethodCalls.get(owner);
 +            if (disallowed != null && disallowed.contains(name))
 +            {
-                 errors.add("call to " + name + "()");
++                errorDisallowed(owner, name);
 +            }
 +            if (!JAVA_UDF_NAME.equals(owner))
 +            {
 +                for (String pkg : disallowedPackages)
 +                {
 +                    if (owner.startsWith(pkg))
-                         errors.add("call to " + owner + '.' + name + "()");
++                        errorDisallowed(owner, name);
 +                }
 +            }
 +            super.visitMethodInsn(opcode, owner, name, desc, itf);
 +        }
 +
++        private void errorDisallowed(String owner, String name)
++        {
++            errors.add("call to " + owner.replace('/', '.') + '.' + name + "()");
++        }
++
 +        public void visitInsn(int opcode)
 +        {
 +            switch (opcode)
 +            {
 +                case Opcodes.MONITORENTER:
 +                case Opcodes.MONITOREXIT:
 +                    errors.add("use of synchronized");
 +                    break;
 +            }
 +            super.visitInsn(opcode);
 +        }
 +    }
 +
 +    private static class ConstructorVisitor extends MethodVisitor
 +    {
 +        private final Set<String> errors;
 +
 +        ConstructorVisitor(Set<String> errors)
 +        {
 +            super(Opcodes.ASM5);
 +            this.errors = errors;
 +        }
 +
 +        public void visitInvokeDynamicInsn(String name, String desc, Handle bsm, Object... bsmArgs)
 +        {
 +            errors.add("Use of invalid method instruction in constructor");
 +            super.visitInvokeDynamicInsn(name, desc, bsm, bsmArgs);
 +        }
 +
 +        public void visitMethodInsn(int opcode, String owner, String name, String desc, boolean itf)
 +        {
 +            if (!(Opcodes.INVOKESPECIAL == opcode && JAVA_UDF_NAME.equals(owner) && "<init>".equals(name) && CTOR_SIG.equals(desc)))
 +            {
 +                errors.add("initializer declared");
 +            }
 +            super.visitMethodInsn(opcode, owner, name, desc, itf);
 +        }
 +
 +        public void visitInsn(int opcode)
 +        {
 +            if (Opcodes.RETURN != opcode)
 +            {
 +                errors.add("initializer declared");
 +            }
 +            super.visitInsn(opcode);
 +        }
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/174a0aa7/src/java/org/apache/cassandra/cql3/functions/UDFunction.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/functions/UDFunction.java
index 5f7c024,1e5cea6..04a4c3d
--- a/src/java/org/apache/cassandra/cql3/functions/UDFunction.java
+++ b/src/java/org/apache/cassandra/cql3/functions/UDFunction.java
@@@ -74,101 -53,6 +74,107 @@@ public abstract class UDFunction extend
      protected final DataType returnDataType;
      protected final boolean calledOnNullInput;
  
 +    //
 +    // Access to classes is controlled via a whitelist and a blacklist.
 +    //
 +    // When a class is requested (both during compilation and runtime),
 +    // the whitelistedPatterns array is searched first, whether the
 +    // requested name matches one of the patterns. If not, nothing is
 +    // returned from the class-loader - meaning ClassNotFoundException
 +    // during runtime and "type could not resolved" during compilation.
 +    //
 +    // If a whitelisted pattern has been found, the blacklistedPatterns
 +    // array is searched for a match. If a match is found, class-loader
 +    // rejects access. Otherwise the class/resource can be loaded.
 +    //
 +    private static final String[] whitelistedPatterns =
 +    {
 +    "com/datastax/driver/core/",
 +    "com/google/common/reflect/TypeToken",
 +    "java/io/IOException.class",
 +    "java/io/Serializable.class",
 +    "java/lang/",
 +    "java/math/",
++    "java/net/InetAddress.class",
++    "java/net/Inet4Address.class",
++    "java/net/Inet6Address.class",
++    "java/net/UnknownHostException.class", // req'd by InetAddress
++    "java/net/NetworkInterface.class", // req'd by InetAddress
++    "java/net/SocketException.class", // req'd by InetAddress
 +    "java/nio/Buffer.class",
 +    "java/nio/ByteBuffer.class",
 +    "java/text/",
 +    "java/time/",
 +    "java/util/",
 +    "org/apache/cassandra/cql3/functions/JavaUDF.class",
 +    "org/apache/cassandra/exceptions/",
 +    };
 +    // Only need to blacklist a pattern, if it would otherwise be allowed via whitelistedPatterns
 +    private static final String[] blacklistedPatterns =
 +    {
 +    "com/datastax/driver/core/Cluster.class",
 +    "com/datastax/driver/core/Metrics.class",
 +    "com/datastax/driver/core/NettyOptions.class",
 +    "com/datastax/driver/core/Session.class",
 +    "com/datastax/driver/core/Statement.class",
 +    "com/datastax/driver/core/TimestampGenerator.class", // indirectly covers ServerSideTimestampGenerator + ThreadLocalMonotonicTimestampGenerator
 +    "java/lang/Compiler.class",
 +    "java/lang/InheritableThreadLocal.class",
 +    "java/lang/Package.class",
 +    "java/lang/Process.class",
 +    "java/lang/ProcessBuilder.class",
 +    "java/lang/ProcessEnvironment.class",
 +    "java/lang/ProcessImpl.class",
 +    "java/lang/Runnable.class",
 +    "java/lang/Runtime.class",
 +    "java/lang/Shutdown.class",
 +    "java/lang/Thread.class",
 +    "java/lang/ThreadGroup.class",
 +    "java/lang/ThreadLocal.class",
 +    "java/lang/instrument/",
 +    "java/lang/invoke/",
 +    "java/lang/management/",
 +    "java/lang/ref/",
 +    "java/lang/reflect/",
 +    "java/util/ServiceLoader.class",
 +    "java/util/Timer.class",
 +    "java/util/concurrent/",
 +    "java/util/function/",
 +    "java/util/jar/",
 +    "java/util/logging/",
 +    "java/util/prefs/",
 +    "java/util/spi/",
 +    "java/util/stream/",
 +    "java/util/zip/",
 +    };
 +
 +    static boolean secureResource(String resource)
 +    {
 +        while (resource.startsWith("/"))
 +            resource = resource.substring(1);
 +
 +        for (String white : whitelistedPatterns)
 +            if (resource.startsWith(white))
 +            {
 +
 +                // resource is in whitelistedPatterns, let's see if it is not explicityl blacklisted
 +                for (String black : blacklistedPatterns)
 +                    if (resource.startsWith(black))
 +                    {
 +                        logger.trace("access denied: resource {}", resource);
 +                        return false;
 +                    }
 +
 +                return true;
 +            }
 +
 +        logger.trace("access denied: resource {}", resource);
 +        return false;
 +    }
 +
 +    // setup the UDF class loader with no parent class loader so that we have full control about what class/resource UDF uses
 +    static final ClassLoader udfClassLoader = new UDFClassLoader();
 +
      protected UDFunction(FunctionName name,
                           List<ColumnIdentifier> argNames,
                           List<AbstractType<?>> argTypes,

http://git-wip-us.apache.org/repos/asf/cassandra/blob/174a0aa7/test/unit/org/apache/cassandra/cql3/validation/entities/UFTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/cql3/validation/entities/UFTest.java
index c0d8902,25566ad..ce62e6c
--- a/test/unit/org/apache/cassandra/cql3/validation/entities/UFTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/UFTest.java
@@@ -27,15 -29,19 +27,16 @@@ import java.util.Set
  import java.util.TreeMap;
  import java.util.TreeSet;
  import java.util.UUID;
 +import java.security.AccessControlException;
  
  import org.junit.Assert;
 -import org.junit.BeforeClass;
  import org.junit.Test;
  
 -import com.datastax.driver.core.DataType;
 -import com.datastax.driver.core.Row;
 -import com.datastax.driver.core.TupleType;
 -import com.datastax.driver.core.TupleValue;
 -import com.datastax.driver.core.UDTValue;
 +import com.datastax.driver.core.*;
  import com.datastax.driver.core.exceptions.InvalidQueryException;
  import org.apache.cassandra.config.DatabaseDescriptor;
 +import org.apache.cassandra.config.Schema;
+ import org.apache.cassandra.cql3.CQL3Type;
  import org.apache.cassandra.cql3.CQLTester;
  import org.apache.cassandra.cql3.QueryProcessor;
  import org.apache.cassandra.cql3.UntypedResultSet;
@@@ -2185,224 -2615,35 +2186,263 @@@ public class UFTest extends CQLTeste
      }
  
      @Test
+     public void testAllNativeTypes() throws Throwable
+     {
+         StringBuilder sig = new StringBuilder();
+         StringBuilder args = new StringBuilder();
+         for (CQL3Type.Native type : CQL3Type.Native.values())
+         {
++            if (type == CQL3Type.Native.EMPTY)
++                continue;
++
+             if (sig.length() > 0)
+                 sig.append(',');
+             sig.append(type.toString());
+ 
+             if (args.length() > 0)
+                 args.append(',');
+             args.append("arg").append(type.toString()).append(' ').append(type.toString());
+         }
+         createFunction(KEYSPACE, sig.toString(),
+                        "CREATE OR REPLACE FUNCTION %s(" + args + ") " +
+                        "RETURNS NULL ON NULL INPUT " +
+                        "RETURNS int " +
+                        "LANGUAGE JAVA\n" +
+                        "AS 'return 0;'");
+ 
+         for (CQL3Type.Native type : CQL3Type.Native.values())
+         {
++            if (type == CQL3Type.Native.EMPTY)
++                continue;
++
+             createFunction(KEYSPACE_PER_TEST, type.toString(),
+                            "CREATE OR REPLACE FUNCTION %s(val " + type.toString() + ") " +
+                            "RETURNS NULL ON NULL INPUT " +
+                            "RETURNS int " +
+                            "LANGUAGE JAVA\n" +
+                            "AS 'return 0;'");
+         }
+     }
++
++    @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.isInShutdownHook(); 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.isInShutdownHook(); 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.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.getWarnings();
 +                Assert.assertNotNull(warnings);
 +                Assert.assertFalse(warnings.isEmpty());
 +                ClientWarn.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);
 +            }
 +        }
 +    }
  }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/174a0aa7/test/unit/org/apache/cassandra/cql3/validation/entities/UFVerifierTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/cql3/validation/entities/UFVerifierTest.java
index 69733d5,0000000..0b78bf2
mode 100644,000000..100644
--- a/test/unit/org/apache/cassandra/cql3/validation/entities/UFVerifierTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/UFVerifierTest.java
@@@ -1,307 -1,0 +1,337 @@@
 +/*
 + * 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.io.ByteArrayOutputStream;
 +import java.io.IOException;
 +import java.io.InputStream;
 +import java.net.URL;
 +import java.util.Arrays;
 +import java.util.Collections;
 +import java.util.HashSet;
 +
 +import org.junit.Test;
 +
 +import org.apache.cassandra.cql3.CQLTester;
 +import org.apache.cassandra.cql3.functions.UDFByteCodeVerifier;
 +import org.apache.cassandra.cql3.validation.entities.udfverify.CallClone;
 +import org.apache.cassandra.cql3.validation.entities.udfverify.CallComDatastax;
 +import org.apache.cassandra.cql3.validation.entities.udfverify.CallFinalize;
 +import org.apache.cassandra.cql3.validation.entities.udfverify.CallOrgApache;
 +import org.apache.cassandra.cql3.validation.entities.udfverify.ClassWithField;
 +import org.apache.cassandra.cql3.validation.entities.udfverify.ClassWithInitializer;
 +import org.apache.cassandra.cql3.validation.entities.udfverify.ClassWithInitializer2;
 +import org.apache.cassandra.cql3.validation.entities.udfverify.ClassWithInitializer3;
 +import org.apache.cassandra.cql3.validation.entities.udfverify.ClassWithStaticInitializer;
 +import org.apache.cassandra.cql3.validation.entities.udfverify.GoodClass;
 +import org.apache.cassandra.cql3.validation.entities.udfverify.UseOfSynchronized;
 +import org.apache.cassandra.cql3.validation.entities.udfverify.UseOfSynchronizedWithNotify;
 +import org.apache.cassandra.cql3.validation.entities.udfverify.UseOfSynchronizedWithNotifyAll;
 +import org.apache.cassandra.cql3.validation.entities.udfverify.UseOfSynchronizedWithWait;
 +import org.apache.cassandra.cql3.validation.entities.udfverify.UseOfSynchronizedWithWaitL;
 +import org.apache.cassandra.cql3.validation.entities.udfverify.UseOfSynchronizedWithWaitLI;
 +
 +import static org.junit.Assert.assertEquals;
 +
 +/**
 + * Test the Java UDF byte code verifier.
 + */
 +public class UFVerifierTest extends CQLTester
 +{
 +    @Test
 +    public void testByteCodeVerifier()
 +    {
 +        new UDFByteCodeVerifier().verify(readClass(GoodClass.class));
 +    }
 +
 +    @Test
 +    public void testClassWithField()
 +    {
 +        assertEquals(new HashSet<>(Collections.singletonList("field declared: field")),
 +                     new UDFByteCodeVerifier().verify(readClass(ClassWithField.class)));
 +    }
 +
 +    @Test
 +    public void testClassWithInitializer()
 +    {
 +        assertEquals(new HashSet<>(Arrays.asList("field declared: field",
 +                                                 "initializer declared")),
 +                     new UDFByteCodeVerifier().verify(readClass(ClassWithInitializer.class)));
 +    }
 +
 +    @Test
 +    public void testClassWithInitializer2()
 +    {
 +        assertEquals(new HashSet<>(Arrays.asList("field declared: field",
 +                                                 "initializer declared")),
 +                     new UDFByteCodeVerifier().verify(readClass(ClassWithInitializer2.class)));
 +    }
 +
 +    @Test
 +    public void testClassWithInitializer3()
 +    {
 +        assertEquals(new HashSet<>(Collections.singletonList("initializer declared")),
 +                     new UDFByteCodeVerifier().verify(readClass(ClassWithInitializer3.class)));
 +    }
 +
 +    @Test
 +    public void testClassWithStaticInitializer()
 +    {
 +        assertEquals(new HashSet<>(Collections.singletonList("static initializer declared")),
 +                     new UDFByteCodeVerifier().verify(readClass(ClassWithStaticInitializer.class)));
 +    }
 +
 +    @Test
 +    public void testUseOfSynchronized()
 +    {
 +        assertEquals(new HashSet<>(Collections.singletonList("use of synchronized")),
 +                     new UDFByteCodeVerifier().verify(readClass(UseOfSynchronized.class)));
 +    }
 +
 +    @Test
 +    public void testUseOfSynchronizedWithNotify()
 +    {
-         assertEquals(new HashSet<>(Arrays.asList("use of synchronized", "call to notify()")),
++        assertEquals(new HashSet<>(Arrays.asList("use of synchronized", "call to java.lang.Object.notify()")),
 +                     new UDFByteCodeVerifier().verify(readClass(UseOfSynchronizedWithNotify.class)));
 +    }
 +
 +    @Test
 +    public void testUseOfSynchronizedWithNotifyAll()
 +    {
-         assertEquals(new HashSet<>(Arrays.asList("use of synchronized", "call to notifyAll()")),
++        assertEquals(new HashSet<>(Arrays.asList("use of synchronized", "call to java.lang.Object.notifyAll()")),
 +                     new UDFByteCodeVerifier().verify(readClass(UseOfSynchronizedWithNotifyAll.class)));
 +    }
 +
 +    @Test
 +    public void testUseOfSynchronizedWithWait()
 +    {
-         assertEquals(new HashSet<>(Arrays.asList("use of synchronized", "call to wait()")),
++        assertEquals(new HashSet<>(Arrays.asList("use of synchronized", "call to java.lang.Object.wait()")),
 +                     new UDFByteCodeVerifier().verify(readClass(UseOfSynchronizedWithWait.class)));
 +    }
 +
 +    @Test
 +    public void testUseOfSynchronizedWithWaitL()
 +    {
-         assertEquals(new HashSet<>(Arrays.asList("use of synchronized", "call to wait()")),
++        assertEquals(new HashSet<>(Arrays.asList("use of synchronized", "call to java.lang.Object.wait()")),
 +                     new UDFByteCodeVerifier().verify(readClass(UseOfSynchronizedWithWaitL.class)));
 +    }
 +
 +    @Test
 +    public void testUseOfSynchronizedWithWaitI()
 +    {
-         assertEquals(new HashSet<>(Arrays.asList("use of synchronized", "call to wait()")),
++        assertEquals(new HashSet<>(Arrays.asList("use of synchronized", "call to java.lang.Object.wait()")),
 +                     new UDFByteCodeVerifier().verify(readClass(UseOfSynchronizedWithWaitLI.class)));
 +    }
 +
 +    @Test
 +    public void testCallClone()
 +    {
-         assertEquals(new HashSet<>(Collections.singletonList("call to clone()")),
++        assertEquals(new HashSet<>(Collections.singletonList("call to java.lang.Object.clone()")),
 +                     new UDFByteCodeVerifier().verify(readClass(CallClone.class)));
 +    }
 +
 +    @Test
 +    public void testCallFinalize()
 +    {
-         assertEquals(new HashSet<>(Collections.singletonList("call to finalize()")),
++        assertEquals(new HashSet<>(Collections.singletonList("call to java.lang.Object.finalize()")),
 +                     new UDFByteCodeVerifier().verify(readClass(CallFinalize.class)));
 +    }
 +
 +    @Test
 +    public void testCallComDatastax()
 +    {
-         assertEquals(new HashSet<>(Collections.singletonList("call to com/datastax/driver/core/DataType.cint()")),
++        assertEquals(new HashSet<>(Collections.singletonList("call to com.datastax.driver.core.DataType.cint()")),
 +                     new UDFByteCodeVerifier().addDisallowedPackage("com/").verify(readClass(CallComDatastax.class)));
 +    }
 +
 +    @Test
 +    public void testCallOrgApache()
 +    {
-         assertEquals(new HashSet<>(Collections.singletonList("call to org/apache/cassandra/config/DatabaseDescriptor.getClusterName()")),
++        assertEquals(new HashSet<>(Collections.singletonList("call to org.apache.cassandra.config.DatabaseDescriptor.getClusterName()")),
 +                     new UDFByteCodeVerifier().addDisallowedPackage("org/").verify(readClass(CallOrgApache.class)));
 +    }
 +
 +    @SuppressWarnings("resource")
 +    private static byte[] readClass(Class<?> clazz)
 +    {
 +        ByteArrayOutputStream out = new ByteArrayOutputStream();
 +        URL res = clazz.getClassLoader().getResource(clazz.getName().replace('.', '/') + ".class");
 +        assert res != null;
 +        try (InputStream input = res.openConnection().getInputStream())
 +        {
 +            int i;
 +            while ((i = input.read()) != -1)
 +                out.write(i);
 +            return out.toByteArray();
 +        }
 +        catch (IOException e)
 +        {
 +            throw new RuntimeException(e);
 +        }
 +    }
 +
 +    @Test
 +    public void testInvalidByteCodeUDFs() throws Throwable
 +    {
 +        assertInvalidByteCode("try\n" +
 +                              "{\n" +
 +                              "    clone();\n" +
 +                              "}\n" +
 +                              "catch (CloneNotSupportedException e)\n" +
 +                              "{\n" +
 +                              "    throw new RuntimeException(e);\n" +
 +                              "}\n" +
-                               "return 0d;", "Java UDF validation failed: [call to clone()]");
++                              "return 0d;", "Java UDF validation failed: [call to java.lang.Object.clone()]");
 +        assertInvalidByteCode("try\n" +
 +                              "{\n" +
 +                              "    finalize();\n" +
 +                              "}\n" +
 +                              "catch (Throwable e)\n" +
 +                              "{\n" +
 +                              "    throw new RuntimeException(e);\n" +
 +                              "}\n" +
-                               "return 0d;", "Java UDF validation failed: [call to finalize()]");
++                              "return 0d;", "Java UDF validation failed: [call to java.lang.Object.finalize()]");
 +        assertInvalidByteCode('\n' +
 +                              "return 0d;\n" +
 +                              "    }\n" +
 +                              '\n' +
 +                              "    Object field;\n" +
 +                              '\n' +
 +                              "    {", "Java UDF validation failed: [field declared: field]");
 +        assertInvalidByteCode('\n' +
 +                              "return 0d;\n" +
 +                              "    }\n" +
 +                              '\n' +
 +                              "    final Object field;\n" +
 +                              '\n' +
 +                              "    {\n" +
 +                              "field = new Object();", "Java UDF validation failed: [field declared: field, initializer declared]");
 +        assertInvalidByteCode('\n' +
 +                              "return 0d;\n" +
 +                              "    }\n" +
 +                              '\n' +
 +                              "    Object field = new Object();\n" +
 +                              '\n' +
 +                              "    {\n" +
 +                              "Math.sin(1d);", "Java UDF validation failed: [field declared: field, initializer declared]");
 +        assertInvalidByteCode('\n' +
 +                              "return 0d;\n" +
 +                              "    }\n" +
 +                              '\n' +
 +                              "    {\n" +
 +                              "Math.sin(1d);", "Java UDF validation failed: [initializer declared]");
 +        assertInvalidByteCode('\n' +
 +                              "return 0d;\n" +
 +                              "    }\n" +
 +                              '\n' +
 +                              "    static\n" +
 +                              "    {\n" +
 +                              "Math.sin(1d);", "Java UDF validation failed: [static initializer declared]");
 +        assertInvalidByteCode("synchronized (this)\n" +
 +                              "{\n" +
 +                              "    Math.sin(1d);\n" +
 +                              "}\n" +
 +                              "return 0d;", "Java UDF validation failed: [use of synchronized]");
 +        assertInvalidByteCode("synchronized (this)\n" +
 +                              "{\n" +
 +                              "    notify();\n" +
 +                              "}\n" +
-                               "return 0d;", "Java UDF validation failed: [call to notify(), use of synchronized]");
++                              "return 0d;", "Java UDF validation failed: [call to java.lang.Object.notify(), use of synchronized]");
 +        assertInvalidByteCode("synchronized (this)\n" +
 +                              "{\n" +
 +                              "    notifyAll();\n" +
 +                              "}\n" +
-                               "return 0d;", "Java UDF validation failed: [call to notifyAll(), use of synchronized]");
++                              "return 0d;", "Java UDF validation failed: [call to java.lang.Object.notifyAll(), use of synchronized]");
 +        assertInvalidByteCode("synchronized (this)\n" +
 +                              "{\n" +
 +                              "    try\n" +
 +                              "    {\n" +
 +                              "        wait();\n" +
 +                              "    }\n" +
 +                              "    catch (InterruptedException e)\n" +
 +                              "    {\n" +
 +                              "        throw new RuntimeException(e);\n" +
 +                              "    }\n" +
 +                              "}\n" +
-                               "return 0d;", "Java UDF validation failed: [call to wait(), use of synchronized]");
++                              "return 0d;", "Java UDF validation failed: [call to java.lang.Object.wait(), use of synchronized]");
 +        assertInvalidByteCode("synchronized (this)\n" +
 +                              "{\n" +
 +                              "    try\n" +
 +                              "    {\n" +
 +                              "        wait(1000L);\n" +
 +                              "    }\n" +
 +                              "    catch (InterruptedException e)\n" +
 +                              "    {\n" +
 +                              "        throw new RuntimeException(e);\n" +
 +                              "    }\n" +
 +                              "}\n" +
-                               "return 0d;", "Java UDF validation failed: [call to wait(), use of synchronized]");
++                              "return 0d;", "Java UDF validation failed: [call to java.lang.Object.wait(), use of synchronized]");
 +        assertInvalidByteCode("synchronized (this)\n" +
 +                              "{\n" +
 +                              "    try\n" +
 +                              "    {\n" +
 +                              "        wait(1000L, 100);\n" +
 +                              "    }\n" +
 +                              "    catch (InterruptedException e)\n" +
 +                              "    {\n" +
 +                              "        throw new RuntimeException(e);\n" +
 +                              "    }\n" +
 +                              "}\n" +
-                               "return 0d;", "Java UDF validation failed: [call to wait(), use of synchronized]");
++                              "return 0d;", "Java UDF validation failed: [call to java.lang.Object.wait(), use of synchronized]");
++        assertInvalidByteCode("try {" +
++                              "     java.nio.ByteBuffer.allocateDirect(123); return 0d;" +
++                              "} catch (Exception t) {" +
++                              "     throw new RuntimeException(t);" +
++                              '}', "Java UDF validation failed: [call to java.nio.ByteBuffer.allocateDirect()]");
++        assertInvalidByteCode("try {" +
++                              "     java.net.InetAddress.getLocalHost(); return 0d;" +
++                              "} catch (Exception t) {" +
++                              "     throw new RuntimeException(t);" +
++                              '}', "Java UDF validation failed: [call to java.net.InetAddress.getLocalHost()]");
++        assertInvalidByteCode("try {" +
++                              "     java.net.InetAddress.getAllByName(\"localhost\"); return 0d;" +
++                              "} catch (Exception t) {" +
++                              "     throw new RuntimeException(t);" +
++                              '}', "Java UDF validation failed: [call to java.net.InetAddress.getAllByName()]");
++        assertInvalidByteCode("try {" +
++                              "     java.net.Inet4Address.getByName(\"127.0.0.1\"); return 0d;" +
++                              "} catch (Exception t) {" +
++                              "     throw new RuntimeException(t);" +
++                              '}', "Java UDF validation failed: [call to java.net.Inet4Address.getByName()]");
++        assertInvalidByteCode("try {" +
++                              "     java.net.Inet6Address.getByAddress(new byte[]{127,0,0,1}); return 0d;" +
++                              "} catch (Exception t) {" +
++                              "     throw new RuntimeException(t);" +
++                              '}', "Java UDF validation failed: [call to java.net.Inet6Address.getByAddress()]");
++        assertInvalidByteCode("try {" +
++                              "     java.net.NetworkInterface.getNetworkInterfaces(); return 0d;" +
++                              "} catch (Exception t) {" +
++                              "     throw new RuntimeException(t);" +
++                              '}', "Java UDF validation failed: [call to java.net.NetworkInterface.getNetworkInterfaces()]");
 +    }
 +
 +    private void assertInvalidByteCode(String body, String error) throws Throwable
 +    {
 +        assertInvalidMessage(error,
 +                             "CREATE FUNCTION " + KEYSPACE + ".mustBeInvalid ( input double ) " +
 +                             "CALLED ON NULL INPUT " +
 +                             "RETURNS double " +
 +                             "LANGUAGE java AS $$" + body + "$$");
 +    }
 +}