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/05/19 23:50:44 UTC

[2/3] cassandra git commit: Ensure that UDF and UDAs are keyspace-isolated

Ensure that UDF and UDAs are keyspace-isolated

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


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

Branch: refs/heads/trunk
Commit: 6eea3ea395957d1e799952de3423cc29fd469e3e
Parents: d085ebb
Author: Robert Stupp <sn...@snazy.de>
Authored: Tue May 19 23:47:46 2015 +0200
Committer: Robert Stupp <sn...@snazy.de>
Committed: Tue May 19 23:47:46 2015 +0200

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../org/apache/cassandra/cql3/CQL3Type.java     |  3 +-
 src/java/org/apache/cassandra/cql3/Cql.g        |  4 +-
 .../statements/CreateAggregateStatement.java    | 35 +++++---
 .../statements/CreateFunctionStatement.java     | 14 +--
 .../apache/cassandra/cql3/AggregationTest.java  | 93 ++++++++++++++++++--
 test/unit/org/apache/cassandra/cql3/UFTest.java | 39 ++++----
 7 files changed, 144 insertions(+), 45 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/6eea3ea3/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 0ce90ce..4e79ea0 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 2.2
+ * Ensure that UDF and UDAs are keyspace-isolated (CASSANDRA-9409)
 Merged from 2.1:
  * Use configured gcgs in anticompaction (CASSANDRA-9397)
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6eea3ea3/src/java/org/apache/cassandra/cql3/CQL3Type.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/CQL3Type.java b/src/java/org/apache/cassandra/cql3/CQL3Type.java
index fb2716e..e3ece93 100644
--- a/src/java/org/apache/cassandra/cql3/CQL3Type.java
+++ b/src/java/org/apache/cassandra/cql3/CQL3Type.java
@@ -22,6 +22,7 @@ import java.util.List;
 
 import org.apache.cassandra.config.KSMetaData;
 import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.db.SystemKeyspace;
 import org.apache.cassandra.db.marshal.*;
 import org.apache.cassandra.exceptions.InvalidRequestException;
 import org.apache.cassandra.exceptions.ConfigurationException;
@@ -499,7 +500,7 @@ public interface CQL3Type
                 {
                     // The provided keyspace is the one of the current statement this is part of. If it's different from the keyspace of
                     // the UTName, we reject since we want to limit user types to their own keyspace (see #6643)
-                    if (keyspace != null && !keyspace.equals(name.getKeyspace()))
+                    if (keyspace != null && !SystemKeyspace.NAME.equals(name.getKeyspace()) && !keyspace.equals(name.getKeyspace()))
                         throw new InvalidRequestException(String.format("Statement on keyspace %s cannot refer to a user type in keyspace %s; "
                                                                         + "user types can only be used in the keyspace they are defined in",
                                                                         keyspace, name.getKeyspace()));

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6eea3ea3/src/java/org/apache/cassandra/cql3/Cql.g
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/Cql.g b/src/java/org/apache/cassandra/cql3/Cql.g
index 3600cd1..ec63183 100644
--- a/src/java/org/apache/cassandra/cql3/Cql.g
+++ b/src/java/org/apache/cassandra/cql3/Cql.g
@@ -555,10 +555,10 @@ createAggregateStatement returns [CreateAggregateStatement expr]
           ( ',' v=comparatorType { argsTypes.add(v); } )*
         )?
       ')'
-      K_SFUNC sfunc = allowedFunctionName
+      K_SFUNC sfunc = functionName
       K_STYPE stype = comparatorType
       (
-        K_FINALFUNC ffunc = allowedFunctionName
+        K_FINALFUNC ffunc = functionName
       )?
       (
         K_INITCOND ival = term

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6eea3ea3/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 2524a7c..7682184 100644
--- a/src/java/org/apache/cassandra/cql3/statements/CreateAggregateStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/CreateAggregateStatement.java
@@ -27,6 +27,7 @@ import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.cql3.*;
 import org.apache.cassandra.cql3.functions.*;
+import org.apache.cassandra.db.SystemKeyspace;
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.exceptions.*;
 import org.apache.cassandra.service.ClientState;
@@ -43,8 +44,8 @@ public final class CreateAggregateStatement extends SchemaAlteringStatement
     private final boolean orReplace;
     private final boolean ifNotExists;
     private FunctionName functionName;
-    private final String stateFunc;
-    private final String finalFunc;
+    private FunctionName stateFunc;
+    private FunctionName finalFunc;
     private final CQL3Type.Raw stateTypeRaw;
 
     private final List<CQL3Type.Raw> argRawTypes;
@@ -61,9 +62,9 @@ public final class CreateAggregateStatement extends SchemaAlteringStatement
 
     public CreateAggregateStatement(FunctionName functionName,
                                     List<CQL3Type.Raw> argRawTypes,
-                                    String stateFunc,
+                                    FunctionName stateFunc,
                                     CQL3Type.Raw stateType,
-                                    String finalFunc,
+                                    FunctionName finalFunc,
                                     Term.Raw ival,
                                     boolean orReplace,
                                     boolean ifNotExists)
@@ -85,10 +86,9 @@ public final class CreateAggregateStatement extends SchemaAlteringStatement
             argTypes.add(rawType.prepare(functionName.keyspace).getType());
 
         AbstractType<?> stateType = stateTypeRaw.prepare(functionName.keyspace).getType();
-        FunctionName stateFuncName = new FunctionName(functionName.keyspace, stateFunc);
-        Function f = Functions.find(stateFuncName, stateArguments(stateType, argTypes));
+        Function f = Functions.find(stateFunc, stateArguments(stateType, argTypes));
         if (!(f instanceof ScalarFunction))
-            throw new InvalidRequestException("State function " + stateFuncSig(stateFuncName, stateTypeRaw, argRawTypes) + " does not exist or is not a scalar function");
+            throw new InvalidRequestException("State function " + stateFuncSig(stateFunc, stateTypeRaw, argRawTypes) + " does not exist or is not a scalar function");
         stateFunction = (ScalarFunction)f;
 
         AbstractType<?> stateReturnType = stateFunction.returnType();
@@ -97,10 +97,9 @@ public final class CreateAggregateStatement extends SchemaAlteringStatement
 
         if (finalFunc != null)
         {
-            FunctionName finalFuncName = new FunctionName(functionName.keyspace, finalFunc);
-            f = Functions.find(finalFuncName, Collections.<AbstractType<?>>singletonList(stateType));
+            f = Functions.find(finalFunc, Collections.<AbstractType<?>>singletonList(stateType));
             if (!(f instanceof ScalarFunction))
-                throw new InvalidRequestException("Final function " + finalFuncName + '(' + stateTypeRaw + ") does not exist or is not a scalar function");
+                throw new InvalidRequestException("Final function " + finalFunc + '(' + stateTypeRaw + ") does not exist or is not a scalar function");
             finalFunction = (ScalarFunction) f;
             returnType = finalFunction.returnType();
         }
@@ -126,9 +125,25 @@ public final class CreateAggregateStatement extends SchemaAlteringStatement
         if (!functionName.hasKeyspace())
             throw new InvalidRequestException("Functions must be fully qualified with a keyspace name if a keyspace is not set for the session");
 
+        stateFunc = validateFunctionKeyspace(stateFunc);
+
+        if (finalFunc != null)
+            finalFunc = validateFunctionKeyspace(finalFunc);
+
         ThriftValidation.validateKeyspaceNotSystem(functionName.keyspace);
     }
 
+    private FunctionName validateFunctionKeyspace(FunctionName func)
+    {
+        if (!func.hasKeyspace())
+            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",
+                                                            functionName.keyspace, func.keyspace));
+        return func;
+    }
+
     protected void grantPermissionsToCreator(QueryState state)
     {
         try

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6eea3ea3/src/java/org/apache/cassandra/cql3/statements/CreateFunctionStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/CreateFunctionStatement.java b/src/java/org/apache/cassandra/cql3/statements/CreateFunctionStatement.java
index 4ceecba..a2a440b 100644
--- a/src/java/org/apache/cassandra/cql3/statements/CreateFunctionStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/CreateFunctionStatement.java
@@ -36,7 +36,7 @@ import org.apache.cassandra.thrift.ThriftValidation;
 import org.apache.cassandra.transport.Event;
 
 /**
- * A <code>CREATE FUNCTION</code> statement parsed from a CQL query.
+ * A {@code CREATE FUNCTION} statement parsed from a CQL query.
  */
 public final class CreateFunctionStatement extends SchemaAlteringStatement
 {
@@ -85,9 +85,9 @@ public final class CreateFunctionStatement extends SchemaAlteringStatement
 
         argTypes = new ArrayList<>(argRawTypes.size());
         for (CQL3Type.Raw rawType : argRawTypes)
-            argTypes.add(rawType.prepare(typeKeyspace(rawType)).getType());
+            argTypes.add(rawType.prepare(functionName.keyspace).getType());
 
-        returnType = rawReturnType.prepare(typeKeyspace(rawReturnType)).getType();
+        returnType = rawReturnType.prepare(functionName.keyspace).getType();
         return super.prepare();
     }
 
@@ -174,12 +174,4 @@ public final class CreateFunctionStatement extends SchemaAlteringStatement
 
         return true;
     }
-
-    private String typeKeyspace(CQL3Type.Raw rawType)
-    {
-        String ks = rawType.keyspace();
-        if (ks != null)
-            return ks;
-        return functionName.keyspace;
-    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6eea3ea3/test/unit/org/apache/cassandra/cql3/AggregationTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/AggregationTest.java b/test/unit/org/apache/cassandra/cql3/AggregationTest.java
index 7bec8a1..f1a0b94 100644
--- a/test/unit/org/apache/cassandra/cql3/AggregationTest.java
+++ b/test/unit/org/apache/cassandra/cql3/AggregationTest.java
@@ -1043,11 +1043,94 @@ public class AggregationTest extends CQLTester
                                        "AS 'return Integer.valueOf(1);';");
 
         assertInvalidMessage("return type must be the same as the first argument type - check STYPE, argument and return types",
-                                   "CREATE AGGREGATE %s(int) " +
-                                   "SFUNC " + shortFunctionName(fState) + ' ' +
-                                   "STYPE int " +
-                                   "FINALFUNC " + shortFunctionName(fFinal) + ' ' +
-                                   "INITCOND 1");
+                             "CREATE AGGREGATE %s(int) " +
+                             "SFUNC " + shortFunctionName(fState) + ' ' +
+                             "STYPE int " +
+                             "FINALFUNC " + shortFunctionName(fFinal) + ' ' +
+                             "INITCOND 1");
+    }
+
+    @Test
+    public void testWrongKeyspace() throws Throwable
+    {
+        String typeName = createType("CREATE TYPE %s (txt text, i int)");
+        String type = KEYSPACE + '.' + typeName;
+
+        String fState = createFunction(KEYSPACE_PER_TEST,
+                                       "int, int",
+                                       "CREATE FUNCTION %s(a int, b int) " +
+                                       "CALLED ON NULL INPUT " +
+                                       "RETURNS double " +
+                                       "LANGUAGE java " +
+                                       "AS 'return Double.valueOf(1.0);'");
+
+        String fFinal = createFunction(KEYSPACE_PER_TEST,
+                                       "int",
+                                       "CREATE FUNCTION %s(a int) " +
+                                       "CALLED ON NULL INPUT " +
+                                       "RETURNS int " +
+                                       "LANGUAGE java " +
+                                       "AS 'return Integer.valueOf(1);';");
+
+        String fStateWrong = createFunction(KEYSPACE,
+                                       "int, int",
+                                       "CREATE FUNCTION %s(a int, b int) " +
+                                       "CALLED ON NULL INPUT " +
+                                       "RETURNS double " +
+                                       "LANGUAGE java " +
+                                       "AS 'return Double.valueOf(1.0);'");
+
+        String fFinalWrong = createFunction(KEYSPACE,
+                                       "int",
+                                       "CREATE FUNCTION %s(a int) " +
+                                       "CALLED ON NULL INPUT " +
+                                       "RETURNS int " +
+                                       "LANGUAGE java " +
+                                       "AS 'return Integer.valueOf(1);';");
+
+        assertInvalidMessage(String.format("Statement on keyspace %s cannot refer to a user type in keyspace %s; user types can only be used in the keyspace they are defined in",
+                                           KEYSPACE_PER_TEST, KEYSPACE),
+                             "CREATE AGGREGATE " + KEYSPACE_PER_TEST + ".test_wrong_ks(int) " +
+                             "SFUNC " + shortFunctionName(fState) + ' ' +
+                             "STYPE frozen<" + type + "> " +
+                             "FINALFUNC " + shortFunctionName(fFinal) + ' ' +
+                             "INITCOND 1");
+
+        assertInvalidMessage(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",
+                                           KEYSPACE_PER_TEST, KEYSPACE),
+                             "CREATE AGGREGATE " + KEYSPACE_PER_TEST + ".test_wrong_ks(int) " +
+                             "SFUNC " + fStateWrong + ' ' +
+                             "STYPE frozen<" + type + "> " +
+                             "FINALFUNC " + shortFunctionName(fFinal) + ' ' +
+                             "INITCOND 1");
+
+        assertInvalidMessage(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",
+                                           KEYSPACE_PER_TEST, KEYSPACE),
+                             "CREATE AGGREGATE " + KEYSPACE_PER_TEST + ".test_wrong_ks(int) " +
+                             "SFUNC " + shortFunctionName(fState) + ' ' +
+                             "STYPE frozen<" + type + "> " +
+                             "FINALFUNC " + fFinalWrong + ' ' +
+                             "INITCOND 1");
+    }
+
+    @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'");
     }
 
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6eea3ea3/test/unit/org/apache/cassandra/cql3/UFTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/UFTest.java b/test/unit/org/apache/cassandra/cql3/UFTest.java
index 78c6dd9..79d63a9 100644
--- a/test/unit/org/apache/cassandra/cql3/UFTest.java
+++ b/test/unit/org/apache/cassandra/cql3/UFTest.java
@@ -919,6 +919,29 @@ public class UFTest extends CQLTester
     }
 
     @Test
+    public void testWrongKeyspace() throws Throwable
+    {
+        String typeName = createType("CREATE TYPE %s (txt text, i int)");
+        String type = KEYSPACE + '.' + typeName;
+
+        assertInvalidMessage(String.format("Statement on keyspace %s cannot refer to a user type in keyspace %s; user types can only be used in the keyspace they are defined in",
+                                           KEYSPACE_PER_TEST, KEYSPACE),
+                             "CREATE FUNCTION " + KEYSPACE_PER_TEST + ".test_wrong_ks( val int ) " +
+                             "CALLED ON NULL INPUT " +
+                             "RETURNS frozen<" + type + "> " +
+                             "LANGUAGE java\n" +
+                             "AS $$return val;$$;");
+
+        assertInvalidMessage(String.format("Statement on keyspace %s cannot refer to a user type in keyspace %s; user types can only be used in the keyspace they are defined in",
+                                           KEYSPACE_PER_TEST, KEYSPACE),
+                             "CREATE FUNCTION " + KEYSPACE_PER_TEST + ".test_wrong_ks( val frozen<" + type + "> ) " +
+                             "CALLED ON NULL INPUT " +
+                             "RETURNS int " +
+                             "LANGUAGE java\n" +
+                             "AS $$return val;$$;");
+    }
+
+    @Test
     public void testComplexNullValues() throws Throwable
     {
         String type = KEYSPACE + '.' + createType("CREATE TYPE %s (txt text, i int)");
@@ -1174,22 +1197,6 @@ public class UFTest extends CQLTester
     }
 
     @Test
-    public void testJavaUserTypeOtherKeyspace() throws Throwable
-    {
-        String type = KEYSPACE + '.' + createType("CREATE TYPE %s (txt text, i int)");
-
-        String fName = createFunction(KEYSPACE_PER_TEST, "frozen<" + type + '>',
-                                      "CREATE FUNCTION %s( udt frozen<" + type + "> ) " +
-                                      "RETURNS NULL ON NULL INPUT " +
-                                      "RETURNS frozen<" + type + "> " +
-                                      "LANGUAGE java " +
-                                      "AS $$return " +
-                                      "     udt;$$;");
-
-        execute("DROP FUNCTION " + fName);
-    }
-
-    @Test
     public void testJavaUserType() throws Throwable
     {
         String type = KEYSPACE + '.' + createType("CREATE TYPE %s (txt text, i int)");