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/07/12 11:02:10 UTC

[2/5] cassandra git commit: Revert CASSANDRA-9542 (allow native functions in UDA)

Revert CASSANDRA-9542 (allow native functions in UDA)

patch by Robert Stupp; reviewed by Aleksey Yeschenko for CASSANDRA-9771


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

Branch: refs/heads/trunk
Commit: fc202a756e666aff36811d995bb95956a1daeff8
Parents: 591216b
Author: Robert Stupp <sn...@snazy.de>
Authored: Sun Jul 12 10:51:25 2015 +0200
Committer: Robert Stupp <sn...@snazy.de>
Committed: Sun Jul 12 10:51:25 2015 +0200

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 doc/cql3/CQL.textile                            |  4 +-
 .../statements/CreateAggregateStatement.java    | 13 ++----
 .../cassandra/schema/LegacySchemaTables.java    | 25 ++--------
 .../validation/operations/AggregationTest.java  | 49 --------------------
 5 files changed, 10 insertions(+), 82 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/fc202a75/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index ced3a1e..15796c4 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 2.2.0-rc3
+ * Revert CASSANDRA-9542 (allow native functions in UDA) (CASSANDRA-9771)
 Merged from 2.1:
  * Fix clientutil jar and tests (CASSANDRA-9760)
  * (cqlsh) Allow the SSL protocol version to be specified through the

http://git-wip-us.apache.org/repos/asf/cassandra/blob/fc202a75/doc/cql3/CQL.textile
----------------------------------------------------------------------
diff --git a/doc/cql3/CQL.textile b/doc/cql3/CQL.textile
index 773ff91..b49d37b 100644
--- a/doc/cql3/CQL.textile
+++ b/doc/cql3/CQL.textile
@@ -708,9 +708,9 @@ Signatures for user-defined aggregates follow the "same rules":#functionSignatur
 
 The optional @INITCOND@ defines the initial state value for the aggregate. It defaults to @null@. A non-@null@ @INITCOND@ must be specified for state functions that are declared with @RETURNS NULL ON NULL INPUT@.
 
-@SFUNC@ references an existing function to be used as the state modifying function. The type of first argument of the state function must match @STYPE@. The remaining argument types of the state function must match the argument types of the aggregate function. State is not updated for state functions declared with @RETURNS NULL ON NULL INPUT@ and called with @null@. Functions from the system keyspace are resolved before functions in the current keyspace.
+@SFUNC@ references an existing function to be used as the state modifying function. The type of first argument of the state function must match @STYPE@. The remaining argument types of the state function must match the argument types of the aggregate function. State is not updated for state functions declared with @RETURNS NULL ON NULL INPUT@ and called with @null@.
 
-The optional @FINALFUNC@ is called just before the aggregate result is returned. It must take only one argument with type @STYPE@. The return type of the @FINALFUNC@ may be a different type. A final function declared with @RETURNS NULL ON NULL INPUT@ means that the aggregate's return value will be @null@, if the last state is @null@. Functions from the system keyspace are resolved before functions in the current keyspace.
+The optional @FINALFUNC@ is called just before the aggregate result is returned. It must take only one argument with type @STYPE@. The return type of the @FINALFUNC@ may be a different type. A final function declared with @RETURNS NULL ON NULL INPUT@ means that the aggregate's return value will be @null@, if the last state is @null@.
 
 If no @FINALFUNC@ is defined, the overall return type of the aggregate function is @STYPE@.  If a @FINALFUNC@ is defined, it is the return type of that function.
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/fc202a75/src/java/org/apache/cassandra/cql3/statements/CreateAggregateStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/CreateAggregateStatement.java b/src/java/org/apache/cassandra/cql3/statements/CreateAggregateStatement.java
index 039993f..1d73e3f 100644
--- a/src/java/org/apache/cassandra/cql3/statements/CreateAggregateStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/CreateAggregateStatement.java
@@ -88,7 +88,7 @@ public final class CreateAggregateStatement extends SchemaAlteringStatement
         AbstractType<?> stateType = prepareType("state type", stateTypeRaw);
 
         List<AbstractType<?>> stateArgs = stateArguments(stateType, argTypes);
-        stateFunc = validateFunctionKeyspace(stateFunc, stateArgs);
+        stateFunc = validateFunctionKeyspace(stateFunc);
 
         Function f = Functions.find(stateFunc, stateArgs);
         if (!(f instanceof ScalarFunction))
@@ -102,7 +102,7 @@ public final class CreateAggregateStatement extends SchemaAlteringStatement
         if (finalFunc != null)
         {
             List<AbstractType<?>> finalArgs = Collections.<AbstractType<?>>singletonList(stateType);
-            finalFunc = validateFunctionKeyspace(finalFunc, finalArgs);
+            finalFunc = validateFunctionKeyspace(finalFunc);
             f = Functions.find(finalFunc, finalArgs);
             if (!(f instanceof ScalarFunction))
                 throw new InvalidRequestException("Final function " + finalFunc + '(' + stateTypeRaw + ") does not exist or is not a scalar function");
@@ -150,17 +150,10 @@ public final class CreateAggregateStatement extends SchemaAlteringStatement
         ThriftValidation.validateKeyspaceNotSystem(functionName.keyspace);
     }
 
-    private FunctionName validateFunctionKeyspace(FunctionName func, List<AbstractType<?>> argTypes)
+    private FunctionName validateFunctionKeyspace(FunctionName func)
     {
         if (!func.hasKeyspace())
-        {
-            // If state/final function has no keyspace, check SYSTEM keyspace before logged keyspace.
-            FunctionName nativeName = FunctionName.nativeFunction(func.name);
-            if (Functions.find(nativeName, argTypes) != null)
-                return nativeName;
-
             return new FunctionName(functionName.keyspace, func.name);
-        }
         else if (!SystemKeyspace.NAME.equals(func.keyspace) && !functionName.keyspace.equals(func.keyspace))
             throw new InvalidRequestException(String.format("Statement on keyspace %s cannot refer to a user function in keyspace %s; "
                                                             + "user functions can only be used in the keyspace they are defined in",

http://git-wip-us.apache.org/repos/asf/cassandra/blob/fc202a75/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 b8f6421..1840829 100644
--- a/src/java/org/apache/cassandra/schema/LegacySchemaTables.java
+++ b/src/java/org/apache/cassandra/schema/LegacySchemaTables.java
@@ -1393,11 +1393,11 @@ public class LegacySchemaTables
 
         adder.resetCollection("argument_types");
         adder.add("return_type", aggregate.returnType().toString());
-        adder.add("state_func", aggregate.stateFunction().name().toString());
+        adder.add("state_func", aggregate.stateFunction().name().name);
         if (aggregate.stateType() != null)
             adder.add("state_type", aggregate.stateType().toString());
         if (aggregate.finalFunction() != null)
-            adder.add("final_func", aggregate.finalFunction().name().toString());
+            adder.add("final_func", aggregate.finalFunction().name().name);
         if (aggregate.initialCondition() != null)
             adder.add("initcond", aggregate.initialCondition());
 
@@ -1439,8 +1439,8 @@ public class LegacySchemaTables
 
         AbstractType<?> returnType = parseType(row.getString("return_type"));
 
-        FunctionName stateFunc = aggregateParseFunctionName(ksName, row.getString("state_func"));
-        FunctionName finalFunc = row.has("final_func") ? aggregateParseFunctionName(ksName, row.getString("final_func")) : null;
+        FunctionName stateFunc = new FunctionName(ksName, row.getString("state_func"));
+        FunctionName finalFunc = row.has("final_func") ? new FunctionName(ksName, row.getString("final_func")) : null;
         AbstractType<?> stateType = row.has("state_type") ? parseType(row.getString("state_type")) : null;
         ByteBuffer initcond = row.has("initcond") ? row.getBytes("initcond") : null;
 
@@ -1454,23 +1454,6 @@ public class LegacySchemaTables
         }
     }
 
-    private static FunctionName aggregateParseFunctionName(String ksName, String func)
-    {
-        int i = func.indexOf('.');
-
-        // function name can be abbreviated (pre 2.2rc2) - it is in the same keyspace as the aggregate
-        if (i == -1)
-            return new FunctionName(ksName, func);
-
-        String ks = func.substring(0, i);
-        String f = func.substring(i + 1);
-
-        // only aggregate's function keyspace and system keyspace are allowed
-        assert ks.equals(ksName) || ks.equals(SystemKeyspace.NAME);
-
-        return new FunctionName(ks, f);
-    }
-
     public static Mutation makeDropAggregateMutation(KSMetaData keyspace, UDAggregate aggregate, long timestamp)
     {
         // Include the serialized keyspace in case the target node missed a CREATE KEYSPACE migration (see CASSANDRA-5631).

http://git-wip-us.apache.org/repos/asf/cassandra/blob/fc202a75/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..7455dbc 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/AggregationTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/AggregationTest.java
@@ -33,7 +33,6 @@ import org.apache.cassandra.cql3.functions.UDAggregate;
 import org.apache.cassandra.cql3.CQLTester;
 import org.apache.cassandra.exceptions.FunctionExecutionException;
 import org.apache.cassandra.exceptions.InvalidRequestException;
-import org.apache.cassandra.serializers.Int32Serializer;
 import org.apache.cassandra.service.ClientState;
 import org.apache.cassandra.transport.Event;
 import org.apache.cassandra.transport.messages.ResultMessage;
@@ -1117,26 +1116,6 @@ public class AggregationTest extends CQLTester
     }
 
     @Test
-    public void testSystemKeyspace() throws Throwable
-    {
-        String fState = createFunction(KEYSPACE,
-                                       "text, text",
-                                       "CREATE FUNCTION %s(a text, b text) " +
-                                       "CALLED ON NULL INPUT " +
-                                       "RETURNS text " +
-                                       "LANGUAGE java " +
-                                       "AS 'return \"foobar\";'");
-
-        createAggregate(KEYSPACE,
-                        "text",
-                        "CREATE AGGREGATE %s(text) " +
-                        "SFUNC " + shortFunctionName(fState) + ' ' +
-                        "STYPE text " +
-                        "FINALFUNC system.varcharasblob " +
-                        "INITCOND 'foobar'");
-    }
-
-    @Test
     public void testFunctionWithFrozenSetType() throws Throwable
     {
         createTable("CREATE TABLE %s (a int PRIMARY KEY, b frozen<set<int>>)");
@@ -1450,32 +1429,4 @@ public class AggregationTest extends CQLTester
         assertRows(execute("SELECT " + aRNON + "(b) FROM %s"), row("fin"));
 
     }
-
-    @Test
-    public void testSystemKsFuncs() throws Throwable
-    {
-
-        String fAdder = createFunction(KEYSPACE,
-                                      "int, int",
-                                      "CREATE FUNCTION %s(a int, b int) " +
-                                      "CALLED ON NULL INPUT " +
-                                      "RETURNS int " +
-                                      "LANGUAGE java " +
-                                      "AS 'return (a != null ? a : 0) + (b != null ? b : 0);'");
-
-        String aAggr = createAggregate(KEYSPACE,
-                                      "int",
-                                      "CREATE AGGREGATE %s(int) " +
-                                      "SFUNC " + shortFunctionName(fAdder) + ' ' +
-                                      "STYPE int " +
-                                      "FINALFUNC intasblob");
-
-        createTable("CREATE TABLE %s (a int primary key, b int)");
-        execute("INSERT INTO %s (a, b) VALUES (1, 1)");
-        execute("INSERT INTO %s (a, b) VALUES (2, 2)");
-        execute("INSERT INTO %s (a, b) VALUES (3, 3)");
-
-        assertRows(execute("SELECT " + aAggr + "(b) FROM %s"), row(Int32Serializer.instance.serialize(6)));
-
-    }
 }