Repository: cassandra
Updated Branches:
  refs/heads/trunk 67db844df -> 356684350


http://git-wip-us.apache.org/repos/asf/cassandra/blob/35668435/src/java/org/apache/cassandra/schema/Functions.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/schema/Functions.java 
b/src/java/org/apache/cassandra/schema/Functions.java
new file mode 100644
index 0000000..8d73e48
--- /dev/null
+++ b/src/java/org/apache/cassandra/schema/Functions.java
@@ -0,0 +1,216 @@
+/*
+ * 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.schema;
+
+import java.util.*;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import com.google.common.collect.ImmutableMultimap;
+
+import org.apache.cassandra.cql3.functions.*;
+import org.apache.cassandra.db.marshal.AbstractType;
+
+import static com.google.common.collect.Iterables.filter;
+
+/**
+ * An immutable container for a keyspace's UDAs and UDFs (and, in case of 
{@link org.apache.cassandra.db.SystemKeyspace},
+ * native functions and aggregates).
+ */
+public final class Functions implements Iterable<Function>
+{
+    private final ImmutableMultimap<FunctionName, Function> functions;
+
+    private Functions(Builder builder)
+    {
+        functions = builder.functions.build();
+    }
+
+    public static Builder builder()
+    {
+        return new Builder();
+    }
+
+    public static Functions none()
+    {
+        return builder().build();
+    }
+
+    public Iterator<Function> iterator()
+    {
+        return functions.values().iterator();
+    }
+
+    public Stream<Function> stream()
+    {
+        return functions.values().stream();
+    }
+
+    /**
+     * @return a stream of keyspace's UDFs
+     */
+    public Stream<UDFunction> udfs()
+    {
+        return stream().filter(f -> f instanceof UDFunction).map(f -> 
(UDFunction) f);
+    }
+
+    /**
+     * @return a stream of keyspace's UDAs
+     */
+    public Stream<UDAggregate> udas()
+    {
+        return stream().filter(f -> f instanceof UDAggregate).map(f -> 
(UDAggregate) f);
+    }
+
+    /**
+     * @return a collection of aggregates that use the provided function as 
either a state or a final function
+     * @param function the referree function
+     */
+    public Collection<UDAggregate> aggregatesUsingFunction(Function function)
+    {
+        return udas().filter(uda -> 
uda.hasReferenceTo(function)).collect(Collectors.toList());
+    }
+
+    /**
+     * Get all function overloads with the specified name
+     *
+     * @param name fully qualified function name
+     * @return an empty list if the function name is not found; a non-empty 
collection of {@link Function} otherwise
+     */
+    public Collection<Function> get(FunctionName name)
+    {
+        return functions.get(name);
+    }
+
+    /**
+     * Find the function with the specified name
+     *
+     * @param name fully qualified function name
+     * @param argTypes function argument types
+     * @return an empty {@link Optional} if the function name is not found; a 
non-empty optional of {@link Function} otherwise
+     */
+    public Optional<Function> find(FunctionName name, List<AbstractType<?>> 
argTypes)
+    {
+        return get(name).stream()
+                        .filter(fun -> typesMatch(fun.argTypes(), argTypes))
+                        .findAny();
+    }
+
+    /*
+     * We need to compare the CQL3 representation of the type because comparing
+     * the AbstractType will fail for example if a UDT has been changed.
+     * Reason is that UserType.equals() takes the field names and types into 
account.
+     * Example CQL sequence that would fail when comparing AbstractType:
+     *    CREATE TYPE foo ...
+     *    CREATE FUNCTION bar ( par foo ) RETURNS foo ...
+     *    ALTER TYPE foo ADD ...
+     * or
+     *    ALTER TYPE foo ALTER ...
+     * or
+     *    ALTER TYPE foo RENAME ...
+     */
+    public static boolean typesMatch(AbstractType<?> t1, AbstractType<?> t2)
+    {
+        return t1.asCQL3Type().toString().equals(t2.asCQL3Type().toString());
+    }
+
+    public static boolean typesMatch(List<AbstractType<?>> t1, 
List<AbstractType<?>> t2)
+    {
+        if (t1.size() != t2.size())
+            return false;
+
+        for (int i = 0; i < t1.size(); i++)
+            if (!typesMatch(t1.get(i), t2.get(i)))
+                return false;
+
+        return true;
+    }
+
+    /**
+     * Create a Functions instance with the provided function added
+     */
+    public Functions with(Function fun)
+    {
+        if (find(fun.name(), fun.argTypes()).isPresent())
+            throw new IllegalStateException(String.format("Function %s already 
exists", fun.name()));
+
+        return builder().add(this).add(fun).build();
+    }
+
+    /**
+     * Creates a Functions instance with the function with the provided name 
and argument types removed
+     */
+    public Functions without(FunctionName name, List<AbstractType<?>> argTypes)
+    {
+        Function fun =
+            find(name, argTypes).orElseThrow(() -> new 
IllegalStateException(String.format("Function %s doesn't exists", name)));
+
+        return builder().add(filter(this, f -> f != fun)).build();
+    }
+
+    @Override
+    public boolean equals(Object o)
+    {
+        return this == o || (o instanceof Functions && 
functions.equals(((Functions) o).functions));
+    }
+
+    @Override
+    public int hashCode()
+    {
+        return functions.hashCode();
+    }
+
+    @Override
+    public String toString()
+    {
+        return functions.values().toString();
+    }
+
+    public static final class Builder
+    {
+        final ImmutableMultimap.Builder<FunctionName, Function> functions = 
new ImmutableMultimap.Builder<>();
+
+        private Builder()
+        {
+        }
+
+        public Functions build()
+        {
+            return new Functions(this);
+        }
+
+        public Builder add(Function fun)
+        {
+            functions.put(fun.name(), fun);
+            return this;
+        }
+
+        public Builder add(Function... funs)
+        {
+            for (Function fun : funs)
+                add(fun);
+            return this;
+        }
+
+        public  Builder add(Iterable<? extends Function> funs)
+        {
+            funs.forEach(this::add);
+            return this;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/35668435/src/java/org/apache/cassandra/schema/LegacySchemaTables.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/schema/LegacySchemaTables.java 
b/src/java/org/apache/cassandra/schema/LegacySchemaTables.java
index 1348d12..42c67db 100644
--- a/src/java/org/apache/cassandra/schema/LegacySchemaTables.java
+++ b/src/java/org/apache/cassandra/schema/LegacySchemaTables.java
@@ -55,8 +55,12 @@ import static 
org.apache.cassandra.utils.FBUtilities.fromJsonMap;
 import static org.apache.cassandra.utils.FBUtilities.json;
 
 /** system.schema_* tables used to store keyspace/table/type attributes prior 
to C* 3.0 */
-public class LegacySchemaTables
+public final class LegacySchemaTables
 {
+    private LegacySchemaTables()
+    {
+    }
+
     private static final Logger logger = 
LoggerFactory.getLogger(LegacySchemaTables.class);
 
     public static final String KEYSPACES = "schema_keyspaces";
@@ -220,17 +224,10 @@ public class LegacySchemaTables
                     DecoratedKey key = partition.partitionKey();
 
                     readSchemaPartitionForKeyspaceAndApply(USERTYPES, key,
-                        types -> 
readSchemaPartitionForKeyspaceAndApply(COLUMNFAMILIES, key, tables -> 
keyspaces.add(createKeyspaceFromSchemaPartitions(partition, tables, types)))
-                    );
-
-                    // Will be moved away in #6717
-                    readSchemaPartitionForKeyspaceAndApply(FUNCTIONS, key,
-                        functions -> { 
createFunctionsFromFunctionsPartition(functions).forEach(function -> 
org.apache.cassandra.cql3.functions.Functions.addOrReplaceFunction(function)); 
return null; }
-                    );
-
-                    // Will be moved away in #6717
-                    readSchemaPartitionForKeyspaceAndApply(AGGREGATES, key,
-                        aggregates -> { 
createAggregatesFromAggregatesPartition(aggregates).forEach(aggregate -> 
org.apache.cassandra.cql3.functions.Functions.addOrReplaceFunction(aggregate)); 
return null; }
+                        types -> 
readSchemaPartitionForKeyspaceAndApply(COLUMNFAMILIES, key,
+                        tables -> 
readSchemaPartitionForKeyspaceAndApply(FUNCTIONS, key,
+                        functions -> 
readSchemaPartitionForKeyspaceAndApply(AGGREGATES, key,
+                        aggregates -> 
keyspaces.add(createKeyspaceFromSchemaPartitions(partition, tables, types, 
functions, aggregates)))))
                     );
                 }
             }
@@ -651,7 +648,7 @@ public class LegacySchemaTables
         return makeCreateKeyspaceMutation(keyspace, timestamp, true);
     }
 
-    private static Mutation makeCreateKeyspaceMutation(KSMetaData keyspace, 
long timestamp, boolean withTablesAndTypesAndFunctions)
+    public static Mutation makeCreateKeyspaceMutation(KSMetaData keyspace, 
long timestamp, boolean withTablesAndTypesAndFunctions)
     {
         // Note that because Keyspaces is a COMPACT TABLE, we're really only 
setting static columns internally and shouldn't set any clustering.
         RowUpdateBuilder adder = new RowUpdateBuilder(Keyspaces, timestamp, 
keyspace.name);
@@ -664,11 +661,10 @@ public class LegacySchemaTables
 
         if (withTablesAndTypesAndFunctions)
         {
-            for (UserType type : keyspace.userTypes.getAllTypes().values())
-                addTypeToSchemaMutation(type, timestamp, mutation);
-
-            for (CFMetaData table : keyspace.cfMetaData().values())
-                addTableToSchemaMutation(table, timestamp, true, mutation);
+            keyspace.userTypes.getAllTypes().values().forEach(type -> 
addTypeToSchemaMutation(type, timestamp, mutation));
+            keyspace.cfMetaData().values().forEach(table -> 
addTableToSchemaMutation(table, timestamp, true, mutation));
+            keyspace.functions.udfs().forEach(udf -> 
addFunctionToSchemaMutation(udf, timestamp, mutation));
+            keyspace.functions.udas().forEach(uda -> 
addAggregateToSchemaMutation(uda, timestamp, mutation));
         }
 
         return mutation;
@@ -684,11 +680,20 @@ public class LegacySchemaTables
         return mutation;
     }
 
-    private static KSMetaData createKeyspaceFromSchemaPartitions(RowIterator 
serializedKeyspace, RowIterator serializedTables, RowIterator serializedTypes)
+    private static KSMetaData createKeyspaceFromSchemaPartitions(RowIterator 
serializedKeyspace,
+                                                                 RowIterator 
serializedTables,
+                                                                 RowIterator 
serializedTypes,
+                                                                 RowIterator 
serializedFunctions,
+                                                                 RowIterator 
seriazliedAggregates)
     {
         Collection<CFMetaData> tables = 
createTablesFromTablesPartition(serializedTables);
         UTMetaData types = new 
UTMetaData(createTypesFromPartition(serializedTypes));
-        return 
createKeyspaceFromSchemaPartition(serializedKeyspace).cloneWith(tables, types);
+
+        Collection<UDFunction> udfs = 
createFunctionsFromFunctionsPartition(serializedFunctions);
+        Collection<UDAggregate> udas = 
createAggregatesFromAggregatesPartition(seriazliedAggregates);
+        Functions functions = 
org.apache.cassandra.schema.Functions.builder().add(udfs).add(udas).build();
+
+        return 
createKeyspaceFromSchemaPartition(serializedKeyspace).cloneWith(tables, types, 
functions);
     }
 
     public static KSMetaData createKeyspaceFromName(String keyspace)
@@ -1360,7 +1365,7 @@ public class LegacySchemaTables
         String body = row.getString("body");
         boolean calledOnNullInput = row.getBoolean("called_on_null_input");
 
-        org.apache.cassandra.cql3.functions.Function existing = 
org.apache.cassandra.cql3.functions.Functions.find(name, argTypes);
+        org.apache.cassandra.cql3.functions.Function existing = 
Schema.instance.findFunction(name, argTypes).orElse(null);
         if (existing instanceof UDFunction)
         {
             // This check prevents duplicate compilation of effectively the 
same UDF.
@@ -1513,15 +1518,4 @@ public class LegacySchemaTables
             strList.add(argType.asCQL3Type().toString());
         return list.decompose(strList);
     }
-
-    public static ByteBuffer 
functionSignatureWithNameAndTypes(AbstractFunction fun)
-    {
-        ListType<String> list = ListType.getInstance(UTF8Type.instance, false);
-        List<String> strList = new ArrayList<>(fun.argTypes().size() + 2);
-        strList.add(fun.name().keyspace);
-        strList.add(fun.name().name);
-        for (AbstractType<?> argType : fun.argTypes())
-            strList.add(argType.asCQL3Type().toString());
-        return list.decompose(strList);
-    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/35668435/src/java/org/apache/cassandra/service/MigrationManager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/MigrationManager.java 
b/src/java/org/apache/cassandra/service/MigrationManager.java
index 9087672..3b95083 100644
--- a/src/java/org/apache/cassandra/service/MigrationManager.java
+++ b/src/java/org/apache/cassandra/service/MigrationManager.java
@@ -182,7 +182,6 @@ public class MigrationManager
             listener.onCreateFunction(udf.name().keyspace, udf.name().name, 
udf.argTypes());
     }
 
-
     public void notifyCreateAggregate(UDAggregate udf)
     {
         for (MigrationListener listener : listeners)
@@ -205,6 +204,9 @@ public class MigrationManager
     {
         for (MigrationListener listener : listeners)
             listener.onUpdateUserType(ut.keyspace, ut.getNameAsString());
+
+        // FIXME: remove when we get rid of AbstractType in metadata. Doesn't 
really belong anywhere.
+        Schema.instance.getKSMetaData(ut.keyspace).functions.udfs().forEach(f 
-> f.userTypeUpdated(ut.keyspace, ut.getNameAsString()));
     }
 
     public void notifyUpdateFunction(UDFunction udf)
@@ -323,7 +325,7 @@ public class MigrationManager
             throw new ConfigurationException(String.format("Cannot update non 
existing keyspace '%s'.", ksm.name));
 
         logger.info(String.format("Update Keyspace '%s' From %s To %s", 
ksm.name, oldKsm, ksm));
-        announce(LegacySchemaTables.makeCreateKeyspaceMutation(ksm, 
FBUtilities.timestampMicros()), announceLocally);
+        announce(LegacySchemaTables.makeCreateKeyspaceMutation(ksm, 
FBUtilities.timestampMicros(), false), announceLocally);
     }
 
     public static void announceColumnFamilyUpdate(CFMetaData cfm, boolean 
fromThrift) throws ConfigurationException

http://git-wip-us.apache.org/repos/asf/cassandra/blob/35668435/test/unit/org/apache/cassandra/cql3/validation/entities/UFAuthTest.java
----------------------------------------------------------------------
diff --git 
a/test/unit/org/apache/cassandra/cql3/validation/entities/UFAuthTest.java 
b/test/unit/org/apache/cassandra/cql3/validation/entities/UFAuthTest.java
index ee20557..25fe227 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/entities/UFAuthTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/UFAuthTest.java
@@ -30,12 +30,12 @@ import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.auth.*;
 import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.cql3.Attributes;
 import org.apache.cassandra.cql3.CQLStatement;
 import org.apache.cassandra.cql3.QueryProcessor;
 import org.apache.cassandra.cql3.functions.Function;
 import org.apache.cassandra.cql3.functions.FunctionName;
-import org.apache.cassandra.cql3.functions.Functions;
 import org.apache.cassandra.cql3.statements.BatchStatement;
 import org.apache.cassandra.cql3.statements.ModificationStatement;
 import org.apache.cassandra.cql3.CQLTester;
@@ -618,12 +618,12 @@ public class UFAuthTest extends CQLTester
         // It is here to avoid having to duplicate the functionality of 
CqlParser
         // for transforming cql types into AbstractTypes
         FunctionName fn = parseFunctionName(functionName);
-        List<Function> functions = Functions.find(fn);
+        Collection<Function> functions = Schema.instance.getFunctions(fn);
         assertEquals(String.format("Expected a single function definition for 
%s, but found %s",
                                    functionName,
                                    functions.size()),
                      1, functions.size());
-        return FunctionResource.function(fn.keyspace, fn.name, 
functions.get(0).argTypes());
+        return FunctionResource.function(fn.keyspace, fn.name, 
functions.iterator().next().argTypes());
     }
 
     private String functionCall(String functionName, String...args)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/35668435/test/unit/org/apache/cassandra/cql3/validation/entities/UFTest.java
----------------------------------------------------------------------
diff --git 
a/test/unit/org/apache/cassandra/cql3/validation/entities/UFTest.java 
b/test/unit/org/apache/cassandra/cql3/validation/entities/UFTest.java
index 7bd208f..e9436f8 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/entities/UFTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/UFTest.java
@@ -37,10 +37,11 @@ import org.junit.Test;
 import com.datastax.driver.core.*;
 import com.datastax.driver.core.exceptions.InvalidQueryException;
 import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.config.KSMetaData;
+import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.cql3.QueryProcessor;
 import org.apache.cassandra.cql3.UntypedResultSet;
 import org.apache.cassandra.cql3.functions.FunctionName;
-import org.apache.cassandra.cql3.functions.Functions;
 import org.apache.cassandra.cql3.functions.UDFunction;
 import org.apache.cassandra.cql3.CQLTester;
 import org.apache.cassandra.db.marshal.CollectionType;
@@ -119,7 +120,7 @@ public class UFTest extends CQLTester
 
         FunctionName fSinName = parseFunctionName(fSin);
 
-        Assert.assertEquals(1, Functions.find(parseFunctionName(fSin)).size());
+        Assert.assertEquals(1, 
Schema.instance.getFunctions(parseFunctionName(fSin)).size());
 
         assertRows(execute("SELECT function_name, language FROM 
system.schema_functions WHERE keyspace_name=?", KEYSPACE_PER_TEST),
                    row(fSinName.name, "java"));
@@ -128,7 +129,7 @@ public class UFTest extends CQLTester
 
         assertRows(execute("SELECT function_name, language FROM 
system.schema_functions WHERE keyspace_name=?", KEYSPACE_PER_TEST));
 
-        Assert.assertEquals(0, Functions.find(fSinName).size());
+        Assert.assertEquals(0, Schema.instance.getFunctions(fSinName).size());
     }
 
     @Test
@@ -145,7 +146,7 @@ public class UFTest extends CQLTester
 
         FunctionName fSinName = parseFunctionName(fSin);
 
-        Assert.assertEquals(1, Functions.find(parseFunctionName(fSin)).size());
+        Assert.assertEquals(1, 
Schema.instance.getFunctions(parseFunctionName(fSin)).size());
 
         // create a pairs of Select and Inserts. One statement in each pair 
uses the function so when we
         // drop it those statements should be removed from the cache in 
QueryProcessor. The other statements
@@ -183,7 +184,7 @@ public class UFTest extends CQLTester
                 "LANGUAGE java " +
                 "AS 'return Double.valueOf(Math.sin(input));'");
 
-        Assert.assertEquals(1, Functions.find(fSinName).size());
+        Assert.assertEquals(1, Schema.instance.getFunctions(fSinName).size());
 
         preparedSelect1= QueryProcessor.prepare(
                                          String.format("SELECT key, %s(d) FROM 
%s.%s", fSin, KEYSPACE, currentTable()),
@@ -298,7 +299,7 @@ public class UFTest extends CQLTester
                                         "RETURNS double " +
                                         "LANGUAGE javascript " +
                                         "AS 'input'");
-        Assert.assertEquals(1, 
Functions.find(parseFunctionName(function)).size());
+        Assert.assertEquals(1, 
Schema.instance.getFunctions(parseFunctionName(function)).size());
 
         List<ResultMessage.Prepared> prepared = new ArrayList<>();
         // prepare statements which use the function to provide a DelayedValue
@@ -1292,7 +1293,7 @@ public class UFTest extends CQLTester
 
         FunctionName fNameName = parseFunctionName(fName);
 
-        Assert.assertEquals(1, Functions.find(fNameName).size());
+        Assert.assertEquals(1, Schema.instance.getFunctions(fNameName).size());
 
         ResultMessage.Prepared prepared = 
QueryProcessor.prepare(String.format("SELECT key, %s(udt) FROM %s.%s", fName, 
KEYSPACE, currentTable()),
                                                                  
ClientState.forInternalCalls(), false);
@@ -1309,7 +1310,7 @@ public class UFTest extends CQLTester
         
Assert.assertNull(QueryProcessor.instance.getPrepared(prepared.statementId));
 
         // function stays
-        Assert.assertEquals(1, Functions.find(fNameName).size());
+        Assert.assertEquals(1, Schema.instance.getFunctions(fNameName).size());
     }
 
     @Test
@@ -1422,7 +1423,7 @@ public class UFTest extends CQLTester
                               "AS $$return " +
                               "     udt.getString(\"txt\");$$;",
                               fName1replace, type));
-        Assert.assertEquals(1, 
Functions.find(parseFunctionName(fName1replace)).size());
+        Assert.assertEquals(1, 
Schema.instance.getFunctions(parseFunctionName(fName1replace)).size());
         execute(String.format("CREATE OR REPLACE FUNCTION %s( udt %s ) " +
                               "CALLED ON NULL INPUT " +
                               "RETURNS int " +
@@ -1430,7 +1431,7 @@ public class UFTest extends CQLTester
                               "AS $$return " +
                               "     Integer.valueOf(udt.getInt(\"i\"));$$;",
                               fName2replace, type));
-        Assert.assertEquals(1, 
Functions.find(parseFunctionName(fName2replace)).size());
+        Assert.assertEquals(1, 
Schema.instance.getFunctions(parseFunctionName(fName2replace)).size());
         execute(String.format("CREATE OR REPLACE FUNCTION %s( udt %s ) " +
                               "CALLED ON NULL INPUT " +
                               "RETURNS double " +
@@ -1438,7 +1439,7 @@ public class UFTest extends CQLTester
                               "AS $$return " +
                               "     
Double.valueOf(udt.getDouble(\"added\"));$$;",
                               fName3replace, type));
-        Assert.assertEquals(1, 
Functions.find(parseFunctionName(fName3replace)).size());
+        Assert.assertEquals(1, 
Schema.instance.getFunctions(parseFunctionName(fName3replace)).size());
         execute(String.format("CREATE OR REPLACE FUNCTION %s( udt %s ) " +
                               "RETURNS NULL ON NULL INPUT " +
                               "RETURNS %s " +
@@ -1446,7 +1447,7 @@ public class UFTest extends CQLTester
                               "AS $$return " +
                               "     udt;$$;",
                               fName4replace, type, type));
-        Assert.assertEquals(1, 
Functions.find(parseFunctionName(fName4replace)).size());
+        Assert.assertEquals(1, 
Schema.instance.getFunctions(parseFunctionName(fName4replace)).size());
 
         assertRows(execute("SELECT " + fName1replace + "(udt) FROM %s WHERE 
key = 2"),
                    row("two"));
@@ -2174,10 +2175,18 @@ public class UFTest extends CQLTester
                                       "LANGUAGE JAVA\n" +
                                       "AS 'throw new RuntimeException();';");
 
-        UDFunction f = (UDFunction) 
Functions.find(parseFunctionName(fName)).get(0);
-
-        
Functions.addOrReplaceFunction(UDFunction.createBrokenFunction(f.name(), 
f.argNames(), f.argTypes(), f.returnType(), true,
-                                                                       "java", 
f.body(), new InvalidRequestException("foo bar is broken")));
+        KSMetaData ksm = Schema.instance.getKSMetaData(KEYSPACE_PER_TEST);
+        UDFunction f = (UDFunction) 
ksm.functions.get(parseFunctionName(fName)).iterator().next();
+
+        UDFunction broken = UDFunction.createBrokenFunction(f.name(),
+                                                            f.argNames(),
+                                                            f.argTypes(),
+                                                            f.returnType(),
+                                                            true,
+                                                            "java",
+                                                            f.body(),
+                                                            new 
InvalidRequestException("foo bar is broken"));
+        
Schema.instance.setKeyspaceDefinition(ksm.cloneWith(ksm.functions.without(f.name(),
 f.argTypes()).with(broken)));
 
         assertInvalidThrowMessage("foo bar is broken", 
InvalidRequestException.class,
                                   "SELECT key, " + fName + "(dval) FROM %s");

http://git-wip-us.apache.org/repos/asf/cassandra/blob/35668435/test/unit/org/apache/cassandra/cql3/validation/operations/AggregationTest.java
----------------------------------------------------------------------
diff --git 
a/test/unit/org/apache/cassandra/cql3/validation/operations/AggregationTest.java
 
b/test/unit/org/apache/cassandra/cql3/validation/operations/AggregationTest.java
index 3f6fdda..a7ce3a6 100644
--- 
a/test/unit/org/apache/cassandra/cql3/validation/operations/AggregationTest.java
+++ 
b/test/unit/org/apache/cassandra/cql3/validation/operations/AggregationTest.java
@@ -27,8 +27,9 @@ import org.apache.commons.lang3.time.DateUtils;
 import org.junit.Assert;
 import org.junit.Test;
 
+import org.apache.cassandra.config.KSMetaData;
+import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.cql3.QueryProcessor;
-import org.apache.cassandra.cql3.functions.Functions;
 import org.apache.cassandra.cql3.functions.UDAggregate;
 import org.apache.cassandra.cql3.CQLTester;
 import org.apache.cassandra.exceptions.FunctionExecutionException;
@@ -1014,10 +1015,16 @@ public class AggregationTest extends CQLTester
                                    "SFUNC " + shortFunctionName(fState) + " " +
                                    "STYPE int ");
 
-        UDAggregate f = (UDAggregate) 
Functions.find(parseFunctionName(a)).get(0);
+        KSMetaData ksm = Schema.instance.getKSMetaData(keyspace());
+        UDAggregate f = (UDAggregate) 
ksm.functions.get(parseFunctionName(a)).iterator().next();
 
-        Functions.addOrReplaceFunction(UDAggregate.createBroken(f.name(), 
f.argTypes(), f.returnType(),
-                                                                null, new 
InvalidRequestException("foo bar is broken")));
+        UDAggregate broken = UDAggregate.createBroken(f.name(),
+                                                      f.argTypes(),
+                                                      f.returnType(),
+                                                      null,
+                                                      new 
InvalidRequestException("foo bar is broken"));
+
+        
Schema.instance.setKeyspaceDefinition(ksm.cloneWith(ksm.functions.without(f.name(),
 f.argTypes()).with(broken)));
 
         assertInvalidThrowMessage("foo bar is broken", 
InvalidRequestException.class,
                                   "SELECT " + a + "(val) FROM %s");

Reply via email to