You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by ty...@apache.org on 2014/09/19 18:24:55 UTC

[1/3] git commit: Validate token() args are in partition key order

Repository: cassandra
Updated Branches:
  refs/heads/cassandra-2.1 1949212f5 -> c5c0585b4


Validate token() args are in partition key order

Patch by Benjamin Lerer; reviewed by Tyler Hobbs for CASSANDRA-6075


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

Branch: refs/heads/cassandra-2.1
Commit: 6198a756338a6101cd0787bd3774ad86b56d4fa5
Parents: 75e4acb
Author: blerer <b_...@hotmail.com>
Authored: Fri Sep 19 11:18:58 2014 -0500
Committer: Tyler Hobbs <ty...@datastax.com>
Committed: Fri Sep 19 11:18:58 2014 -0500

----------------------------------------------------------------------
 CHANGES.txt                                     |   2 +
 .../cql3/statements/SelectStatement.java        |  22 +++
 .../cql3/SelectWithTokenFunctionTest.java       | 141 +++++++++++++++++++
 3 files changed, 165 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/6198a756/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index e0b5688..abd7c68 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,6 @@
 2.0.11:
+ * token() should only accept columns in the partitioning
+   key order (CASSANDRA-6075)
  * Add method to invalidate permission cache via JMX (CASSANDRA-7977)
  * Allow propagating multiple gossip states atomically (CASSANDRA-6125)
  * Log exceptions related to unclean native protocol client disconnects

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6198a756/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
index ea1d8e9..363e3d3 100644
--- a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
@@ -20,6 +20,7 @@ package org.apache.cassandra.cql3.statements;
 import java.nio.ByteBuffer;
 import java.util.*;
 
+import com.google.common.base.Joiner;
 import com.google.common.base.Objects;
 import com.google.common.base.Predicate;
 import com.google.common.collect.Iterables;
@@ -1813,6 +1814,27 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
                 }
                 previous = cname;
             }
+
+            if (stmt.onToken && cfDef.partitionKeyCount() > 0)
+                checkTokenFunctionArgumentsOrder(cfDef);
+        }
+
+        /**
+         * Checks that the column identifiers used as argument for the token function have been specified in the
+         * partition key order.
+         * @param cfDef the Column Family Definition
+         * @throws InvalidRequestException if the arguments have not been provided in the proper order.
+         */
+        private void checkTokenFunctionArgumentsOrder(CFDefinition cfDef) throws InvalidRequestException
+        {
+            Iterator<Name> iter = cfDef.partitionKeys().iterator();
+            for (Relation relation : whereClause)
+            {
+                SingleColumnRelation singleColumnRelation = (SingleColumnRelation) relation;
+                if (singleColumnRelation.onToken && !cfDef.get(singleColumnRelation.getEntity()).equals(iter.next()))
+                    throw new InvalidRequestException(String.format("The token function arguments must be in the partition key order: %s",
+                                                                    Joiner.on(',').join(cfDef.partitionKeys())));
+            }
         }
 
         private void processColumnRestrictions(SelectStatement stmt, CFDefinition cfDef, boolean hasQueriableIndex) throws InvalidRequestException

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6198a756/test/unit/org/apache/cassandra/cql3/SelectWithTokenFunctionTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/SelectWithTokenFunctionTest.java b/test/unit/org/apache/cassandra/cql3/SelectWithTokenFunctionTest.java
new file mode 100644
index 0000000..f089a5b
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/SelectWithTokenFunctionTest.java
@@ -0,0 +1,141 @@
+/*
+ * 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;
+
+import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.db.ConsistencyLevel;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.gms.Gossiper;
+import org.apache.cassandra.service.ClientState;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.cassandra.cql3.QueryProcessor.process;
+import static org.apache.cassandra.cql3.QueryProcessor.processInternal;
+import static org.junit.Assert.assertEquals;
+
+public class SelectWithTokenFunctionTest
+{
+    private static final Logger logger = LoggerFactory.getLogger(SelectWithTokenFunctionTest.class);
+    static ClientState clientState;
+    static String keyspace = "token_function_test";
+
+    @BeforeClass
+    public static void setUpClass() throws Throwable
+    {
+        SchemaLoader.loadSchema();
+        executeSchemaChange("CREATE KEYSPACE IF NOT EXISTS %s WITH replication = {'class': 'SimpleStrategy', 'replication_factor': '1'}");
+        executeSchemaChange("CREATE TABLE IF NOT EXISTS %s.single_partition (a int PRIMARY KEY, b text)");
+        executeSchemaChange("CREATE TABLE IF NOT EXISTS %s.compound_partition (a int, b text, PRIMARY KEY ((a, b)))");
+        executeSchemaChange("CREATE TABLE IF NOT EXISTS %s.single_clustering (a int, b text, PRIMARY KEY (a, b))");
+        clientState = ClientState.forInternalCalls();
+    }
+
+    @AfterClass
+    public static void stopGossiper()
+    {
+        Gossiper.instance.stop();
+    }
+
+    private static void executeSchemaChange(String query) throws Throwable
+    {
+        try
+        {
+            process(String.format(query, keyspace), ConsistencyLevel.ONE);
+        }
+        catch (RuntimeException exc)
+        {
+            throw exc.getCause();
+        }
+    }
+
+    private static UntypedResultSet execute(String query) throws Throwable
+    {
+        try
+        {
+            return processInternal(String.format(query, keyspace));
+        }
+        catch (RuntimeException exc)
+        {
+            if (exc.getCause() != null)
+                throw exc.getCause();
+            throw exc;
+        }
+    }
+
+    @Test
+    public void testTokenFunctionWithSinglePartitionArgument() throws Throwable
+    {
+        execute("INSERT INTO %s.single_partition (a, b) VALUES (0, 'a')");
+
+        try
+        {
+            UntypedResultSet results = execute("SELECT * FROM %s.single_partition WHERE token(a) >= token(0)");
+            assertEquals(1, results.size());
+        }
+        finally
+        {
+            execute("DELETE FROM %s.single_partition WHERE a = 0");
+        }
+    }
+
+    @Test(expected = InvalidRequestException.class)
+    public void testTokenFunctionWithWrongLiteralArgument() throws Throwable
+    {
+        execute("SELECT * FROM %s.single_partition WHERE token(a) > token('a')");
+    }
+
+    @Test(expected = InvalidRequestException.class)
+    public void testTokenFunctionWithPartitionKeyAndClusteringKeyArguments() throws Throwable
+    {
+        execute("SELECT * FROM %s.single_clustering WHERE token(a, b) > token(0, 'c')");
+    }
+
+    @Test(expected = InvalidRequestException.class)
+    public void testTokenFunctionWithCompoundPartitionKeyAndWrongLiteralArgument() throws Throwable
+    {
+        execute("SELECT * FROM %s.single_partition WHERE token(a, b) >= token('c', 0)");
+    }
+
+    @Test
+    public void testTokenFunctionWithCompoundPartition() throws Throwable
+    {
+        execute("INSERT INTO %s.compound_partition (a, b) VALUES (0, 'a')");
+        execute("INSERT INTO %s.compound_partition (a, b) VALUES (0, 'b')");
+        execute("INSERT INTO %s.compound_partition (a, b) VALUES (0, 'c')");
+
+        try
+        {
+            UntypedResultSet results = execute("SELECT * FROM %s.compound_partition WHERE token(a, b) > token(0, 'a')");
+            assertEquals(2, results.size());
+        }
+        finally
+        {
+            execute("DELETE FROM %s.compound_partition WHERE a = 0 and b in ('a', 'b', 'c')");
+        }
+    }
+
+    @Test(expected = InvalidRequestException.class)
+    public void testTokenFunctionWithCompoundPartitionKeyAndColumnIdentifierInWrongOrder() throws Throwable
+    {
+        execute("SELECT * FROM %s.compound_partition WHERE token(b, a) > token(0, 'c')");
+    }
+}


[3/3] git commit: Validate token() args are in partition key order

Posted by ty...@apache.org.
Validate token() args are in partition key order

Patch by Benjamin Lerer; reviewed by Tyler Hobbs for CASSANDRA-6075


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

Branch: refs/heads/cassandra-2.1
Commit: c5c0585b4b5a1bad22ced00e01cc90eed8448ce7
Parents: 20eb8f0
Author: blerer <b_...@hotmail.com>
Authored: Fri Sep 19 11:24:24 2014 -0500
Committer: Tyler Hobbs <ty...@datastax.com>
Committed: Fri Sep 19 11:24:24 2014 -0500

----------------------------------------------------------------------
 CHANGES.txt                                     |  3 +-
 .../cql3/statements/SelectStatement.java        | 22 ++++++++
 .../cql3/SelectWithTokenFunctionTest.java       | 55 ++++++++++++++++++++
 3 files changed, 79 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/c5c0585b/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 9a69dfd..d3ee7d9 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -44,7 +44,8 @@
  * Include snippet of CQL query near syntax error in messages (CASSANDRA-7111)
  * Make repair -pr work with -local (CASSANDRA-7450)
 Merged from 2.0:
-2.0.11:
+ * token() should only accept columns in the partitioning
+   key order (CASSANDRA-6075)
  * Add method to invalidate permission cache via JMX (CASSANDRA-7977)
  * Allow propagating multiple gossip states atomically (CASSANDRA-6125)
  * Log exceptions related to unclean native protocol client disconnects

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c5c0585b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
index 586eb85..22c8468 100644
--- a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
@@ -20,6 +20,7 @@ package org.apache.cassandra.cql3.statements;
 import java.nio.ByteBuffer;
 import java.util.*;
 
+import com.google.common.base.Joiner;
 import com.google.common.base.Objects;
 import com.google.common.base.Predicate;
 import com.google.common.collect.AbstractIterator;
@@ -1822,6 +1823,27 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
                 }
                 previous = cdef;
             }
+
+            if (stmt.onToken && cfm.partitionKeyColumns().size() > 0)
+                checkTokenFunctionArgumentsOrder(cfm);
+        }
+
+        /**
+         * Checks that the column identifiers used as argument for the token function have been specified in the
+         * partition key order.
+         * @param cfm the Column Family MetaData
+         * @throws InvalidRequestException if the arguments have not been provided in the proper order.
+         */
+        private void checkTokenFunctionArgumentsOrder(CFMetaData cfm) throws InvalidRequestException
+        {
+            Iterator<ColumnDefinition> iter = cfm.partitionKeyColumns().iterator();
+            for (Relation relation : whereClause)
+            {
+                SingleColumnRelation singleColumnRelation = (SingleColumnRelation) relation;
+                if (singleColumnRelation.onToken && !cfm.getColumnDefinition(singleColumnRelation.getEntity()).equals(iter.next()))
+                    throw new InvalidRequestException(String.format("The token function arguments must be in the partition key order: %s",
+                                                                    Joiner.on(',').join(cfm.partitionKeyColumns())));
+            }
         }
 
         private void processColumnRestrictions(SelectStatement stmt, boolean hasQueriableIndex, CFMetaData cfm) throws InvalidRequestException

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c5c0585b/test/unit/org/apache/cassandra/cql3/SelectWithTokenFunctionTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/SelectWithTokenFunctionTest.java b/test/unit/org/apache/cassandra/cql3/SelectWithTokenFunctionTest.java
new file mode 100644
index 0000000..73a7209
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/SelectWithTokenFunctionTest.java
@@ -0,0 +1,55 @@
+/*
+ * 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;
+
+import org.junit.Test;
+
+public class SelectWithTokenFunctionTest extends CQLTester
+{
+    @Test
+    public void testTokenFunctionWithSingleColumnPartitionKey() throws Throwable
+    {
+        createTable("CREATE TABLE IF NOT EXISTS %s (a int PRIMARY KEY, b text)");
+        execute("INSERT INTO %s (a, b) VALUES (0, 'a')");
+
+        assertRows(execute("SELECT * FROM %s WHERE token(a) >= token(?)", 0), row(0, "a"));
+        assertInvalid("SELECT * FROM %s WHERE token(a) > token(?)", "a");
+        assertInvalid("SELECT * FROM %s WHERE token(a, b) >= token(?, ?)", "b", 0);
+    }
+
+    @Test
+    public void testTokenFunctionWithPartitionKeyAndClusteringKeyArguments() throws Throwable
+    {
+        createTable("CREATE TABLE IF NOT EXISTS %s (a int, b text, PRIMARY KEY (a, b))");
+        assertInvalid("SELECT * FROM %s WHERE token(a, b) > token(0, 'c')");
+    }
+
+    @Test
+    public void testTokenFunctionWithMultiColumnPartitionKey() throws Throwable
+    {
+        createTable("CREATE TABLE IF NOT EXISTS %s (a int, b text, PRIMARY KEY ((a, b)))");
+        execute("INSERT INTO %s (a, b) VALUES (0, 'a')");
+        execute("INSERT INTO %s (a, b) VALUES (0, 'b')");
+        execute("INSERT INTO %s (a, b) VALUES (0, 'c')");
+
+        assertRows(execute("SELECT * FROM %s WHERE token(a, b) > token(?, ?)", 0, "a"),
+                   row(0, "b"),
+                   row(0, "c"));
+        assertInvalid("SELECT * FROM %s WHERE token(b, a) > token(0, 'c')");
+    }
+}


[2/3] git commit: Merge branch 'cassandra-2.0' into cassandra-2.1

Posted by ty...@apache.org.
Merge branch 'cassandra-2.0' into cassandra-2.1


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

Branch: refs/heads/cassandra-2.1
Commit: 20eb8f0c6c670f88a2ad9f54508c7e06f01b74a5
Parents: 1949212 6198a75
Author: Tyler Hobbs <ty...@datastax.com>
Authored: Fri Sep 19 11:19:56 2014 -0500
Committer: Tyler Hobbs <ty...@datastax.com>
Committed: Fri Sep 19 11:19:56 2014 -0500

----------------------------------------------------------------------

----------------------------------------------------------------------