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 2015/03/05 19:25:46 UTC
[1/2] cassandra git commit: Fix mixed single and multi-column
relation support
Repository: cassandra
Updated Branches:
refs/heads/cassandra-2.0 fb67c41ad -> 90a012a1f
http://git-wip-us.apache.org/repos/asf/cassandra/blob/90a012a1/test/unit/org/apache/cassandra/cql3/statements/SelectStatementTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/statements/SelectStatementTest.java b/test/unit/org/apache/cassandra/cql3/statements/SelectStatementTest.java
new file mode 100644
index 0000000..1a66002
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/statements/SelectStatementTest.java
@@ -0,0 +1,973 @@
+/*
+ * 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.statements;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+import org.junit.Test;
+
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.cql3.*;
+import org.apache.cassandra.cql3.CFDefinition.Name;
+import org.apache.cassandra.db.ColumnFamilyType;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.CompositeType;
+import org.apache.cassandra.db.marshal.Int32Type;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+import static org.junit.Assert.assertArrayEquals;
+
+import static java.util.Arrays.asList;
+import static org.junit.Assert.assertEquals;
+
+public class SelectStatementTest
+{
+ @Test
+ public void testBuildBoundWithNoRestrictions() throws Exception
+ {
+ Restriction[] restrictions = new Restriction[2];
+ CFDefinition cfDef = createCFDefinition(restrictions.length);
+
+ List<ByteBuffer> bounds = executeBuildBound(cfDef, restrictions, Bound.START);
+ assertEquals(1, bounds.size());
+ assertComposite(cfDef, bounds.get(0));
+
+ bounds = executeBuildBound(cfDef, restrictions, Bound.END);
+ assertEquals(1, bounds.size());
+ assertComposite(cfDef, bounds.get(0));
+ }
+
+ /**
+ * Test 'clustering_0 = 1' with only one clustering column
+ */
+ @Test
+ public void testBuildBoundWithOneEqRestrictionsAndOneClusteringColumn() throws Exception
+ {
+ ByteBuffer clustering_0 = ByteBufferUtil.bytes(1);
+ SingleColumnRestriction.EQ eq = new SingleColumnRestriction.EQ(toTerm(clustering_0), false);
+ Restriction[] restrictions = new Restriction[] { eq };
+ CFDefinition cfDef = createCFDefinition(restrictions.length);
+
+ List<ByteBuffer> bounds = executeBuildBound(cfDef, restrictions, Bound.START);
+ assertEquals(1, bounds.size());
+ assertComposite(cfDef, bounds.get(0), clustering_0);
+
+ bounds = executeBuildBound(cfDef, restrictions, Bound.END);
+ assertEquals(1, bounds.size());
+ assertComposite(cfDef, bounds.get(0), clustering_0);
+ }
+
+ /**
+ * Test 'clustering_1 = 1' with 2 clustering columns
+ */
+ @Test
+ public void testBuildBoundWithOneEqRestrictionsAndTwoClusteringColumns() throws Exception
+ {
+ ByteBuffer clustering_0 = ByteBufferUtil.bytes(1);
+ SingleColumnRestriction.EQ eq = new SingleColumnRestriction.EQ(toTerm(clustering_0), false);
+ Restriction[] restrictions = new Restriction[] { eq, null };
+ CFDefinition cfDef = createCFDefinition(restrictions.length);
+
+ List<ByteBuffer> bounds = executeBuildBound(cfDef, restrictions, Bound.START);
+ assertEquals(1, bounds.size());
+ assertComposite(cfDef, bounds.get(0), clustering_0);
+
+ bounds = executeBuildBound(cfDef,restrictions, Bound.END);
+ assertEquals(1, bounds.size());
+ assertEndOfRangeComposite(cfDef, bounds.get(0), clustering_0);
+ }
+
+ /**
+ * Test 'clustering_0 IN (1, 2, 3)' with only one clustering column
+ */
+ @Test
+ public void testBuildBoundWithOneInRestrictionsAndOneClusteringColumn() throws Exception
+ {
+ ByteBuffer value1 = ByteBufferUtil.bytes(1);
+ ByteBuffer value2 = ByteBufferUtil.bytes(2);
+ ByteBuffer value3 = ByteBufferUtil.bytes(3);
+ SingleColumnRestriction.IN in = new SingleColumnRestriction.InWithValues(toTerms(value1, value2, value3));
+ Restriction[] restrictions = new Restriction[] { in };
+ CFDefinition cfDef = createCFDefinition(restrictions.length);
+
+ List<ByteBuffer> bounds = executeBuildBound(cfDef,restrictions, Bound.START);
+ assertEquals(3, bounds.size());
+ assertComposite(cfDef, bounds.get(0), value1);
+ assertComposite(cfDef, bounds.get(1), value2);
+ assertComposite(cfDef, bounds.get(2), value3);
+
+ bounds = executeBuildBound(cfDef,restrictions, Bound.END);
+ assertEquals(3, bounds.size());
+ assertComposite(cfDef, bounds.get(0), value1);
+ assertComposite(cfDef, bounds.get(1), value2);
+ assertComposite(cfDef, bounds.get(2), value3);
+ }
+
+ /**
+ * Test slice restriction (e.g 'clustering_0 > 1') with only one clustering column
+ */
+ @Test
+ public void testBuildBoundWithSliceRestrictionsAndOneClusteringColumn() throws Exception
+ {
+ ByteBuffer value1 = ByteBufferUtil.bytes(1);
+ ByteBuffer value2 = ByteBufferUtil.bytes(2);
+
+ SingleColumnRestriction.Slice slice = new SingleColumnRestriction.Slice(false);
+ slice.setBound(Relation.Type.GT, toTerm(value1));
+ Restriction[] restrictions = new Restriction[] { slice };
+ CFDefinition cfDef = createCFDefinition(restrictions.length);
+
+ List<ByteBuffer> bounds = executeBuildBound(cfDef,restrictions, Bound.START);
+ assertEquals(1, bounds.size());
+ assertForRelationComposite(cfDef, bounds.get(0), Relation.Type.GT, value1);
+
+ bounds = executeBuildBound(cfDef,restrictions, Bound.END);
+ assertEquals(1, bounds.size());
+ assertComposite(cfDef, bounds.get(0));
+
+ slice = new SingleColumnRestriction.Slice(false);
+ slice.setBound(Relation.Type.GTE, toTerm(value1));
+ restrictions = new Restriction[] { slice };
+
+ bounds = executeBuildBound(cfDef,restrictions, Bound.START);
+ assertEquals(1, bounds.size());
+ assertForRelationComposite(cfDef, bounds.get(0), Relation.Type.GTE, value1);
+
+ bounds = executeBuildBound(cfDef,restrictions, Bound.END);
+ assertEquals(1, bounds.size());
+ assertComposite(cfDef, bounds.get(0));
+
+ slice = new SingleColumnRestriction.Slice(false);
+ slice.setBound(Relation.Type.LTE, toTerm(value1));
+ restrictions = new Restriction[] { slice };
+
+ bounds = executeBuildBound(cfDef,restrictions, Bound.START);
+ assertEquals(1, bounds.size());
+ assertComposite(cfDef, bounds.get(0));
+
+ bounds = executeBuildBound(cfDef,restrictions, Bound.END);
+ assertEquals(1, bounds.size());
+ assertForRelationComposite(cfDef, bounds.get(0), Relation.Type.LTE, value1);
+
+ slice = new SingleColumnRestriction.Slice(false);
+ slice.setBound(Relation.Type.LT, toTerm(value1));
+ restrictions = new Restriction[] { slice };
+
+ bounds = executeBuildBound(cfDef,restrictions, Bound.START);
+ assertEquals(1, bounds.size());
+ assertComposite(cfDef, bounds.get(0));
+
+ bounds = executeBuildBound(cfDef,restrictions, Bound.END);
+ assertEquals(1, bounds.size());
+ assertForRelationComposite(cfDef, bounds.get(0), Relation.Type.LT, value1);
+
+ slice = new SingleColumnRestriction.Slice(false);
+ slice.setBound(Relation.Type.GT, toTerm(value1));
+ slice.setBound(Relation.Type.LT, toTerm(value2));
+ restrictions = new Restriction[] { slice };
+
+ bounds = executeBuildBound(cfDef,restrictions, Bound.START);
+ assertEquals(1, bounds.size());
+ assertForRelationComposite(cfDef, bounds.get(0), Relation.Type.GT, value1);
+
+ bounds = executeBuildBound(cfDef,restrictions, Bound.END);
+ assertEquals(1, bounds.size());
+ assertForRelationComposite(cfDef, bounds.get(0), Relation.Type.LT, value2);
+
+ slice = new SingleColumnRestriction.Slice(false);
+ slice.setBound(Relation.Type.GTE, toTerm(value1));
+ slice.setBound(Relation.Type.LTE, toTerm(value2));
+ restrictions = new Restriction[] { slice };
+
+ bounds = executeBuildBound(cfDef,restrictions, Bound.START);
+ assertEquals(1, bounds.size());
+ assertForRelationComposite(cfDef, bounds.get(0), Relation.Type.GTE, value1);
+
+ bounds = executeBuildBound(cfDef,restrictions, Bound.END);
+ assertEquals(1, bounds.size());
+ assertForRelationComposite(cfDef, bounds.get(0), Relation.Type.LTE, value2);
+ }
+//
+ /**
+ * Test 'clustering_0 = 1 AND clustering_1 IN (1, 2, 3)' with two clustering columns
+ */
+ @Test
+ public void testBuildBoundWithEqAndInRestrictions() throws Exception
+ {
+ ByteBuffer value1 = ByteBufferUtil.bytes(1);
+ ByteBuffer value2 = ByteBufferUtil.bytes(2);
+ ByteBuffer value3 = ByteBufferUtil.bytes(3);
+ SingleColumnRestriction.EQ eq = new SingleColumnRestriction.EQ(toTerm(value1), false);
+ SingleColumnRestriction.IN in = new SingleColumnRestriction.InWithValues(toTerms(value1, value2, value3));
+ Restriction[] restrictions = new Restriction[] { eq, in };
+ CFDefinition cfDef = createCFDefinition(restrictions.length);
+
+ List<ByteBuffer> bounds = executeBuildBound(cfDef,restrictions, Bound.START);
+ assertEquals(3, bounds.size());
+ assertComposite(cfDef, bounds.get(0), value1, value1);
+ assertComposite(cfDef, bounds.get(1), value1, value2);
+ assertComposite(cfDef, bounds.get(2), value1, value3);
+
+ bounds = executeBuildBound(cfDef,restrictions, Bound.END);
+ assertEquals(3, bounds.size());
+ assertComposite(cfDef, bounds.get(0), value1, value1);
+ assertComposite(cfDef, bounds.get(1), value1, value2);
+ assertComposite(cfDef, bounds.get(2), value1, value3);
+ }
+
+ /**
+ * Test slice restriction (e.g 'clustering_0 > 1') with only one clustering column
+ */
+ @Test
+ public void testBuildBoundWithEqAndSliceRestrictions() throws Exception
+ {
+ ByteBuffer value1 = ByteBufferUtil.bytes(1);
+ ByteBuffer value2 = ByteBufferUtil.bytes(2);
+ ByteBuffer value3 = ByteBufferUtil.bytes(3);
+
+ SingleColumnRestriction.EQ eq = new SingleColumnRestriction.EQ(toTerm(value3), false);
+
+ SingleColumnRestriction.Slice slice = new SingleColumnRestriction.Slice(false);
+ slice.setBound(Relation.Type.GT, toTerm(value1));
+ Restriction[] restrictions = new Restriction[] { eq, slice };
+ CFDefinition cfDef = createCFDefinition(restrictions.length);
+
+ List<ByteBuffer> bounds = executeBuildBound(cfDef,restrictions, Bound.START);
+ assertEquals(1, bounds.size());
+ assertForRelationComposite(cfDef, bounds.get(0), Relation.Type.GT, value3, value1);
+
+ bounds = executeBuildBound(cfDef,restrictions, Bound.END);
+ assertEquals(1, bounds.size());
+ assertEndOfRangeComposite(cfDef, bounds.get(0), value3);
+
+ slice = new SingleColumnRestriction.Slice(false);
+ slice.setBound(Relation.Type.GTE, toTerm(value1));
+ restrictions = new Restriction[] { eq, slice };
+
+ bounds = executeBuildBound(cfDef,restrictions, Bound.START);
+ assertEquals(1, bounds.size());
+ assertForRelationComposite(cfDef, bounds.get(0), Relation.Type.GTE, value3, value1);
+
+ bounds = executeBuildBound(cfDef, restrictions, Bound.END);
+ assertEquals(1, bounds.size());
+ assertEndOfRangeComposite(cfDef, bounds.get(0), value3);
+
+ slice = new SingleColumnRestriction.Slice(false);
+ slice.setBound(Relation.Type.LTE, toTerm(value1));
+ restrictions = new Restriction[] { eq, slice };
+
+ bounds = executeBuildBound(cfDef, restrictions, Bound.START);
+ assertEquals(1, bounds.size());
+ assertComposite(cfDef, bounds.get(0), value3);
+
+ bounds = executeBuildBound(cfDef, restrictions, Bound.END);
+ assertEquals(1, bounds.size());
+ assertForRelationComposite(cfDef, bounds.get(0), Relation.Type.LTE, value3, value1);
+
+ slice = new SingleColumnRestriction.Slice(false);
+ slice.setBound(Relation.Type.LT, toTerm(value1));
+ restrictions = new Restriction[] { eq, slice };
+
+ bounds = executeBuildBound(cfDef, restrictions, Bound.START);
+ assertEquals(1, bounds.size());
+ assertComposite(cfDef, bounds.get(0), value3);
+
+ bounds = executeBuildBound(cfDef, restrictions, Bound.END);
+ assertEquals(1, bounds.size());
+ assertForRelationComposite(cfDef, bounds.get(0), Relation.Type.LT, value3, value1);
+
+ slice = new SingleColumnRestriction.Slice(false);
+ slice.setBound(Relation.Type.GT, toTerm(value1));
+ slice.setBound(Relation.Type.LT, toTerm(value2));
+ restrictions = new Restriction[] { eq, slice };
+
+ bounds = executeBuildBound(cfDef, restrictions, Bound.START);
+ assertEquals(1, bounds.size());
+ assertForRelationComposite(cfDef, bounds.get(0), Relation.Type.GT, value3, value1);
+
+ bounds = executeBuildBound(cfDef, restrictions, Bound.END);
+ assertEquals(1, bounds.size());
+ assertForRelationComposite(cfDef, bounds.get(0), Relation.Type.LT, value3, value2);
+
+ slice = new SingleColumnRestriction.Slice(false);
+ slice.setBound(Relation.Type.GTE, toTerm(value1));
+ slice.setBound(Relation.Type.LTE, toTerm(value2));
+ restrictions = new Restriction[] { eq, slice };
+
+ bounds = executeBuildBound(cfDef, restrictions, Bound.START);
+ assertEquals(1, bounds.size());
+ assertForRelationComposite(cfDef, bounds.get(0), Relation.Type.GTE, value3, value1);
+
+ bounds = executeBuildBound(cfDef, restrictions, Bound.END);
+ assertEquals(1, bounds.size());
+ assertForRelationComposite(cfDef, bounds.get(0), Relation.Type.LTE, value3, value2);
+ }
+
+ /**
+ * Test '(clustering_0, clustering_1) = (1, 2)' with two clustering column
+ */
+ @Test
+ public void testBuildBoundWithMultiEqRestrictions() throws Exception
+ {
+ ByteBuffer value1 = ByteBufferUtil.bytes(1);
+ ByteBuffer value2 = ByteBufferUtil.bytes(2);
+ MultiColumnRestriction.EQ eq = new MultiColumnRestriction.EQ(toMultiItemTerminal(value1, value2), false);
+ Restriction[] restrictions = new Restriction[] { eq, eq };
+ CFDefinition cfDef = createCFDefinition(restrictions.length);
+
+ List<ByteBuffer> bounds = executeBuildBound(cfDef, restrictions, Bound.START);
+ assertEquals(1, bounds.size());
+ assertComposite(cfDef, bounds.get(0), value1, value2);
+
+ bounds = executeBuildBound(cfDef, restrictions, Bound.END);
+ assertEquals(1, bounds.size());
+ assertComposite(cfDef, bounds.get(0), value1, value2);
+ }
+
+ /**
+ * Test '(clustering_0, clustering_1) IN ((1, 2), (2, 3))' with two clustering column
+ */
+ @Test
+ public void testBuildBoundWithMultiInRestrictions() throws Exception
+ {
+ ByteBuffer value1 = ByteBufferUtil.bytes(1);
+ ByteBuffer value2 = ByteBufferUtil.bytes(2);
+ ByteBuffer value3 = ByteBufferUtil.bytes(3);
+ List<Term> terms = asList(toMultiItemTerminal(value1, value2), toMultiItemTerminal(value2, value3));
+ MultiColumnRestriction.IN in = new MultiColumnRestriction.InWithValues(terms);
+ Restriction[] restrictions = new Restriction[] { in, in };
+ CFDefinition cfDef = createCFDefinition(restrictions.length);
+
+ List<ByteBuffer> bounds = executeBuildBound(cfDef, restrictions, Bound.START);
+ assertEquals(2, bounds.size());
+ assertComposite(cfDef, bounds.get(0), value1, value2);
+ assertComposite(cfDef, bounds.get(1), value2, value3);
+
+ bounds = executeBuildBound(cfDef, restrictions, Bound.END);
+ assertEquals(2, bounds.size());
+ assertComposite(cfDef, bounds.get(0), value1, value2);
+ assertComposite(cfDef, bounds.get(1), value2, value3);
+ }
+
+ /**
+ * Test multi-column slice restrictions (e.g '(clustering_0) > (1)') with only one clustering column
+ */
+ @Test
+ public void testBuildBoundWithMultiSliceRestrictionsWithOneClusteringColumn() throws Exception
+ {
+ ByteBuffer value1 = ByteBufferUtil.bytes(1);
+ ByteBuffer value2 = ByteBufferUtil.bytes(2);
+
+ MultiColumnRestriction.Slice slice = new MultiColumnRestriction.Slice(false);
+ slice.setBound(Relation.Type.GT, toMultiItemTerminal(value1));
+ Restriction[] restrictions = new Restriction[] { slice };
+ CFDefinition cfDef = createCFDefinition(restrictions.length);
+
+ List<ByteBuffer> bounds = executeBuildBound(cfDef, restrictions, Bound.START);
+ assertEquals(1, bounds.size());
+ assertForRelationComposite(cfDef, bounds.get(0), Relation.Type.GT, value1);
+
+ bounds = executeBuildBound(cfDef, restrictions, Bound.END);
+ assertEquals(1, bounds.size());
+ assertComposite(cfDef, bounds.get(0));
+
+ slice = new MultiColumnRestriction.Slice(false);
+ slice.setBound(Relation.Type.GTE, toMultiItemTerminal(value1));
+ restrictions = new Restriction[] { slice };
+
+ bounds = executeBuildBound(cfDef, restrictions, Bound.START);
+ assertEquals(1, bounds.size());
+ assertForRelationComposite(cfDef, bounds.get(0), Relation.Type.GTE, value1);
+
+ bounds = executeBuildBound(cfDef, restrictions, Bound.END);
+ assertEquals(1, bounds.size());
+ assertComposite(cfDef, bounds.get(0));
+
+ slice = new MultiColumnRestriction.Slice(false);
+ slice.setBound(Relation.Type.LTE, toMultiItemTerminal(value1));
+ restrictions = new Restriction[] { slice };
+
+ bounds = executeBuildBound(cfDef, restrictions, Bound.START);
+ assertEquals(1, bounds.size());
+ assertComposite(cfDef, bounds.get(0));
+
+ bounds = executeBuildBound(cfDef, restrictions, Bound.END);
+ assertEquals(1, bounds.size());
+ assertForRelationComposite(cfDef, bounds.get(0), Relation.Type.LTE, value1);
+
+ slice = new MultiColumnRestriction.Slice(false);
+ slice.setBound(Relation.Type.LT, toMultiItemTerminal(value1));
+ restrictions = new Restriction[] { slice };
+
+ bounds = executeBuildBound(cfDef, restrictions, Bound.START);
+ assertEquals(1, bounds.size());
+ assertComposite(cfDef, bounds.get(0));
+
+ bounds = executeBuildBound(cfDef, restrictions, Bound.END);
+ assertEquals(1, bounds.size());
+ assertForRelationComposite(cfDef, bounds.get(0), Relation.Type.LT, value1);
+
+ slice = new MultiColumnRestriction.Slice(false);
+ slice.setBound(Relation.Type.GT, toMultiItemTerminal(value1));
+ slice.setBound(Relation.Type.LT, toMultiItemTerminal(value2));
+ restrictions = new Restriction[] { slice };
+
+ bounds = executeBuildBound(cfDef, restrictions, Bound.START);
+ assertEquals(1, bounds.size());
+ assertForRelationComposite(cfDef, bounds.get(0), Relation.Type.GT, value1);
+
+ bounds = executeBuildBound(cfDef, restrictions, Bound.END);
+ assertEquals(1, bounds.size());
+ assertForRelationComposite(cfDef, bounds.get(0), Relation.Type.LT, value2);
+
+ slice = new MultiColumnRestriction.Slice(false);
+ slice.setBound(Relation.Type.GTE, toMultiItemTerminal(value1));
+ slice.setBound(Relation.Type.LTE, toMultiItemTerminal(value2));
+ restrictions = new Restriction[] { slice };
+
+ bounds = executeBuildBound(cfDef, restrictions, Bound.START);
+ assertEquals(1, bounds.size());
+ assertForRelationComposite(cfDef, bounds.get(0), Relation.Type.GTE, value1);
+
+ bounds = executeBuildBound(cfDef, restrictions, Bound.END);
+ assertEquals(1, bounds.size());
+ assertForRelationComposite(cfDef, bounds.get(0), Relation.Type.LTE, value2);
+ }
+
+ /**
+ * Test multi-column slice restrictions (e.g '(clustering_0, clustering_1) > (1, 2)') with two clustering
+ * columns
+ */
+ @Test
+ public void testBuildBoundWithMultiSliceRestrictionsWithTwoClusteringColumn() throws Exception
+ {
+ ByteBuffer value1 = ByteBufferUtil.bytes(1);
+ ByteBuffer value2 = ByteBufferUtil.bytes(2);
+
+ // (clustering_0, clustering1) > (1, 2)
+ MultiColumnRestriction.Slice slice = new MultiColumnRestriction.Slice(false);
+ slice.setBound(Relation.Type.GT, toMultiItemTerminal(value1, value2));
+ Restriction[] restrictions = new Restriction[] { slice, slice };
+ CFDefinition cfDef = createCFDefinition(restrictions.length);
+
+ List<ByteBuffer> bounds = executeBuildBound(cfDef, restrictions, Bound.START);
+ assertEquals(1, bounds.size());
+ assertForRelationComposite(cfDef, bounds.get(0), Relation.Type.GT, value1, value2);
+
+ bounds = executeBuildBound(cfDef, restrictions, Bound.END);
+ assertEquals(1, bounds.size());
+ assertComposite(cfDef, bounds.get(0));
+
+ // (clustering_0, clustering1) >= (1, 2)
+ slice = new MultiColumnRestriction.Slice(false);
+ slice.setBound(Relation.Type.GTE, toMultiItemTerminal(value1, value2));
+ restrictions = new Restriction[] { slice, slice };
+
+ bounds = executeBuildBound(cfDef, restrictions, Bound.START);
+ assertEquals(1, bounds.size());
+ assertForRelationComposite(cfDef, bounds.get(0), Relation.Type.GTE, value1, value2);
+
+ bounds = executeBuildBound(cfDef, restrictions, Bound.END);
+ assertEquals(1, bounds.size());
+ assertComposite(cfDef, bounds.get(0));
+
+ // (clustering_0, clustering1) <= (1, 2)
+ slice = new MultiColumnRestriction.Slice(false);
+ slice.setBound(Relation.Type.LTE, toMultiItemTerminal(value1, value2));
+ restrictions = new Restriction[] { slice, slice };
+
+ bounds = executeBuildBound(cfDef, restrictions, Bound.START);
+ assertEquals(1, bounds.size());
+ assertComposite(cfDef, bounds.get(0));
+
+ bounds = executeBuildBound(cfDef, restrictions, Bound.END);
+ assertEquals(1, bounds.size());
+ assertForRelationComposite(cfDef, bounds.get(0), Relation.Type.LTE, value1, value2);
+
+ // (clustering_0, clustering1) < (1, 2)
+ slice = new MultiColumnRestriction.Slice(false);
+ slice.setBound(Relation.Type.LT, toMultiItemTerminal(value1, value2));
+ restrictions = new Restriction[] { slice, slice };
+
+ bounds = executeBuildBound(cfDef, restrictions, Bound.START);
+ assertEquals(1, bounds.size());
+ assertComposite(cfDef, bounds.get(0));
+
+ bounds = executeBuildBound(cfDef, restrictions, Bound.END);
+ assertEquals(1, bounds.size());
+ assertForRelationComposite(cfDef, bounds.get(0), Relation.Type.LT, value1, value2);
+
+ // (clustering_0, clustering1) > (1, 2) AND (clustering_0) < (2)
+ slice = new MultiColumnRestriction.Slice(false);
+ slice.setBound(Relation.Type.GT, toMultiItemTerminal(value1, value2));
+ slice.setBound(Relation.Type.LT, toMultiItemTerminal(value2));
+ restrictions = new Restriction[] { slice, slice };
+
+ bounds = executeBuildBound(cfDef, restrictions, Bound.START);
+ assertEquals(1, bounds.size());
+ assertForRelationComposite(cfDef, bounds.get(0), Relation.Type.GT, value1, value2);
+
+ bounds = executeBuildBound(cfDef, restrictions, Bound.END);
+ assertEquals(1, bounds.size());
+ assertForRelationComposite(cfDef, bounds.get(0), Relation.Type.LT, value2);
+
+ // (clustering_0, clustering1) >= (1, 2) AND (clustering_0, clustering1) <= (2, 1)
+ slice = new MultiColumnRestriction.Slice(false);
+ slice.setBound(Relation.Type.GTE, toMultiItemTerminal(value1, value2));
+ slice.setBound(Relation.Type.LTE, toMultiItemTerminal(value2, value1));
+ restrictions = new Restriction[] { slice, slice };
+
+ bounds = executeBuildBound(cfDef, restrictions, Bound.START);
+ assertEquals(1, bounds.size());
+ assertForRelationComposite(cfDef, bounds.get(0), Relation.Type.GTE, value1, value2);
+
+ bounds = executeBuildBound(cfDef, restrictions, Bound.END);
+ assertEquals(1, bounds.size());
+ assertForRelationComposite(cfDef, bounds.get(0), Relation.Type.LTE, value2, value1);
+ }
+
+ /**
+ * Test mixing single and multi equals restrictions (e.g. clustering_0 = 1 AND (clustering_1, clustering_2) = (2, 3))
+ */
+ @Test
+ public void testBuildBoundWithSingleEqAndMultiEqRestrictions() throws Exception
+ {
+ ByteBuffer value1 = ByteBufferUtil.bytes(1);
+ ByteBuffer value2 = ByteBufferUtil.bytes(2);
+ ByteBuffer value3 = ByteBufferUtil.bytes(3);
+ ByteBuffer value4 = ByteBufferUtil.bytes(4);
+
+ // clustering_0 = 1 AND (clustering_1, clustering_2) = (2, 3)
+ SingleColumnRestriction.EQ singleEq = new SingleColumnRestriction.EQ(toTerm(value1), false);
+ MultiColumnRestriction.EQ multiEq = new MultiColumnRestriction.EQ(toMultiItemTerminal(value2, value3), false);
+ Restriction[] restrictions = new Restriction[] { singleEq, multiEq, multiEq };
+ CFDefinition cfDef = createCFDefinition(restrictions.length);
+
+ List<ByteBuffer> bounds = executeBuildBound(cfDef, restrictions, Bound.START);
+ assertEquals(1, bounds.size());
+ assertComposite(cfDef, bounds.get(0), value1, value2, value3);
+
+ bounds = executeBuildBound(cfDef, restrictions, Bound.END);
+ assertEquals(1, bounds.size());
+ assertComposite(cfDef, bounds.get(0), value1, value2, value3);
+
+ // clustering_0 = 1 AND clustering_1 = 2 AND (clustering_2, clustering_3) = (3, 4)
+ singleEq = new SingleColumnRestriction.EQ(toTerm(value1), false);
+ SingleColumnRestriction.EQ singleEq2 = new SingleColumnRestriction.EQ(toTerm(value2), false);
+ multiEq = new MultiColumnRestriction.EQ(toMultiItemTerminal(value3, value4), false);
+ restrictions = new Restriction[] { singleEq, singleEq2, multiEq, multiEq };
+ cfDef = createCFDefinition(restrictions.length);
+
+ bounds = executeBuildBound(cfDef, restrictions, Bound.START);
+ assertEquals(1, bounds.size());
+ assertComposite(cfDef, bounds.get(0), value1, value2, value3, value4);
+
+ bounds = executeBuildBound(cfDef, restrictions, Bound.END);
+ assertEquals(1, bounds.size());
+ assertComposite(cfDef, bounds.get(0), value1, value2, value3, value4);
+
+ // (clustering_0, clustering_1) = (1, 2) AND clustering_2 = 3
+ singleEq = new SingleColumnRestriction.EQ(toTerm(value3), false);
+ multiEq = new MultiColumnRestriction.EQ(toMultiItemTerminal(value1, value2), false);
+ restrictions = new Restriction[] { multiEq, multiEq, singleEq };
+ cfDef = createCFDefinition(restrictions.length);
+
+ bounds = executeBuildBound(cfDef, restrictions, Bound.START);
+ assertEquals(1, bounds.size());
+ assertComposite(cfDef, bounds.get(0), value1, value2, value3);
+
+ bounds = executeBuildBound(cfDef, restrictions, Bound.END);
+ assertEquals(1, bounds.size());
+ assertComposite(cfDef, bounds.get(0), value1, value2, value3);
+
+ // clustering_0 = 1 AND (clustering_1, clustering_2) = (2, 3) AND clustering_3 = 4
+ singleEq = new SingleColumnRestriction.EQ(toTerm(value1), false);
+ singleEq2 = new SingleColumnRestriction.EQ(toTerm(value4), false);
+ multiEq = new MultiColumnRestriction.EQ(toMultiItemTerminal(value2, value3), false);
+ restrictions = new Restriction[] { singleEq, multiEq, multiEq, singleEq2 };
+ cfDef = createCFDefinition(restrictions.length);
+
+ bounds = executeBuildBound(cfDef, restrictions, Bound.START);
+ assertEquals(1, bounds.size());
+ assertComposite(cfDef, bounds.get(0), value1, value2, value3, value4);
+
+ bounds = executeBuildBound(cfDef, restrictions, Bound.END);
+ assertEquals(1, bounds.size());
+ assertComposite(cfDef, bounds.get(0), value1, value2, value3, value4);
+ }
+
+ /**
+ * Test clustering_0 = 1 AND (clustering_1, clustering_2) IN ((2, 3), (4, 5))
+ */
+ @Test
+ public void testBuildBoundWithSingleEqAndMultiINRestrictions() throws Exception
+ {
+ ByteBuffer value1 = ByteBufferUtil.bytes(1);
+ ByteBuffer value2 = ByteBufferUtil.bytes(2);
+ ByteBuffer value3 = ByteBufferUtil.bytes(3);
+ ByteBuffer value4 = ByteBufferUtil.bytes(4);
+ ByteBuffer value5 = ByteBufferUtil.bytes(5);
+
+ // clustering_0 = 1 AND (clustering_1, clustering_2) IN ((2, 3), (4, 5))
+ SingleColumnRestriction.EQ singleEq = new SingleColumnRestriction.EQ(toTerm(value1), false);
+ MultiColumnRestriction.IN multiIn =
+ new MultiColumnRestriction.InWithValues(asList(toMultiItemTerminal(value2, value3),
+ toMultiItemTerminal(value4, value5)));
+
+ Restriction[] restrictions = new Restriction[] { singleEq, multiIn, multiIn };
+ CFDefinition cfDef = createCFDefinition(restrictions.length);
+
+ List<ByteBuffer> bounds = executeBuildBound(cfDef, restrictions, Bound.START);
+ assertEquals(2, bounds.size());
+ assertComposite(cfDef, bounds.get(0), value1, value2, value3);
+ assertComposite(cfDef, bounds.get(1), value1, value4, value5);
+
+ bounds = executeBuildBound(cfDef, restrictions, Bound.END);
+ assertEquals(2, bounds.size());
+ assertComposite(cfDef, bounds.get(0), value1, value2, value3);
+ assertComposite(cfDef, bounds.get(1), value1, value4, value5);
+
+ // clustering_0 = 1 AND (clustering_1, clustering_2) IN ((2, 3))
+ singleEq = new SingleColumnRestriction.EQ(toTerm(value1), false);
+ multiIn = new MultiColumnRestriction.InWithValues(asList(toMultiItemTerminal(value2, value3),
+ toMultiItemTerminal(value4, value5)));
+
+ restrictions = new Restriction[] { singleEq, multiIn, multiIn };
+ cfDef = createCFDefinition(restrictions.length);
+
+ bounds = executeBuildBound(cfDef, restrictions, Bound.START);
+ assertEquals(2, bounds.size());
+ assertComposite(cfDef, bounds.get(0), value1, value2, value3);
+ assertComposite(cfDef, bounds.get(1), value1, value4, value5);
+
+ // clustering_0 = 1 AND clustering_1 = 5 AND (clustering_2, clustering_3) IN ((2, 3), (4, 5))
+ singleEq = new SingleColumnRestriction.EQ(toTerm(value1), false);
+ SingleColumnRestriction.EQ singleEq2 = new SingleColumnRestriction.EQ(toTerm(value5), false);
+ multiIn = new MultiColumnRestriction.InWithValues(asList(toMultiItemTerminal(value2, value3),
+ toMultiItemTerminal(value4, value5)));
+
+ restrictions = new Restriction[] { singleEq, singleEq2, multiIn, multiIn };
+ cfDef = createCFDefinition(restrictions.length);
+
+ bounds = executeBuildBound(cfDef, restrictions, Bound.START);
+ assertEquals(2, bounds.size());
+ assertComposite(cfDef, bounds.get(0), value1, value5, value2, value3);
+ assertComposite(cfDef, bounds.get(1), value1, value5, value4, value5);
+
+ bounds = executeBuildBound(cfDef, restrictions, Bound.END);
+ assertEquals(2, bounds.size());
+ assertComposite(cfDef, bounds.get(0), value1, value5, value2, value3);
+ assertComposite(cfDef, bounds.get(1), value1, value5, value4, value5);
+ }
+
+ /**
+ * Test mixing single equal restrictions with multi-column slice restrictions
+ * (e.g. clustering_0 = 1 AND (clustering_1, clustering_2) > (2, 3))
+ */
+ @Test
+ public void testBuildBoundWithSingleEqAndSliceRestrictions() throws Exception
+ {
+ ByteBuffer value1 = ByteBufferUtil.bytes(1);
+ ByteBuffer value2 = ByteBufferUtil.bytes(2);
+ ByteBuffer value3 = ByteBufferUtil.bytes(3);
+ ByteBuffer value4 = ByteBufferUtil.bytes(4);
+ ByteBuffer value5 = ByteBufferUtil.bytes(5);
+
+ // clustering_0 = 1 AND (clustering_1, clustering_2) > (2, 3)
+ SingleColumnRestriction.EQ singleEq = new SingleColumnRestriction.EQ(toTerm(value1), false);
+ MultiColumnRestriction.Slice multiSlice = new MultiColumnRestriction.Slice(false);
+ multiSlice.setBound(Relation.Type.GT, toMultiItemTerminal(value2, value3));
+
+ Restriction[] restrictions = new Restriction[] { singleEq, multiSlice, multiSlice };
+ CFDefinition cfDef = createCFDefinition(restrictions.length);
+
+ List<ByteBuffer> bounds = executeBuildBound(cfDef, restrictions, Bound.START);
+ assertEquals(1, bounds.size());
+ assertForRelationComposite(cfDef, bounds.get(0), Relation.Type.GT, value1, value2, value3);
+
+ bounds = executeBuildBound(cfDef, restrictions, Bound.END);
+ assertEquals(1, bounds.size());
+ assertEndOfRangeComposite(cfDef, bounds.get(0), value1);
+
+ // clustering_0 = 1 AND (clustering_1, clustering_2) > (2, 3) AND (clustering_1) < (4)
+ singleEq = new SingleColumnRestriction.EQ(toTerm(value1), false);
+ multiSlice = new MultiColumnRestriction.Slice(false);
+ multiSlice.setBound(Relation.Type.GT, toMultiItemTerminal(value2, value3));
+ multiSlice.setBound(Relation.Type.LT, toMultiItemTerminal(value4));
+
+ restrictions = new Restriction[] { singleEq, multiSlice, multiSlice };
+
+ bounds = executeBuildBound(cfDef, restrictions, Bound.START);
+ assertEquals(1, bounds.size());
+ assertForRelationComposite(cfDef, bounds.get(0), Relation.Type.GT, value1, value2, value3);
+
+ bounds = executeBuildBound(cfDef, restrictions, Bound.END);
+ assertEquals(1, bounds.size());
+ assertForRelationComposite(cfDef, bounds.get(0), Relation.Type.LT, value1, value4);
+
+ // clustering_0 = 1 AND (clustering_1, clustering_2) => (2, 3) AND (clustering_1, clustering_2) <= (4, 5)
+ singleEq = new SingleColumnRestriction.EQ(toTerm(value1), false);
+ multiSlice = new MultiColumnRestriction.Slice(false);
+ multiSlice.setBound(Relation.Type.GTE, toMultiItemTerminal(value2, value3));
+ multiSlice.setBound(Relation.Type.LTE, toMultiItemTerminal(value4, value5));
+
+ restrictions = new Restriction[] { singleEq, multiSlice, multiSlice };
+
+ bounds = executeBuildBound(cfDef, restrictions, Bound.START);
+ assertEquals(1, bounds.size());
+ assertForRelationComposite(cfDef, bounds.get(0), Relation.Type.GTE, value1, value2, value3);
+
+ bounds = executeBuildBound(cfDef, restrictions, Bound.END);
+ assertEquals(1, bounds.size());
+ assertForRelationComposite(cfDef, bounds.get(0), Relation.Type.LTE, value1, value4, value5);
+ }
+
+ /**
+ * Test mixing multi equal restrictions with single-column slice restrictions
+ * (e.g. clustering_0 = 1 AND (clustering_1, clustering_2) > (2, 3))
+ */
+ @Test
+ public void testBuildBoundWithMultiEqAndSingleSliceRestrictions() throws Exception
+ {
+ ByteBuffer value1 = ByteBufferUtil.bytes(1);
+ ByteBuffer value2 = ByteBufferUtil.bytes(2);
+ ByteBuffer value3 = ByteBufferUtil.bytes(3);
+
+ // (clustering_0, clustering_1) = (1, 2) AND clustering_2 > 3
+ MultiColumnRestriction.EQ multiEq = new MultiColumnRestriction.EQ(toMultiItemTerminal(value1, value2), false);
+ SingleColumnRestriction.Slice singleSlice = new SingleColumnRestriction.Slice(false);
+ singleSlice.setBound(Relation.Type.GT, toTerm(value3));
+
+ Restriction[] restrictions = new Restriction[] { multiEq, multiEq, singleSlice };
+ CFDefinition cfDef = createCFDefinition(restrictions.length);
+
+ List<ByteBuffer> bounds = executeBuildBound(cfDef, restrictions, Bound.START);
+ assertEquals(1, bounds.size());
+ assertForRelationComposite(cfDef, bounds.get(0), Relation.Type.GT, value1, value2, value3);
+
+ bounds = executeBuildBound(cfDef, restrictions, Bound.END);
+ assertEquals(1, bounds.size());
+ assertEndOfRangeComposite(cfDef, bounds.get(0), value1, value2);
+ }
+
+ @Test
+ public void testBuildBoundWithSeveralMultiColumnRestrictions() throws Exception
+ {
+ ByteBuffer value1 = ByteBufferUtil.bytes(1);
+ ByteBuffer value2 = ByteBufferUtil.bytes(2);
+ ByteBuffer value3 = ByteBufferUtil.bytes(3);
+ ByteBuffer value4 = ByteBufferUtil.bytes(4);
+ ByteBuffer value5 = ByteBufferUtil.bytes(5);
+
+ // (clustering_0, clustering_1) = (1, 2) AND (clustering_2, clustering_3) > (3, 4)
+ MultiColumnRestriction.EQ multiEq = new MultiColumnRestriction.EQ(toMultiItemTerminal(value1, value2), false);
+ MultiColumnRestriction.Slice multiSlice = new MultiColumnRestriction.Slice(false);
+ multiSlice.setBound(Relation.Type.GT, toMultiItemTerminal(value3, value4));
+
+ Restriction[] restrictions = new Restriction[] { multiEq, multiEq, multiSlice, multiSlice};
+ CFDefinition cfDef = createCFDefinition(restrictions.length);
+
+ List<ByteBuffer> bounds = executeBuildBound(cfDef, restrictions, Bound.START);
+ assertEquals(1, bounds.size());
+ assertForRelationComposite(cfDef, bounds.get(0), Relation.Type.GT,value1, value2, value3, value4);
+
+ bounds = executeBuildBound(cfDef, restrictions, Bound.END);
+ assertEquals(1, bounds.size());
+ assertEndOfRangeComposite(cfDef, bounds.get(0), value1, value2);
+
+ // (clustering_0, clustering_1) = (1, 2) AND (clustering_2, clustering_3) IN ((3, 4), (4, 5))
+ multiEq = new MultiColumnRestriction.EQ(toMultiItemTerminal(value1, value2), false);
+ MultiColumnRestriction.IN multiIn =
+ new MultiColumnRestriction.InWithValues(asList(toMultiItemTerminal(value3, value4),
+ toMultiItemTerminal(value4, value5)));
+
+ restrictions = new Restriction[] { multiEq, multiEq, multiIn, multiIn};
+ cfDef = createCFDefinition(restrictions.length);
+
+ bounds = executeBuildBound(cfDef, restrictions, Bound.START);
+ assertEquals(2, bounds.size());
+ assertComposite(cfDef, bounds.get(0), value1, value2, value3, value4);
+ assertComposite(cfDef, bounds.get(1), value1, value2, value4, value5);
+
+ bounds = executeBuildBound(cfDef, restrictions, Bound.END);
+ assertEquals(2, bounds.size());
+ assertComposite(cfDef, bounds.get(0), value1, value2, value3, value4);
+ assertComposite(cfDef, bounds.get(1), value1, value2, value4, value5);
+
+ // (clustering_0, clustering_1) = (1, 2) AND (clustering_2, clustering_3) = ((3, 4), (4, 5))
+ multiEq = new MultiColumnRestriction.EQ(toMultiItemTerminal(value1, value2), false);
+ MultiColumnRestriction.EQ multiEq2 = new MultiColumnRestriction.EQ(toMultiItemTerminal(value3, value4), false);
+
+ restrictions = new Restriction[] { multiEq, multiEq, multiEq2, multiEq2};
+ cfDef = createCFDefinition(restrictions.length);
+
+ bounds = executeBuildBound(cfDef, restrictions, Bound.START);
+ assertEquals(1, bounds.size());
+ assertComposite(cfDef, bounds.get(0), value1, value2, value3, value4);
+
+ bounds = executeBuildBound(cfDef, restrictions, Bound.END);
+ assertEquals(1, bounds.size());
+ assertComposite(cfDef, bounds.get(0), value1, value2, value3, value4);
+ }
+
+ /**
+ * Asserts that the specified composite contains the specified elements.
+ *
+ * @param cfDef the Column Family Definition
+ * @param actual the buffer to test
+ * @param elements the expected elements of the composite
+ */
+ private static void assertComposite(CFDefinition cfDef, ByteBuffer actual, ByteBuffer... elements)
+ {
+ ColumnNameBuilder builder = addElements(cfDef.getColumnNameBuilder(), elements);
+ assertArrayEquals("the composite is not the expected one:", actual.array(), builder.build().array());
+ }
+
+ /**
+ * Asserts that the specified composite is an end of range composite that contains the specified elements.
+ *
+ * @param cfDef the Column Family Definition
+ * @param actual the buffer to test
+ * @param elements the expected elements of the composite
+ */
+ private static void assertEndOfRangeComposite(CFDefinition cfDef, ByteBuffer actual, ByteBuffer... elements)
+ {
+ ColumnNameBuilder builder = addElements(cfDef.getColumnNameBuilder(), elements);
+ assertArrayEquals("the composite is not the expected one:", actual.array(), builder.buildAsEndOfRange().array());
+ }
+
+ /**
+ * Asserts that the specified composite is an end of range composite that contains the specified elements.
+ *
+ * @param cfDef the Column Family Definition
+ * @param actual the buffer to test
+ * @param elements the expected elements of the composite
+ */
+ private static void assertForRelationComposite(CFDefinition cfDef,
+ ByteBuffer actual,
+ Relation.Type relType,
+ ByteBuffer... elements)
+ {
+ ColumnNameBuilder builder = addElements(cfDef.getColumnNameBuilder(), elements);
+ assertArrayEquals("the composite is not the expected one:", actual.array(), builder.buildForRelation(relType).array());
+ }
+
+ /**
+ * Adds all the specified elements to the specified builder.
+ *
+ * @param builder the builder to add to
+ * @param elements the elements to add
+ * @return the builder
+ */
+ private static ColumnNameBuilder addElements(ColumnNameBuilder builder, ByteBuffer... elements)
+ {
+ for (int i = 0, m = elements.length; i < m; i++)
+ builder.add(elements[i]);
+ return builder;
+ }
+
+ /**
+ * Calls the <code>SelectStatement.buildBound</code> with the specified restrictions.
+ *
+ * @param cfDef the Column Family Definition
+ * @param restrictions the restrictions
+ * @return the result from the method call to <code>SelectStatement.buildBound</code>
+ * @throws InvalidRequestException if buildBound throw an <code>Exception</code>
+ */
+ private static List<ByteBuffer> executeBuildBound(CFDefinition cfDef,
+ Restriction[] restrictions,
+ Bound bound) throws InvalidRequestException
+ {
+ return SelectStatement.buildBound(bound,
+ new ArrayList<Name>(cfDef.clusteringColumns()),
+ restrictions,
+ false,
+ cfDef,
+ cfDef.getColumnNameBuilder(),
+ Collections.<ByteBuffer>emptyList());
+ }
+
+ /**
+ * Creates a <code>CFDefinition</code> to be used in the tests.
+ *
+ * @param numberOfClusteringColumns the number of clustering columns
+ * @return a new a <code>CFDefinition</code> instance
+ * @throws ConfigurationException if the CFDefinition cannot be created
+ */
+ private static CFDefinition createCFDefinition(int numberOfClusteringColumns) throws ConfigurationException
+ {
+ List<AbstractType<?>> types = new ArrayList<>();
+ for (int i = 0, m = numberOfClusteringColumns; i < m; i++)
+ types.add(Int32Type.instance);
+
+ CompositeType cType = CompositeType.getInstance(types);
+ CFMetaData cfMetaData = new CFMetaData("keyspace", "test", ColumnFamilyType.Standard, cType);
+ ByteBuffer partitionKey = ByteBufferUtil.bytes("partitionKey");
+ cfMetaData.addColumnDefinition(ColumnDefinition.partitionKeyDef(partitionKey, Int32Type.instance, 0));
+
+ for (int i = 0, m = numberOfClusteringColumns; i < m; i++)
+ {
+ ByteBuffer name = ByteBufferUtil.bytes("clustering_" + i);
+ cfMetaData.addColumnDefinition(ColumnDefinition.clusteringKeyDef(name, types.get(i), i));
+ }
+ cfMetaData.rebuild();
+ return new CFDefinition(cfMetaData);
+ }
+
+ /**
+ * Converts the specified values into a <code>MultiItemTerminal</code>.
+ *
+ * @param values the values to convert.
+ * @return the term corresponding to the specified values.
+ */
+ private static Term toMultiItemTerminal(ByteBuffer... values)
+ {
+ return new Tuples.Value(values);
+ }
+
+ /**
+ * Converts the specified value into a term.
+ *
+ * @param value the value to convert.
+ * @return the term corresponding to the specified value.
+ */
+ private static Term toTerm(ByteBuffer value)
+ {
+ return new Constants.Value(value);
+ }
+
+ /**
+ * Converts the specified values into a <code>List</code> of terms.
+ *
+ * @param values the values to convert.
+ * @return a <code>List</code> of terms corresponding to the specified values.
+ */
+ private static List<Term> toTerms(ByteBuffer... values)
+ {
+ List<Term> terms = new ArrayList<>();
+ for (ByteBuffer value : values)
+ terms.add(toTerm(value));
+ return terms;
+ }
+}
[2/2] cassandra git commit: Fix mixed single and multi-column
relation support
Posted by ty...@apache.org.
Fix mixed single and multi-column relation support
Patch by Benjamin Lerer; reviewed by Tyler Hobbs for CASSANDRA-8613
Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/90a012a1
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/90a012a1
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/90a012a1
Branch: refs/heads/cassandra-2.0
Commit: 90a012a1fe29edbb70de30d5f683e10cc27e1602
Parents: fb67c41
Author: blerer <b_...@hotmail.com>
Authored: Thu Mar 5 12:25:03 2015 -0600
Committer: Tyler Hobbs <ty...@datastax.com>
Committed: Thu Mar 5 12:25:03 2015 -0600
----------------------------------------------------------------------
CHANGES.txt | 2 +
.../cql3/statements/SelectStatement.java | 384 +++++---
.../cassandra/cql3/MultiColumnRelationTest.java | 186 +++-
.../cql3/statements/SelectStatementTest.java | 973 +++++++++++++++++++
4 files changed, 1386 insertions(+), 159 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/cassandra/blob/90a012a1/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index faa14d5..462a8d1 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,6 @@
2.0.13:
+ * Fix regression in mixed single and multi-column relation support for
+ SELECT statements (CASSANDRA-8613)
* Add ability to limit number of native connections (CASSANDRA-8086)
* Add offline tool to relevel sstables (CASSANDRA-8301)
* Preserve stream ID for more protocol errors (CASSANDRA-8848)
http://git-wip-us.apache.org/repos/asf/cassandra/blob/90a012a1/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 59ed6e1..6b3c781 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.annotations.VisibleForTesting;
import com.google.common.base.Joiner;
import com.google.common.base.Objects;
import com.google.common.base.Predicate;
@@ -31,7 +32,6 @@ import org.github.jamm.MemoryMeter;
import org.apache.cassandra.auth.Permission;
import org.apache.cassandra.cql3.*;
import org.apache.cassandra.cql3.CFDefinition.Name;
-import org.apache.cassandra.cql3.CFDefinition.Name.Kind;
import org.apache.cassandra.transport.messages.ResultMessage;
import org.apache.cassandra.config.CFMetaData;
import org.apache.cassandra.db.*;
@@ -641,7 +641,13 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
return ByteBufferUtil.EMPTY_BYTE_BUFFER;
// We deal with IN queries for keys in other places, so we know buildBound will return only one result
- return buildBound(b, cfDef.partitionKeys(), keyRestrictions, false, cfDef.getKeyNameBuilder(), variables).get(0);
+ return buildBound(b,
+ new ArrayList<Name>(cfDef.partitionKeys()),
+ keyRestrictions,
+ false,
+ cfDef,
+ cfDef.getKeyNameBuilder(),
+ variables).get(0);
}
private Token getTokenBound(Bound b, List<ByteBuffer> variables, IPartitioner<?> p) throws InvalidRequestException
@@ -713,18 +719,35 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
ColumnNameBuilder builder = cfDef.getColumnNameBuilder();
Iterator<CFDefinition.Name> idIter = cfDef.clusteringColumns().iterator();
- for (Restriction r : columnRestrictions)
+ while (idIter.hasNext())
{
CFDefinition.Name name = idIter.next();
+ Restriction r = columnRestrictions[name.position];
assert r != null && !r.isSlice();
if (r.isEQ())
{
List<ByteBuffer> values = r.values(variables);
- ByteBuffer val = values.get(0);
- if (val == null)
- throw new InvalidRequestException(String.format("Invalid null value for clustering key part %s", name.name));
- builder.add(val);
+ if (r.isMultiColumn())
+ {
+ for (int i = 0, m = values.size(); i < m; i++)
+ {
+ if (i != 0)
+ name = idIter.next();
+
+ ByteBuffer val = values.get(i);
+ if (val == null)
+ throw new InvalidRequestException(String.format("Invalid null value in condition for column %s", name));
+ builder.add(val);
+ }
+ }
+ else
+ {
+ ByteBuffer val = values.get(0);
+ if (val == null)
+ throw new InvalidRequestException(String.format("Invalid null value for clustering key part %s", name.name));
+ builder.add(val);
+ }
}
else
{
@@ -841,37 +864,23 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
return false;
}
- private List<ByteBuffer> buildBound(Bound bound,
- Collection<CFDefinition.Name> names,
+ @VisibleForTesting
+ static List<ByteBuffer> buildBound(Bound bound,
+ List<CFDefinition.Name> names,
Restriction[] restrictions,
boolean isReversed,
+ CFDefinition cfDef,
ColumnNameBuilder builder,
List<ByteBuffer> variables) throws InvalidRequestException
{
-
- // check the first restriction to see if we're dealing with a multi-column restriction
- if (!names.isEmpty())
- {
- Restriction firstRestriction = restrictions[0];
- if (firstRestriction != null && firstRestriction.isMultiColumn())
- {
- if (firstRestriction.isSlice())
- return buildMultiColumnSliceBound(bound, names, (MultiColumnRestriction.Slice) firstRestriction, isReversed, builder, variables);
- else if (firstRestriction.isIN())
- return buildMultiColumnInBound(bound, (MultiColumnRestriction.IN) firstRestriction, isReversed, builder, variables);
- else
- return buildMultiColumnEQBound(bound, (MultiColumnRestriction.EQ) firstRestriction, isReversed, builder, variables);
- }
- }
-
// The end-of-component of composite doesn't depend on whether the
// component type is reversed or not (i.e. the ReversedType is applied
// to the component comparator but not to the end-of-component itself),
// it only depends on whether the slice is reversed
Bound eocBound = isReversed ? Bound.reverse(bound) : bound;
- for (Iterator<CFDefinition.Name> iter = names.iterator(); iter.hasNext();)
+ for (int i = 0; i < names.size(); i++)
{
- CFDefinition.Name name = iter.next();
+ CFDefinition.Name name = names.get(i);
// In a restriction, we always have Bound.START < Bound.END for the "base" comparator.
// So if we're doing a reverse slice, we must inverse the bounds when giving them as start and end of the slice filter.
@@ -889,116 +898,130 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
}
if (r.isSlice())
{
- builder.add(getSliceValue(r, b, variables));
+ if (r.isMultiColumn())
+ {
+ List<ByteBuffer> values = ((MultiColumnRestriction.Slice) r).componentBounds(b, variables);
+ List<Name> columns = subList(names, i, values.size());
+ addComponents(builder, columns, values);
+ }
+ else
+ {
+ addComponent(builder, name, getSliceValue(r, b, variables));
+ }
+
Relation.Type relType = ((Restriction.Slice)r).getRelation(eocBound, b);
return Collections.singletonList(builder.buildForRelation(relType));
}
- else
+
+ if (r.isIN())
{
+ // The IN query might not have listed the values in comparator order, so we need to re-sort
+ // the bounds lists to make sure the slices works correctly (also, to avoid duplicates).
+ TreeSet<ByteBuffer> inValues = new TreeSet<>(isReversed ? cfDef.cfm.comparator.reverseComparator : cfDef.cfm.comparator);
+
+ if (r.isMultiColumn())
+ {
+ List<List<ByteBuffer>> splitInValues = ((MultiColumnRestriction.IN) r).splitValues(variables);
+ for (List<ByteBuffer> components : splitInValues)
+ {
+ ColumnNameBuilder copy = builder.copy();
+ List<Name> columns = subList(names, i, components.size());
+ addComponents(copy, columns, components);
+
+ inValues.add(buildColumnName(copy, eocBound));
+ }
+ return new ArrayList<>(inValues);
+ }
+
List<ByteBuffer> values = r.values(variables);
if (values.size() != 1)
{
// IN query, we only support it on the clustering columns
assert name.position == names.size() - 1;
- // The IN query might not have listed the values in comparator order, so we need to re-sort
- // the bounds lists to make sure the slices works correctly (also, to avoid duplicates).
- TreeSet<ByteBuffer> s = new TreeSet<>(isReversed ? cfDef.cfm.comparator.reverseComparator : cfDef.cfm.comparator);
+
for (ByteBuffer val : values)
{
- if (val == null)
- throw new InvalidRequestException(String.format("Invalid null clustering key part %s", name));
- ColumnNameBuilder copy = builder.copy().add(val);
- // See below for why this
- s.add((eocBound == Bound.END && copy.remainingCount() > 0) ? copy.buildAsEndOfRange() : copy.build());
+ ColumnNameBuilder copy = builder.copy();
+ addComponent(copy, name, val);
+ inValues.add(buildColumnName(copy, eocBound));
}
- return new ArrayList<>(s);
+ return new ArrayList<>(inValues);
}
+ }
- ByteBuffer val = values.get(0);
- if (val == null)
- throw new InvalidRequestException(String.format("Invalid null clustering key part %s", name));
- builder.add(val);
+ List<ByteBuffer> values = r.values(variables);
+ if (r.isMultiColumn())
+ {
+ List<Name> columns = subList(names, i, values.size());
+ addComponents(builder, columns, values);
+ i += values.size() - 1;
+ }
+ else
+ {
+ addComponent(builder, name, values.get(0));
}
}
- // Means no relation at all or everything was an equal
+ return Collections.singletonList(buildColumnName(builder, eocBound));
+ }
+
+ /**
+ * Returns a view of the portion of the list starting at the index offset and containing the number of elements
+ * specified.
+ *
+ * @param list the original list
+ * @param offset the index offset
+ * @param length the number of elements
+ * @return a view of the specified range within this list
+ */
+ private static <T> List<T> subList(List<T> list, int offset, int length)
+ {
+ return list.subList(offset, offset + length);
+ }
+
+ /**
+ * Builds the column name when there was not slice.
+ *
+ * @param builder the column name builder
+ * @param eocBound the End of Component bound
+ * @return the column name
+ */
+ private static ByteBuffer buildColumnName(ColumnNameBuilder builder, Bound eocBound)
+ {
// Note: if the builder is "full", there is no need to use the end-of-component bit. For columns selection,
// it would be harmless to do it. However, we use this method got the partition key too. And when a query
// with 2ndary index is done, and with the the partition provided with an EQ, we'll end up here, and in that
// case using the eoc would be bad, since for the random partitioner we have no guarantee that
// builder.buildAsEndOfRange() will sort after builder.build() (see #5240).
- return Collections.singletonList((eocBound == Bound.END && builder.remainingCount() > 0) ? builder.buildAsEndOfRange() : builder.build());
+ return (eocBound == Bound.END && builder.remainingCount() > 0) ? builder.buildAsEndOfRange() : builder.build();
}
- private List<ByteBuffer> buildMultiColumnSliceBound(Bound bound,
- Collection<CFDefinition.Name> names,
- MultiColumnRestriction.Slice slice,
- boolean isReversed,
- ColumnNameBuilder builder,
- List<ByteBuffer> variables) throws InvalidRequestException
- {
- Bound eocBound = isReversed ? Bound.reverse(bound) : bound;
-
- Iterator<CFDefinition.Name> iter = names.iterator();
- CFDefinition.Name firstName = iter.next();
- // A hack to preserve pre-6875 behavior for tuple-notation slices where the comparator mixes ASCENDING
- // and DESCENDING orders. This stores the bound for the first component; we will re-use it for all following
- // components, even if they don't match the first component's reversal/non-reversal. Note that this does *not*
- // guarantee correct query results, it just preserves the previous behavior.
- Bound firstComponentBound = isReversed == isReversedType(firstName) ? bound : Bound.reverse(bound);
-
- if (!slice.hasBound(firstComponentBound))
- return Collections.singletonList(builder.componentCount() > 0 && eocBound == Bound.END
- ? builder.buildAsEndOfRange()
- : builder.build());
-
- List<ByteBuffer> vals = slice.componentBounds(firstComponentBound, variables);
- builder.add(vals.get(firstName.position));
-
- while(iter.hasNext())
- {
- CFDefinition.Name name = iter.next();
- if (name.position >= vals.size())
- break;
-
- builder.add(vals.get(name.position));
- }
- Relation.Type relType = slice.getRelation(eocBound, firstComponentBound);
- return Collections.singletonList(builder.buildForRelation(relType));
- }
-
- private List<ByteBuffer> buildMultiColumnInBound(Bound bound,
- MultiColumnRestriction.IN restriction,
- boolean isReversed,
- ColumnNameBuilder builder,
- List<ByteBuffer> variables) throws InvalidRequestException
+ /**
+ * Adds the specified component values to the specified builder.
+ *
+ * @param builder the ColumnNameBuilder to which the values must be added
+ * @param names the column names
+ * @param values the values to add
+ * @throws InvalidRequestException if one of the values is null
+ */
+ private static void addComponents(ColumnNameBuilder builder, List<Name> names, List<ByteBuffer> values) throws InvalidRequestException
{
- List<List<ByteBuffer>> splitInValues = restriction.splitValues(variables);
- Bound eocBound = isReversed ? Bound.reverse(bound) : bound;
-
- // The IN query might not have listed the values in comparator order, so we need to re-sort
- // the bounds lists to make sure the slices works correctly (also, to avoid duplicates).
- TreeSet<ByteBuffer> inValues = new TreeSet<>(isReversed ? cfDef.cfm.comparator.reverseComparator : cfDef.cfm.comparator);
- for (List<ByteBuffer> components : splitInValues)
- {
- ColumnNameBuilder nameBuilder = builder.copy();
- for (ByteBuffer component : components)
- nameBuilder.add(component);
-
- inValues.add((eocBound == Bound.END && nameBuilder.remainingCount() > 0) ? nameBuilder.buildAsEndOfRange() : nameBuilder.build());
- }
- return new ArrayList<>(inValues);
+ for (int i = 0, m = values.size(); i < m; i++)
+ addComponent(builder, names.get(i), values.get(i));
}
- private List<ByteBuffer> buildMultiColumnEQBound(Bound bound, MultiColumnRestriction.EQ restriction, boolean isReversed, ColumnNameBuilder builder, List<ByteBuffer> variables) throws InvalidRequestException
+ /**
+ * Adds the specified component value to the specified builder
+ *
+ * @param builder the ColumnNameBuilder to which the value must be added
+ * @param name the column associated to the value
+ * @param value the value to add
+ * @throws InvalidRequestException if the value is null
+ */
+ private static void addComponent(ColumnNameBuilder builder, Name name, ByteBuffer value) throws InvalidRequestException
{
- Bound eocBound = isReversed ? Bound.reverse(bound) : bound;
- for (ByteBuffer component : restriction.values(variables))
- builder.add(component);
-
- ByteBuffer result = builder.componentCount() > 0 && eocBound == Bound.END
- ? builder.buildAsEndOfRange()
- : builder.build();
- return Collections.singletonList(result);
+ if (value == null)
+ throw new InvalidRequestException(String.format("Invalid null value in condition for column %s", name));
+ builder.add(value);
}
private static boolean isNullRestriction(Restriction r, Bound b)
@@ -1010,16 +1033,19 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
{
Restriction.Slice slice = (Restriction.Slice)r;
assert slice.hasBound(b);
- ByteBuffer val = slice.bound(b, variables);
- if (val == null)
- throw new InvalidRequestException(String.format("Invalid null clustering key part %s", r));
- return val;
+ return slice.bound(b, variables);
}
private List<ByteBuffer> getRequestedBound(Bound b, List<ByteBuffer> variables) throws InvalidRequestException
{
assert isColumnRange();
- return buildBound(b, cfDef.clusteringColumns(), columnRestrictions, isReversed, cfDef.getColumnNameBuilder(), variables);
+ return buildBound(b,
+ new ArrayList<Name>(cfDef.clusteringColumns()),
+ columnRestrictions,
+ isReversed,
+ cfDef,
+ cfDef.getColumnNameBuilder(),
+ variables);
}
public List<IndexExpression> getIndexExpressions(List<ByteBuffer> variables) throws InvalidRequestException
@@ -1409,16 +1435,22 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
{
Collection<CFDefinition.Name> requestedColumns = selection.getColumns();
for (CFDefinition.Name name : requestedColumns)
+ {
if (name.kind != CFDefinition.Name.Kind.KEY_ALIAS && name.kind != CFDefinition.Name.Kind.STATIC)
- throw new InvalidRequestException(String.format("SELECT DISTINCT queries must only request partition key columns and/or static columns (not %s)", name));
+ throw new InvalidRequestException(String.format(
+ "SELECT DISTINCT queries must only request partition key columns and/or static columns (not %s)", name));
+ }
// If it's a key range, we require that all partition key columns are selected so we don't have to bother with post-query grouping.
if (!isKeyRange)
return;
for (CFDefinition.Name name : cfDef.partitionKeys())
+ {
if (!requestedColumns.contains(name))
- throw new InvalidRequestException(String.format("SELECT DISTINCT queries must request all the partition key columns (missing %s)", name));
+ throw new InvalidRequestException(String.format(
+ "SELECT DISTINCT queries must request all the partition key columns (missing %s)", name));
+ }
}
public static class RawStatement extends CFStatement
@@ -1465,8 +1497,6 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
*/
boolean hasQueriableIndex = false;
boolean hasQueriableClusteringColumnIndex = false;
- boolean hasSingleColumnRelations = false;
- boolean hasMultiColumnRelations = false;
for (Relation relation : whereClause)
{
if (relation.isMultiColumn())
@@ -1481,7 +1511,6 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
hasQueriableClusteringColumnIndex |= queriable[1];
Name name = cfDef.get(entity);
names.add(name);
- hasMultiColumnRelations |= Kind.COLUMN_ALIAS.equals(name.kind);
}
updateRestrictionsForRelation(stmt, names, rel, boundNames);
}
@@ -1493,12 +1522,9 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
hasQueriableIndex |= queriable[0];
hasQueriableClusteringColumnIndex |= queriable[1];
Name name = cfDef.get(entity);
- hasSingleColumnRelations |= Kind.COLUMN_ALIAS.equals(name.kind);
updateRestrictionsForRelation(stmt, name, rel, boundNames);
}
}
- if (hasSingleColumnRelations && hasMultiColumnRelations)
- throw new InvalidRequestException("Mixing single column relations and multi column relations on clustering columns is not allowed");
// At this point, the select statement if fully constructed, but we still have a few things to validate
processPartitionKeyRestrictions(stmt, cfDef, hasQueriableIndex);
@@ -1592,10 +1618,12 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
{
List<CFDefinition.Name> restrictedColumns = new ArrayList<>();
Set<CFDefinition.Name> seen = new HashSet<>();
+ Restriction existing = null;
- int previousPosition = -1;
- for (CFDefinition.Name name : names)
+ int previousPosition = names.get(0).position - 1;
+ for (int i = 0, m = names.size(); i < m; i++)
{
+ Name name = names.get(i);
// ensure multi-column restriction only applies to clustering columns
if (name.kind != CFDefinition.Name.Kind.COLUMN_ALIAS)
throw new InvalidRequestException(String.format("Multi-column relations can only be applied to clustering columns: %s", name));
@@ -1611,20 +1639,51 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
throw new InvalidRequestException(String.format(
"Clustering columns may not be skipped in multi-column relations. " +
"They should appear in the PRIMARY KEY order. Got %s", relation));
- else
- throw new InvalidRequestException(String.format(
- "Clustering columns must appear in the PRIMARY KEY order in multi-column relations: %s", relation));
+
+ throw new InvalidRequestException(String.format(
+ "Clustering columns must appear in the PRIMARY KEY order in multi-column relations: %s", relation));
}
previousPosition++;
- Restriction existing = getExistingRestriction(stmt, name);
+ Restriction previous = existing;
+ existing = getExistingRestriction(stmt, name);
Relation.Type operator = relation.operator();
if (existing != null)
{
if (operator == Relation.Type.EQ || operator == Relation.Type.IN)
- throw new InvalidRequestException(String.format("Column \"%s\" cannot be restricted by more than one relation if it is in an %s relation", name, relation.operator()));
+ {
+ throw new InvalidRequestException(String.format(
+ "Column \"%s\" cannot be restricted by more than one relation if it is in an %s relation",
+ name, relation.operator()));
+ }
else if (!existing.isSlice())
- throw new InvalidRequestException(String.format("Column \"%s\" cannot be restricted by an equality relation and an inequality relation", name));
+ {
+ throw new InvalidRequestException(String.format(
+ "Column \"%s\" cannot be restricted by an equality relation and an inequality relation", name));
+ }
+ else
+ {
+ if (!existing.isMultiColumn())
+ {
+ throw new InvalidRequestException(String.format(
+ "Column \"%s\" cannot have both tuple-notation inequalities and single-column inequalities: %s",
+ name, relation));
+ }
+
+ boolean existingRestrictionStartBefore =
+ (i == 0 && name.position != 0 && stmt.columnRestrictions[name.position - 1] == existing);
+
+ boolean existingRestrictionStartAfter = (i != 0 && previous != existing);
+
+ if (existingRestrictionStartBefore || existingRestrictionStartAfter)
+ {
+ throw new InvalidRequestException(String.format(
+ "Column \"%s\" cannot be restricted by two tuple-notation inequalities not starting with the same column: %s",
+ name, relation));
+ }
+
+ checkBound(existing, name, operator);
+ }
}
restrictedColumns.add(name);
}
@@ -1678,20 +1737,41 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
{
Term t = relation.getValue().prepare(names);
t.collectMarkerSpecification(boundNames);
+
+ Restriction.Slice restriction = (Restriction.Slice) getExistingRestriction(stmt, names.get(0));
+ if (restriction == null)
+ restriction = new MultiColumnRestriction.Slice(false);
+ restriction.setBound(relation.operator(), t);
+
for (CFDefinition.Name name : names)
{
- Restriction.Slice restriction = (Restriction.Slice)getExistingRestriction(stmt, name);
- if (restriction == null)
- restriction = new MultiColumnRestriction.Slice(onToken);
- else if (!restriction.isMultiColumn())
- throw new InvalidRequestException(String.format("Column \"%s\" cannot have both tuple-notation inequalities and single-column inequalities: %s", name, relation));
- restriction.setBound(relation.operator(), t);
stmt.columnRestrictions[name.position] = restriction;
}
}
}
}
+ /**
+ * Checks that the operator for the specified column is compatible with the bounds of the existing restriction.
+ *
+ * @param existing the existing restriction
+ * @param name the column name
+ * @param relType the type of relation
+ * @throws InvalidRequestException if the operator is not compatible with the bounds of the existing restriction
+ */
+ private static void checkBound(Restriction existing, Name name, Relation.Type relType) throws InvalidRequestException
+ {
+ Restriction.Slice existingSlice = (Restriction.Slice) existing;
+
+ if (existingSlice.hasBound(Bound.START) && (relType == Relation.Type.GT || relType == Relation.Type.GTE))
+ throw new InvalidRequestException(String.format(
+ "More than one restriction was found for the start bound on %s", name.name));
+
+ if (existingSlice.hasBound(Bound.END) && (relType == Relation.Type.LT || relType == Relation.Type.LTE))
+ throw new InvalidRequestException(String.format(
+ "More than one restriction was found for the end bound on %s", name.name));
+ }
+
private Restriction getExistingRestriction(SelectStatement stmt, CFDefinition.Name name)
{
switch (name.kind)
@@ -1924,7 +2004,7 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
// the column is indexed that is.
boolean canRestrictFurtherComponents = true;
CFDefinition.Name previous = null;
- boolean previousIsSlice = false;
+ Restriction previousRestriction = null;
Iterator<CFDefinition.Name> iter = cfDef.clusteringColumns().iterator();
for (int i = 0; i < stmt.columnRestrictions.length; i++)
{
@@ -1934,29 +2014,35 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
if (restriction == null)
{
canRestrictFurtherComponents = false;
- previousIsSlice = false;
}
else if (!canRestrictFurtherComponents)
{
// We're here if the previous clustering column was either not restricted or was a slice.
- // We can't restrict the current column unless:
- // 1) we're in the special case of the 'tuple' notation from #4851 which we expand as multiple
- // consecutive slices: in which case we're good with this restriction and we continue
- // 2) we have a 2ndary index, in which case we have to use it but can skip more validation
- if (!(previousIsSlice && restriction.isSlice() && restriction.isMultiColumn()))
+
+ // we can continue if we are in the special case of a slice 'tuple' notation from #4851
+ if (restriction != previousRestriction)
{
+ // if we have a 2ndary index, we need to use it
if (hasQueriableIndex)
{
- stmt.usesSecondaryIndexing = true; // handle gaps and non-keyrange cases.
+ stmt.usesSecondaryIndexing = true;
break;
}
+
+ if (previousRestriction == null)
+ throw new InvalidRequestException(String.format(
+ "PRIMARY KEY column \"%s\" cannot be restricted (preceding column \"%s\" is not restricted)", cname, previous));
+
+ if (previousRestriction.isMultiColumn() && previousRestriction.isIN())
+ throw new InvalidRequestException(String.format(
+ "PRIMARY KEY column \"%s\" cannot be restricted (preceding column \"%s\" is restricted by an IN tuple notation)", cname, previous));
+
throw new InvalidRequestException(String.format(
- "PRIMARY KEY column \"%s\" cannot be restricted (preceding column \"%s\" is either not restricted or by a non-EQ relation)", cname, previous));
+ "PRIMARY KEY column \"%s\" cannot be restricted (preceding column \"%s\" is restricted by a non-EQ relation)", cname, previous));
}
}
else if (restriction.isSlice())
{
- previousIsSlice = true;
canRestrictFurtherComponents = false;
Restriction.Slice slice = (Restriction.Slice)restriction;
// For non-composite slices, we don't support internally the difference between exclusive and
@@ -1970,9 +2056,13 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
throw new InvalidRequestException(String.format("Clustering column \"%s\" cannot be restricted by an IN relation", cname));
if (stmt.selectACollection())
throw new InvalidRequestException(String.format("Cannot restrict column \"%s\" by IN relation as a collection is selected by the query", cname));
+
+ if (restriction.isMultiColumn())
+ canRestrictFurtherComponents = false;
}
previous = cname;
+ previousRestriction = restriction;
}
}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/90a012a1/test/unit/org/apache/cassandra/cql3/MultiColumnRelationTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/MultiColumnRelationTest.java b/test/unit/org/apache/cassandra/cql3/MultiColumnRelationTest.java
index 30a9226..e3ccba5 100644
--- a/test/unit/org/apache/cassandra/cql3/MultiColumnRelationTest.java
+++ b/test/unit/org/apache/cassandra/cql3/MultiColumnRelationTest.java
@@ -65,7 +65,7 @@ public class MultiColumnRelationTest
executeSchemaChange(
"CREATE TABLE IF NOT EXISTS %s.single_partition" + tableSuffix + "(a int PRIMARY KEY, b int)" + compactOption);
executeSchemaChange(
- "CREATE TABLE IF NOT EXISTS %s.compound_partition" +tableSuffix + "(a int, b int, c int, PRIMARY KEY ((a, b)))" + compactOption);
+ "CREATE TABLE IF NOT EXISTS %s.compound_partition" + tableSuffix + "(a int, b int, c int, PRIMARY KEY ((a, b)))" + compactOption);
executeSchemaChange(
"CREATE TABLE IF NOT EXISTS %s.single_clustering" + tableSuffix + "(a int, b int, c int, PRIMARY KEY (a, b))" + compactOption);
executeSchemaChange(
@@ -135,12 +135,6 @@ public class MultiColumnRelationTest
return null;
}
- @Test(expected=InvalidRequestException.class)
- public void testMixMultiColumnRelationsAndSingleColumn() throws Throwable
- {
- execute("SELECT * FROM %s.multiple_clustering WHERE a = 1 AND (b) in ((2),(3)) AND c > 4");
- }
-
@Test(expected=SyntaxException.class)
public void testEmptyIdentifierTuple() throws Throwable
{
@@ -297,15 +291,25 @@ public class MultiColumnRelationTest
}
@Test
- public void testMixSingleAndTupleInequalities() throws Throwable
+ public void testInvalidMultiAndSingleColumnRelationMix() throws Throwable
{
for (String tableSuffix : new String[]{"", "_compact"})
{
String[] queries = new String[]{
- "SELECT * FROM %s.multiple_clustering" + tableSuffix + " WHERE a=0 AND (b, c, d) > (0, 1, 0) AND b < 1",
- "SELECT * FROM %s.multiple_clustering" + tableSuffix + " WHERE a=0 AND (b, c, d) > (0, 1, 0) AND c < 1",
- "SELECT * FROM %s.multiple_clustering" + tableSuffix + " WHERE a=0 AND b > 1 AND (b, c, d) < (1, 1, 0)",
- "SELECT * FROM %s.multiple_clustering" + tableSuffix + " WHERE a=0 AND c > 1 AND (b, c, d) < (1, 1, 0)",
+ "SELECT * FROM %s.multiple_clustering" + tableSuffix + " WHERE a = 0 AND (b, c, d) > (0, 1, 0) AND b < 1",
+ "SELECT * FROM %s.multiple_clustering" + tableSuffix + " WHERE a = 0 AND (b, c, d) > (0, 1, 0) AND c < 1",
+ "SELECT * FROM %s.multiple_clustering" + tableSuffix + " WHERE a = 0 AND b > 1 AND (b, c, d) < (1, 1, 0)",
+ "SELECT * FROM %s.multiple_clustering" + tableSuffix + " WHERE a = 0 AND c > 1 AND (b, c, d) < (1, 1, 0)",
+ "SELECT * FROM %s.multiple_clustering" + tableSuffix + " WHERE (a, b, c, d) IN ((0, 1, 2, 3))",
+ "SELECT * FROM %s.multiple_clustering" + tableSuffix + " WHERE (c, d) IN ((0, 1))",
+ "SELECT * FROM %s.multiple_clustering" + tableSuffix + " WHERE a = 0 AND b > 0 AND (c, d) IN ((0, 0))",
+ "SELECT * FROM %s.multiple_clustering" + tableSuffix + " WHERE a = 0 AND b > 0 AND (c, d) > (0, 0)",
+ "SELECT * FROM %s.multiple_clustering" + tableSuffix + " WHERE a = 0 AND (c, d) > (0, 0) AND b > 0 ",
+ "SELECT * FROM %s.multiple_clustering" + tableSuffix + " WHERE a = 0 AND (b, c) > (0, 0) AND (b) < (0) AND (c) < (0)",
+ "SELECT * FROM %s.multiple_clustering" + tableSuffix + " WHERE a = 0 AND (c) < (0) AND (b, c) > (0, 0) AND (b) < (0)",
+ "SELECT * FROM %s.multiple_clustering" + tableSuffix + " WHERE a = 0 AND (b) < (0) AND (c) < (0) AND (b, c) > (0, 0)",
+ "SELECT * FROM %s.multiple_clustering" + tableSuffix + " WHERE a = 0 AND (b, c) > (0, 0) AND (c) < (0)",
+ "SELECT * FROM %s.multiple_clustering" + tableSuffix + " WHERE a = 0 AND (b, c) in ((0, 0), (0, 0)) AND d > 0"
};
for (String query : queries)
@@ -323,6 +327,164 @@ public class MultiColumnRelationTest
}
@Test
+ public void testMultiAndSingleColumnRelationMix() throws Throwable
+ {
+ for (String tableSuffix : new String[]{"", "_compact"})
+ {
+ execute("INSERT INTO %s.multiple_clustering" + tableSuffix + " (a, b, c, d) VALUES (0, 0, 0, 0)");
+ execute("INSERT INTO %s.multiple_clustering" + tableSuffix + " (a, b, c, d) VALUES (0, 0, 1, 0)");
+ execute("INSERT INTO %s.multiple_clustering" + tableSuffix + " (a, b, c, d) VALUES (0, 0, 1, 1)");
+
+ execute("INSERT INTO %s.multiple_clustering" + tableSuffix + " (a, b, c, d) VALUES (0, 1, 0, 0)");
+ execute("INSERT INTO %s.multiple_clustering" + tableSuffix + " (a, b, c, d) VALUES (0, 1, 1, 0)");
+ execute("INSERT INTO %s.multiple_clustering" + tableSuffix + " (a, b, c, d) VALUES (0, 1, 1, 1)");
+
+ UntypedResultSet results = execute("SELECT * FROM %s.multiple_clustering" + tableSuffix
+ + " WHERE a = 0 and b = 1 and (c, d) = (0, 0)");
+ assertEquals(1, results.size());
+ checkRow(0, results, 0, 1, 0, 0);
+
+ results = execute("SELECT * FROM %s.multiple_clustering" + tableSuffix
+ + " WHERE a = 0 and b = 1 and (c) IN ((0))");
+ assertEquals(1, results.size());
+ checkRow(0, results, 0, 1, 0, 0);
+
+ results = execute("SELECT * FROM %s.multiple_clustering" + tableSuffix
+ + " WHERE a = 0 and b = 1 and (c) IN ((0), (1))");
+ assertEquals(3, results.size());
+ checkRow(0, results, 0, 1, 0, 0);
+ checkRow(1, results, 0, 1, 1, 0);
+ checkRow(2, results, 0, 1, 1, 1);
+
+ results = execute("SELECT * FROM %s.multiple_clustering" + tableSuffix
+ + " WHERE a = 0 and b = 1 and (c, d) IN ((0, 0))");
+ assertEquals(1, results.size());
+ checkRow(0, results, 0, 1, 0, 0);
+
+ results = execute("SELECT * FROM %s.multiple_clustering" + tableSuffix
+ + " WHERE a = 0 and b = 1 and (c, d) IN ((0, 0), (1, 1))");
+ assertEquals(2, results.size());
+ checkRow(0, results, 0, 1, 0, 0);
+ checkRow(1, results, 0, 1, 1, 1);
+
+ results = execute("SELECT * FROM %s.multiple_clustering" + tableSuffix
+ + " WHERE a = 0 and b = 1 and (c, d) > (0, 0)");
+ assertEquals(2, results.size());
+ checkRow(0, results, 0, 1, 1, 0);
+ checkRow(1, results, 0, 1, 1, 1);
+
+ results = execute("SELECT * FROM %s.multiple_clustering" + tableSuffix
+ + " WHERE a = 0 and b = 1 and (c, d) > (0, 0) and (c) <= (1)");
+ assertEquals(2, results.size());
+ checkRow(0, results, 0, 1, 1, 0);
+ checkRow(1, results, 0, 1, 1, 1);
+
+ results = execute("SELECT * FROM %s.multiple_clustering" + tableSuffix
+ + " WHERE a = 0 and b = 1 and (c, d) >= (0, 0) and (c, d) < (1, 1)");
+ assertEquals(2, results.size());
+ checkRow(0, results, 0, 1, 0, 0);
+ checkRow(1, results, 0, 1, 1, 0);
+
+ results = execute("SELECT * FROM %s.multiple_clustering" + tableSuffix
+ + " WHERE a = 0 and (b, c) = (0, 1) and d = 0");
+ assertEquals(1, results.size());
+ checkRow(0, results, 0, 0, 1, 0);
+
+ results = execute("SELECT * FROM %s.multiple_clustering" + tableSuffix
+ + " WHERE a = 0 and b = 0 and (c) = (1) and d = 0");
+ assertEquals(1, results.size());
+ checkRow(0, results, 0, 0, 1, 0);
+
+ results = execute("SELECT * FROM %s.multiple_clustering" + tableSuffix
+ + " WHERE a = 0 and (b, c) = (0, 1) and d IN (0, 2)");
+ assertEquals(1, results.size());
+ checkRow(0, results, 0, 0, 1, 0);
+
+ results = execute("SELECT * FROM %s.multiple_clustering" + tableSuffix
+ + " WHERE a = 0 and b = 0 and (c) = (1) and d IN (0, 2)");
+ assertEquals(1, results.size());
+ checkRow(0, results, 0, 0, 1, 0);
+
+ results = execute("SELECT * FROM %s.multiple_clustering" + tableSuffix
+ + " WHERE a = 0 and (b, c) = (0, 1) and d >= 0");
+ assertEquals(2, results.size());
+ checkRow(0, results, 0, 0, 1, 0);
+ checkRow(1, results, 0, 0, 1, 1);
+
+ results = execute("SELECT * FROM %s.multiple_clustering" + tableSuffix
+ + " WHERE a = 0 and d < 1 and (b, c) = (0, 1) and d >= 0");
+ assertEquals(1, results.size());
+ checkRow(0, results, 0, 0, 1, 0);
+ }
+ }
+
+ @Test
+ public void testMultipleMultiColumnRelation() throws Throwable
+ {
+ for (String tableSuffix : new String[]{"", "_compact"})
+ {
+ execute("INSERT INTO %s.multiple_clustering" + tableSuffix + " (a, b, c, d) VALUES (0, 0, 0, 0)");
+ execute("INSERT INTO %s.multiple_clustering" + tableSuffix + " (a, b, c, d) VALUES (0, 0, 1, 0)");
+ execute("INSERT INTO %s.multiple_clustering" + tableSuffix + " (a, b, c, d) VALUES (0, 0, 1, 1)");
+
+ execute("INSERT INTO %s.multiple_clustering" + tableSuffix + " (a, b, c, d) VALUES (0, 1, 0, 0)");
+ execute("INSERT INTO %s.multiple_clustering" + tableSuffix + " (a, b, c, d) VALUES (0, 1, 1, 0)");
+ execute("INSERT INTO %s.multiple_clustering" + tableSuffix + " (a, b, c, d) VALUES (0, 1, 1, 1)");
+
+ UntypedResultSet results = execute("SELECT * FROM %s.multiple_clustering" + tableSuffix
+ + " WHERE a = 0 and (b) = (1) and (c, d) = (0, 0)");
+ assertEquals(1, results.size());
+ checkRow(0, results, 0, 1, 0, 0);
+
+ results = execute("SELECT * FROM %s.multiple_clustering" + tableSuffix
+ + " WHERE a = 0 and (b) = (1) and (c) = (0) and (d) = (0)");
+ assertEquals(1, results.size());
+ checkRow(0, results, 0, 1, 0, 0);
+
+ results = execute("SELECT * FROM %s.multiple_clustering" + tableSuffix
+ + " WHERE a = 0 and (b) = (1) and (c) IN ((0))");
+ assertEquals(1, results.size());
+ checkRow(0, results, 0, 1, 0, 0);
+
+ results = execute("SELECT * FROM %s.multiple_clustering" + tableSuffix
+ + " WHERE a = 0 and (b) = (1) and (c) IN ((0), (1))");
+ assertEquals(3, results.size());
+ checkRow(0, results, 0, 1, 0, 0);
+ checkRow(1, results, 0, 1, 1, 0);
+ checkRow(2, results, 0, 1, 1, 1);
+
+ results = execute("SELECT * FROM %s.multiple_clustering" + tableSuffix
+ + " WHERE a = 0 and (b) = (1) and (c, d) IN ((0, 0))");
+ assertEquals(1, results.size());
+ checkRow(0, results, 0, 1, 0, 0);
+
+ results = execute("SELECT * FROM %s.multiple_clustering" + tableSuffix
+ + " WHERE a = 0 and (b) = (1) and (c, d) IN ((0, 0), (1, 1))");
+ assertEquals(2, results.size());
+ checkRow(0, results, 0, 1, 0, 0);
+ checkRow(1, results, 0, 1, 1, 1);
+
+ results = execute("SELECT * FROM %s.multiple_clustering" + tableSuffix
+ + " WHERE a = 0 and (b) = (1) and (c, d) > (0, 0)");
+ assertEquals(2, results.size());
+ checkRow(0, results, 0, 1, 1, 0);
+ checkRow(1, results, 0, 1, 1, 1);
+
+ results = execute("SELECT * FROM %s.multiple_clustering" + tableSuffix
+ + " WHERE a = 0 and (b) = (1) and (c, d) > (0, 0) and (c) <= (1)");
+ assertEquals(2, results.size());
+ checkRow(0, results, 0, 1, 1, 0);
+ checkRow(1, results, 0, 1, 1, 1);
+
+ results = execute("SELECT * FROM %s.multiple_clustering" + tableSuffix
+ + " WHERE a = 0 and (b) = (1) and (c, d) >= (0, 0) and (c, d) < (1, 1)");
+ assertEquals(2, results.size());
+ checkRow(0, results, 0, 1, 0, 0);
+ checkRow(1, results, 0, 1, 1, 0);
+ }
+ }
+
+ @Test
public void testSingleClusteringColumnInequality() throws Throwable
{
for (String tableSuffix : new String[]{"", "_compact"})