Return-Path: X-Original-To: apmail-cassandra-commits-archive@www.apache.org Delivered-To: apmail-cassandra-commits-archive@www.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id 44AD711656 for ; Thu, 22 May 2014 19:20:22 +0000 (UTC) Received: (qmail 83196 invoked by uid 500); 22 May 2014 19:20:22 -0000 Delivered-To: apmail-cassandra-commits-archive@cassandra.apache.org Received: (qmail 83030 invoked by uid 500); 22 May 2014 19:20:22 -0000 Mailing-List: contact commits-help@cassandra.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@cassandra.apache.org Delivered-To: mailing list commits@cassandra.apache.org Received: (qmail 82588 invoked by uid 99); 22 May 2014 19:20:21 -0000 Received: from tyr.zones.apache.org (HELO tyr.zones.apache.org) (140.211.11.114) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 22 May 2014 19:20:21 +0000 Received: by tyr.zones.apache.org (Postfix, from userid 65534) id 6F20393EC18; Thu, 22 May 2014 19:20:21 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: tylerhobbs@apache.org To: commits@cassandra.apache.org Date: Thu, 22 May 2014 19:20:24 -0000 Message-Id: <10c2cb5ce7754957a2e81dffbd798c88@git.apache.org> In-Reply-To: References: X-Mailer: ASF-Git Admin Mailer Subject: [4/6] Merge branch 'cassandra-2.0' into cassandra-2.1 http://git-wip-us.apache.org/repos/asf/cassandra/blob/bf521900/src/java/org/apache/cassandra/cql3/statements/SingleColumnRestriction.java ---------------------------------------------------------------------- diff --cc src/java/org/apache/cassandra/cql3/statements/SingleColumnRestriction.java index 0000000,2e63272..bc77357 mode 000000,100644..100644 --- a/src/java/org/apache/cassandra/cql3/statements/SingleColumnRestriction.java +++ b/src/java/org/apache/cassandra/cql3/statements/SingleColumnRestriction.java @@@ -1,0 -1,300 +1,413 @@@ + /* + * 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 org.apache.cassandra.cql3.*; ++import org.apache.cassandra.db.IndexExpression; + import org.apache.cassandra.exceptions.InvalidRequestException; -import org.apache.cassandra.thrift.IndexOperator; + + import java.nio.ByteBuffer; + import java.util.ArrayList; + import java.util.Collections; + import java.util.List; + + public abstract class SingleColumnRestriction implements Restriction + { + public boolean isMultiColumn() + { + return false; + } + + public static class EQ extends SingleColumnRestriction implements Restriction.EQ + { + protected final Term value; + private final boolean onToken; + + public EQ(Term value, boolean onToken) + { + this.value = value; + this.onToken = onToken; + } + - public List values(List variables) throws InvalidRequestException ++ public List values(QueryOptions options) throws InvalidRequestException + { - return Collections.singletonList(value.bindAndGet(variables)); ++ return Collections.singletonList(value.bindAndGet(options)); + } + + public boolean isSlice() + { + return false; + } + + public boolean isEQ() + { + return true; + } + + public boolean isIN() + { + return false; + } + ++ public boolean isContains() ++ { ++ return false; ++ } ++ + public boolean isOnToken() + { + return onToken; + } + + @Override + public String toString() + { + return String.format("EQ(%s)%s", value, onToken ? "*" : ""); + } + } + + public static class InWithValues extends SingleColumnRestriction implements Restriction.IN + { + protected final List values; + + public InWithValues(List values) + { + this.values = values; + } + - public List values(List variables) throws InvalidRequestException ++ public List values(QueryOptions options) throws InvalidRequestException + { + List buffers = new ArrayList<>(values.size()); + for (Term value : values) - buffers.add(value.bindAndGet(variables)); ++ buffers.add(value.bindAndGet(options)); + return buffers; + } + + public boolean canHaveOnlyOneValue() + { + return values.size() == 1; + } + + public boolean isSlice() + { + return false; + } + + public boolean isEQ() + { + return false; + } + + public boolean isIN() + { + return true; + } + ++ public boolean isContains() ++ { ++ return false; ++ } ++ + public boolean isOnToken() + { + return false; + } + + @Override + public String toString() + { + return String.format("IN(%s)", values); + } + } + + public static class InWithMarker extends SingleColumnRestriction implements Restriction.IN + { + protected final AbstractMarker marker; + + public InWithMarker(AbstractMarker marker) + { + this.marker = marker; + } + - public List values(List variables) throws InvalidRequestException ++ public List values(QueryOptions options) throws InvalidRequestException + { - Term.MultiItemTerminal lval = (Term.MultiItemTerminal)marker.bind(variables); ++ Term.MultiItemTerminal lval = (Term.MultiItemTerminal)marker.bind(options); + if (lval == null) + throw new InvalidRequestException("Invalid null value for IN restriction"); + return lval.getElements(); + } + + public boolean canHaveOnlyOneValue() + { + return false; + } + + public boolean isSlice() + { + return false; + } + + public boolean isEQ() + { + return false; + } + + public boolean isIN() + { + return true; + } + ++ public boolean isContains() ++ { ++ return false; ++ } ++ + public boolean isOnToken() + { + return false; + } + + @Override + public String toString() + { + return "IN ?"; + } + } + + public static class Slice extends SingleColumnRestriction implements Restriction.Slice + { + protected final Term[] bounds; + protected final boolean[] boundInclusive; + protected final boolean onToken; + + public Slice(boolean onToken) + { + this.bounds = new Term[2]; + this.boundInclusive = new boolean[2]; + this.onToken = onToken; + } + + public boolean isSlice() + { + return true; + } + + public boolean isEQ() + { + return false; + } + + public boolean isIN() + { + return false; + } + - public List values(List variables) throws InvalidRequestException ++ public boolean isContains() ++ { ++ return false; ++ } ++ ++ public List values(QueryOptions options) throws InvalidRequestException + { + throw new UnsupportedOperationException(); + } + + public boolean isOnToken() + { + return onToken; + } + + /** Returns true if the start or end bound (depending on the argument) is set, false otherwise */ + public boolean hasBound(Bound b) + { + return bounds[b.idx] != null; + } + - public ByteBuffer bound(Bound b, List variables) throws InvalidRequestException ++ public ByteBuffer bound(Bound b, QueryOptions options) throws InvalidRequestException + { - return bounds[b.idx].bindAndGet(variables); ++ return bounds[b.idx].bindAndGet(options); + } + + /** Returns true if the start or end bound (depending on the argument) is inclusive, false otherwise */ + public boolean isInclusive(Bound b) + { + return bounds[b.idx] == null || boundInclusive[b.idx]; + } + + public Relation.Type getRelation(Bound eocBound, Bound inclusiveBound) + { + switch (eocBound) + { + case START: + return boundInclusive[inclusiveBound.idx] ? Relation.Type.GTE : Relation.Type.GT; + case END: + return boundInclusive[inclusiveBound.idx] ? Relation.Type.LTE : Relation.Type.LT; + } + throw new AssertionError(); + } + - public IndexOperator getIndexOperator(Bound b) ++ public IndexExpression.Operator getIndexOperator(Bound b) + { + switch (b) + { + case START: - return boundInclusive[b.idx] ? IndexOperator.GTE : IndexOperator.GT; ++ return boundInclusive[b.idx] ? IndexExpression.Operator.GTE : IndexExpression.Operator.GT; + case END: - return boundInclusive[b.idx] ? IndexOperator.LTE : IndexOperator.LT; ++ return boundInclusive[b.idx] ? IndexExpression.Operator.LTE : IndexExpression.Operator.LT; + } + throw new AssertionError(); + } + - public void setBound(Relation.Type type, Term t) throws InvalidRequestException ++ public void setBound(ColumnIdentifier name, Relation.Type type, Term t) throws InvalidRequestException + { + Bound b; + boolean inclusive; + switch (type) + { + case GT: + b = Bound.START; + inclusive = false; + break; + case GTE: + b = Bound.START; + inclusive = true; + break; + case LT: + b = Bound.END; + inclusive = false; + break; + case LTE: + b = Bound.END; + inclusive = true; + break; + default: + throw new AssertionError(); + } + + if (bounds[b.idx] != null) + throw new InvalidRequestException(String.format( - "More than one restriction was found for the %s bound", b.name().toLowerCase())); ++ "More than one restriction was found for the %s bound on %s", b.name().toLowerCase(), name)); + + bounds[b.idx] = t; + boundInclusive[b.idx] = inclusive; + } + + @Override + public String toString() + { + return String.format("SLICE(%s %s, %s %s)%s", boundInclusive[0] ? ">=" : ">", + bounds[0], + boundInclusive[1] ? "<=" : "<", + bounds[1], + onToken ? "*" : ""); + } + } ++ ++ // This holds both CONTAINS and CONTAINS_KEY restriction because we might want to have both of them. ++ public static class Contains extends SingleColumnRestriction ++ { ++ private List values; // for CONTAINS ++ private List keys; // for CONTAINS_KEY ++ ++ public boolean hasContains() ++ { ++ return values != null; ++ } ++ ++ public boolean hasContainsKey() ++ { ++ return keys != null; ++ } ++ ++ public void add(Term t, boolean isKey) ++ { ++ if (isKey) ++ addKey(t); ++ else ++ addValue(t); ++ } ++ ++ public void addValue(Term t) ++ { ++ if (values == null) ++ values = new ArrayList<>(); ++ values.add(t); ++ } ++ ++ public void addKey(Term t) ++ { ++ if (keys == null) ++ keys = new ArrayList<>(); ++ keys.add(t); ++ } ++ ++ public List values(QueryOptions options) throws InvalidRequestException ++ { ++ if (values == null) ++ return Collections.emptyList(); ++ ++ List buffers = new ArrayList(values.size()); ++ for (Term value : values) ++ buffers.add(value.bindAndGet(options)); ++ return buffers; ++ } ++ ++ public List keys(QueryOptions options) throws InvalidRequestException ++ { ++ if (keys == null) ++ return Collections.emptyList(); ++ ++ List buffers = new ArrayList(keys.size()); ++ for (Term value : keys) ++ buffers.add(value.bindAndGet(options)); ++ return buffers; ++ } ++ ++ public boolean isSlice() ++ { ++ return false; ++ } ++ ++ public boolean isEQ() ++ { ++ return false; ++ } ++ ++ public boolean isIN() ++ { ++ return false; ++ } ++ ++ public boolean isContains() ++ { ++ return true; ++ } ++ ++ public boolean isOnToken() ++ { ++ return false; ++ } ++ ++ ++ @Override ++ public String toString() ++ { ++ return String.format("CONTAINS(values=%s, keys=%s)", values, keys); ++ } ++ } + } http://git-wip-us.apache.org/repos/asf/cassandra/blob/bf521900/src/java/org/apache/cassandra/db/composites/CBuilder.java ---------------------------------------------------------------------- diff --cc src/java/org/apache/cassandra/db/composites/CBuilder.java index 490fb67,0000000..39035cb mode 100644,000000..100644 --- a/src/java/org/apache/cassandra/db/composites/CBuilder.java +++ b/src/java/org/apache/cassandra/db/composites/CBuilder.java @@@ -1,34 -1,0 +1,36 @@@ +/* + * 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.db.composites; + +import java.nio.ByteBuffer; ++import java.util.List; + +/** + * A builder of Composite. + */ +public interface CBuilder +{ + public int remainingCount(); + + public CBuilder add(ByteBuffer value); + public CBuilder add(Object value); + + public Composite build(); + public Composite buildWith(ByteBuffer value); ++ public Composite buildWith(List values); +} http://git-wip-us.apache.org/repos/asf/cassandra/blob/bf521900/src/java/org/apache/cassandra/db/composites/Composites.java ---------------------------------------------------------------------- diff --cc src/java/org/apache/cassandra/db/composites/Composites.java index 58938c6,0000000..154e9f7 mode 100644,000000..100644 --- a/src/java/org/apache/cassandra/db/composites/Composites.java +++ b/src/java/org/apache/cassandra/db/composites/Composites.java @@@ -1,125 -1,0 +1,127 @@@ +/* + * 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.db.composites; + +import java.nio.ByteBuffer; ++import java.util.List; + +import org.apache.cassandra.config.CFMetaData; +import org.apache.cassandra.db.filter.ColumnSlice; +import org.apache.cassandra.utils.memory.AbstractAllocator; +import org.apache.cassandra.utils.ByteBufferUtil; + +public abstract class Composites +{ + private Composites() {} + + public static final Composite EMPTY = new EmptyComposite(); + + static final CBuilder EMPTY_BUILDER = new CBuilder() + { + public int remainingCount() { return 0; } + + public CBuilder add(ByteBuffer value) { throw new IllegalStateException(); } + public CBuilder add(Object value) { throw new IllegalStateException(); } + + public Composite build() { return EMPTY; } + public Composite buildWith(ByteBuffer value) { throw new IllegalStateException(); } ++ public Composite buildWith(List values) { throw new IllegalStateException(); } + }; + + private static class EmptyComposite implements Composite + { + public boolean isEmpty() + { + return true; + } + + public int size() + { + return 0; + } + + public ByteBuffer get(int i) + { + throw new IndexOutOfBoundsException(); + } + + public EOC eoc() + { + return EOC.NONE; + } + + public Composite start() + { + // Note that SimpleCType/AbstractSimpleCellNameType compare method + // indirectly rely on the fact that EMPTY == EMPTY.start() == EMPTY.end() + // (or more precisely on the fact that the EOC is NONE for all of those). + return this; + } + + public Composite end() + { + // Note that SimpleCType/AbstractSimpleCellNameType compare method + // indirectly rely on the fact that EMPTY == EMPTY.start() == EMPTY.end() + // (or more precisely on the fact that the EOC is NONE for all of those). + return this; + } + + public Composite withEOC(EOC newEoc) + { + // Note that SimpleCType/AbstractSimpleCellNameType compare method + // indirectly rely on the fact that EMPTY == EMPTY.start() == EMPTY.end() + // (or more precisely on the fact that the EOC is NONE for all of those). + return this; + } + + public ColumnSlice slice() + { + return ColumnSlice.ALL_COLUMNS; + } + + public ByteBuffer toByteBuffer() + { + return ByteBufferUtil.EMPTY_BYTE_BUFFER; + } + + public boolean isStatic() + { + return false; + } + + public int dataSize() + { + return 0; + } + + public long unsharedHeapSize() + { + return 0; + } + + public boolean isPrefixOf(CType type, Composite c) + { + return true; + } + + public Composite copy(CFMetaData cfm, AbstractAllocator allocator) + { + return this; + } + } +} http://git-wip-us.apache.org/repos/asf/cassandra/blob/bf521900/src/java/org/apache/cassandra/db/composites/CompoundCType.java ---------------------------------------------------------------------- diff --cc src/java/org/apache/cassandra/db/composites/CompoundCType.java index 4322055,0000000..0458748 mode 100644,000000..100644 --- a/src/java/org/apache/cassandra/db/composites/CompoundCType.java +++ b/src/java/org/apache/cassandra/db/composites/CompoundCType.java @@@ -1,167 -1,0 +1,180 @@@ +/* + * 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.db.composites; + +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +import org.apache.cassandra.db.marshal.AbstractType; +import org.apache.cassandra.db.marshal.CompositeType; + +/** + * A truly-composite CType. + */ +public class CompoundCType extends AbstractCType +{ + final List> types; + + // It's up to the caller to pass a list that is effectively immutable + public CompoundCType(List> types) + { + super(isByteOrderComparable(types)); + this.types = types; + } + + public boolean isCompound() + { + return true; + } + + public int size() + { + return types.size(); + } + + public AbstractType subtype(int i) + { + return types.get(i); + } + + public Composite fromByteBuffer(ByteBuffer bytes) + { + if (!bytes.hasRemaining()) + return Composites.EMPTY; + + ByteBuffer[] elements = new ByteBuffer[size()]; + int idx = bytes.position(), i = 0; + byte eoc = 0; + + boolean isStatic = false; + if (CompositeType.isStaticName(bytes)) + { + isStatic = true; + idx += 2; + } + + while (idx < bytes.limit()) + { + checkRemaining(bytes, idx, 2); + int length = bytes.getShort(idx) & 0xFFFF; + idx += 2; + + checkRemaining(bytes, idx, length + 1); + elements[i++] = sliceBytes(bytes, idx, length); + idx += length; + eoc = bytes.get(idx++); + } + return new CompoundComposite(elements, i, isStatic).withEOC(Composite.EOC.from(eoc)); + } + + public CBuilder builder() + { + return new CompoundCBuilder(this); + } + + public CompoundCType setSubtype(int position, AbstractType newType) + { + List> newTypes = new ArrayList>(types); + newTypes.set(position, newType); + return new CompoundCType(newTypes); + } + + public AbstractType asAbstractType() + { + return CompositeType.getInstance(types); + } + + public static class CompoundCBuilder implements CBuilder + { + private final CType type; + private final ByteBuffer[] values; + private int size; + private boolean built; + + public CompoundCBuilder(CType type) + { + this.type = type; + this.values = new ByteBuffer[type.size()]; + } + + public int remainingCount() + { + return values.length - size; + } + + public CBuilder add(ByteBuffer value) + { + if (isDone()) + throw new IllegalStateException(); + values[size++] = value; + return this; + } + + public CBuilder add(Object value) + { + return add(((AbstractType)type.subtype(size)).decompose(value)); + } + + private boolean isDone() + { + return remainingCount() == 0 || built; + } + + public Composite build() + { + if (size == 0) + return Composites.EMPTY; + + // We don't allow to add more element to a builder that has been built so + // that we don't have to copy values. + built = true; + + // If the builder is full and we're building a dense cell name, then we can + // directly allocate the CellName object as it's complete. + if (size == values.length && type instanceof CellNameType && ((CellNameType)type).isDense()) + return new CompoundDenseCellName(values); + return new CompoundComposite(values, size, false); + } + + public Composite buildWith(ByteBuffer value) + { + ByteBuffer[] newValues = Arrays.copyOf(values, values.length); + newValues[size] = value; + // Same as above + if (size+1 == newValues.length && type instanceof CellNameType && ((CellNameType)type).isDense()) + return new CompoundDenseCellName(newValues); + + return new CompoundComposite(newValues, size+1, false); + } ++ ++ public Composite buildWith(List newValues) ++ { ++ ByteBuffer[] buffers = Arrays.copyOf(values, values.length); ++ int newSize = size; ++ for (ByteBuffer value : newValues) ++ buffers[newSize++] = value; ++ ++ if (newSize == buffers.length && type instanceof CellNameType && ((CellNameType)type).isDense()) ++ return new CompoundDenseCellName(buffers); ++ ++ return new CompoundComposite(buffers, newSize, false); ++ } + } +} http://git-wip-us.apache.org/repos/asf/cassandra/blob/bf521900/src/java/org/apache/cassandra/db/composites/SimpleCType.java ---------------------------------------------------------------------- diff --cc src/java/org/apache/cassandra/db/composites/SimpleCType.java index 35e14f9,0000000..229d538 mode 100644,000000..100644 --- a/src/java/org/apache/cassandra/db/composites/SimpleCType.java +++ b/src/java/org/apache/cassandra/db/composites/SimpleCType.java @@@ -1,160 -1,0 +1,170 @@@ +/* + * 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.db.composites; + +import java.nio.ByteBuffer; ++import java.util.List; + +import org.apache.cassandra.db.marshal.AbstractType; +import org.apache.cassandra.utils.ByteBufferUtil; + +/** + * A not truly-composite CType. + */ +public class SimpleCType extends AbstractCType +{ + protected final AbstractType type; + + public SimpleCType(AbstractType type) + { + super(type.isByteOrderComparable()); + this.type = type; + } + + public boolean isCompound() + { + return false; + } + + public int size() + { + return 1; + } + + public int compare(Composite c1, Composite c2) + { + // This method assumes that simple composites never have an EOC != NONE. This assumption + // stands in particular on the fact that a Composites.EMPTY never has a non-NONE EOC. If + // this ever change, we'll need to update this. + + if (isByteOrderComparable) + { + // toByteBuffer is always cheap for simple types, and we keep virtual method calls to a minimum: + // hasRemaining will always be inlined, as will most of the call-stack for BBU.compareUnsigned + ByteBuffer b1 = c1.toByteBuffer(); + ByteBuffer b2 = c2.toByteBuffer(); + if (!b1.hasRemaining() || !b2.hasRemaining()) + return b1.hasRemaining() ? 1 : (b2.hasRemaining() ? -1 : 0); + return ByteBufferUtil.compareUnsigned(b1, b2); + } + + boolean c1isEmpty = c1.isEmpty(); + boolean c2isEmpty = c2.isEmpty(); + if (c1isEmpty || c2isEmpty) + return !c1isEmpty ? 1 : (!c2isEmpty ? -1 : 0); + + return type.compare(c1.get(0), c2.get(0)); + } + + public AbstractType subtype(int i) + { + if (i != 0) + throw new IndexOutOfBoundsException(); + return type; + } + + public Composite fromByteBuffer(ByteBuffer bytes) + { + return !bytes.hasRemaining() ? Composites.EMPTY : new SimpleComposite(bytes); + } + + public CBuilder builder() + { + return new SimpleCBuilder(this); + } + + public CType setSubtype(int position, AbstractType newType) + { + if (position != 0) + throw new IndexOutOfBoundsException(); + return new SimpleCType(newType); + } + + // Use sparingly, it defeats the purpose + public AbstractType asAbstractType() + { + return type; + } + + public static class SimpleCBuilder implements CBuilder + { + private final CType type; + private ByteBuffer value; + + public SimpleCBuilder(CType type) + { + this.type = type; + } + + public int remainingCount() + { + return value == null ? 1 : 0; + } + + public CBuilder add(ByteBuffer value) + { + if (this.value != null) + throw new IllegalStateException(); + this.value = value; + return this; + } + + public CBuilder add(Object value) + { + return add(((AbstractType)type.subtype(0)).decompose(value)); + } + + public Composite build() + { + if (value == null || !value.hasRemaining()) + return Composites.EMPTY; + + // If we're building a dense cell name, then we can directly allocate the + // CellName object as it's complete. + if (type instanceof CellNameType && ((CellNameType)type).isDense()) + return new SimpleDenseCellName(value); + + return new SimpleComposite(value); + } + + public Composite buildWith(ByteBuffer value) + { + if (this.value != null) + throw new IllegalStateException(); + + if (value == null || !value.hasRemaining()) + return Composites.EMPTY; + + // If we're building a dense cell name, then we can directly allocate the + // CellName object as it's complete. + if (type instanceof CellNameType && ((CellNameType)type).isDense()) + return new SimpleDenseCellName(value); + + return new SimpleComposite(value); + } ++ ++ public Composite buildWith(List values) ++ { ++ if (values.size() > 1) ++ throw new IllegalStateException(); ++ if (values.isEmpty()) ++ return Composites.EMPTY; ++ return buildWith(values.get(0)); ++ } + } +} http://git-wip-us.apache.org/repos/asf/cassandra/blob/bf521900/test/unit/org/apache/cassandra/cql3/MultiColumnRelationTest.java ---------------------------------------------------------------------- diff --cc test/unit/org/apache/cassandra/cql3/MultiColumnRelationTest.java index 0000000,b728cba..35646be mode 000000,100644..100644 --- a/test/unit/org/apache/cassandra/cql3/MultiColumnRelationTest.java +++ b/test/unit/org/apache/cassandra/cql3/MultiColumnRelationTest.java @@@ -1,0 -1,1112 +1,1114 @@@ + /* + * 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.cql3.statements.ParsedStatement; + import org.apache.cassandra.db.ConsistencyLevel; + import org.apache.cassandra.db.marshal.*; + import org.apache.cassandra.exceptions.InvalidRequestException; + import org.apache.cassandra.exceptions.RequestExecutionException; + import org.apache.cassandra.exceptions.RequestValidationException; + import org.apache.cassandra.exceptions.SyntaxException; + import org.apache.cassandra.gms.Gossiper; ++import org.apache.cassandra.serializers.CollectionSerializer; + import org.apache.cassandra.service.ClientState; + import org.apache.cassandra.service.QueryState; + import org.apache.cassandra.transport.messages.ResultMessage; + import org.apache.cassandra.utils.ByteBufferUtil; + import org.apache.cassandra.utils.MD5Digest; + import org.junit.AfterClass; + import org.junit.BeforeClass; + import org.junit.Test; + import org.slf4j.Logger; + import org.slf4j.LoggerFactory; + + import java.nio.ByteBuffer; + import java.util.*; + ++import static org.apache.cassandra.cql3.QueryProcessor.executeOnceInternal; + import static org.apache.cassandra.cql3.QueryProcessor.process; -import static org.apache.cassandra.cql3.QueryProcessor.processInternal; + import static org.junit.Assert.assertTrue; + import static org.junit.Assert.assertEquals; + import static com.google.common.collect.Lists.newArrayList; + import static org.junit.Assert.fail; + + public class MultiColumnRelationTest + { + private static final Logger logger = LoggerFactory.getLogger(MultiColumnRelationTest.class); + static ClientState clientState; + static String keyspace = "multi_column_relation_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 int)"); + executeSchemaChange("CREATE TABLE IF NOT EXISTS %s.compound_partition (a int, b int, c int, PRIMARY KEY ((a, b)))"); + executeSchemaChange("CREATE TABLE IF NOT EXISTS %s.single_clustering (a int, b int, c int, PRIMARY KEY (a, b))"); + executeSchemaChange("CREATE TABLE IF NOT EXISTS %s.multiple_clustering (a int, b int, c int, d int, PRIMARY KEY (a, b, c, d))"); + executeSchemaChange("CREATE TABLE IF NOT EXISTS %s.multiple_clustering_reversed (a int, b int, c int, d int, PRIMARY KEY (a, b, c, d)) WITH CLUSTERING ORDER BY (b DESC, c ASC, d DESC)"); + 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)); ++ return executeOnceInternal(String.format(query, keyspace)); + } catch (RuntimeException exc) + { + if (exc.getCause() != null) + throw exc.getCause(); + throw exc; + } + } + + private MD5Digest prepare(String query) throws RequestValidationException + { + ResultMessage.Prepared prepared = QueryProcessor.prepare(String.format(query, keyspace), clientState, false); + return prepared.statementId; + } + + private UntypedResultSet executePrepared(MD5Digest statementId, QueryOptions options) throws RequestValidationException, RequestExecutionException + { - CQLStatement statement = QueryProcessor.instance.getPrepared(statementId); - ResultMessage message = statement.executeInternal(QueryState.forInternalCalls(), options); ++ ParsedStatement.Prepared prepared = QueryProcessor.instance.getPrepared(statementId); ++ ResultMessage message = prepared.statement.executeInternal(QueryState.forInternalCalls(), options); + + if (message instanceof ResultMessage.Rows) - return new UntypedResultSet(((ResultMessage.Rows)message).result); ++ return UntypedResultSet.create(((ResultMessage.Rows)message).result); + else + return null; + } + + @Test(expected=SyntaxException.class) + public void testEmptyIdentifierTuple() throws Throwable + { + execute("SELECT * FROM %s.single_clustering WHERE () = (1, 2)"); + } + + @Test(expected=SyntaxException.class) + public void testEmptyValueTuple() throws Throwable + { + execute("SELECT * FROM %s.multiple_clustering WHERE (b, c) > ()"); + } + + @Test(expected=InvalidRequestException.class) + public void testDifferentTupleLengths() throws Throwable + { + execute("SELECT * FROM %s.multiple_clustering WHERE (b, c) > (1, 2, 3)"); + } + + @Test(expected=InvalidRequestException.class) + public void testNullInTuple() throws Throwable + { + execute("SELECT * FROM %s.multiple_clustering WHERE (b, c) > (1, null)"); + } + + @Test + public void testEmptyIN() throws Throwable + { + UntypedResultSet results = execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) IN ()"); + assertTrue(results.isEmpty()); + } + + @Test(expected=InvalidRequestException.class) + public void testNullInINValues() throws Throwable + { + UntypedResultSet results = execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) IN ((1, 2, null))"); + assertTrue(results.isEmpty()); + } + + @Test(expected=InvalidRequestException.class) + public void testPartitionKeyInequality() throws Throwable + { + execute("SELECT * FROM %s.single_partition WHERE (a) > (1)"); + } + + @Test(expected=InvalidRequestException.class) + public void testPartitionKeyEquality() throws Throwable + { + execute("SELECT * FROM %s.single_partition WHERE (a) = (0)"); + } + + @Test(expected=InvalidRequestException.class) + public void testRestrictNonPrimaryKey() throws Throwable + { + execute("SELECT * FROM %s.single_partition WHERE (b) = (0)"); + } + + @Test(expected=InvalidRequestException.class) + public void testMixEqualityAndInequality() throws Throwable + { + execute("SELECT * FROM %s.single_clustering WHERE a=0 AND (b) = (0) AND (b) > (0)"); + } + + @Test(expected=InvalidRequestException.class) + public void testMixMultipleInequalitiesOnSameBound() throws Throwable + { + execute("SELECT * FROM %s.single_clustering WHERE a=0 AND (b) > (0) AND (b) > (1)"); + } + + @Test(expected=InvalidRequestException.class) + public void testClusteringColumnsOutOfOrderInInequality() throws Throwable + { + execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (d, c, b) > (0, 0, 0)"); + } + + @Test(expected=InvalidRequestException.class) + public void testSkipClusteringColumnInEquality() throws Throwable + { + execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (c, d) = (0, 0)"); + } + + @Test(expected=InvalidRequestException.class) + public void testSkipClusteringColumnInInequality() throws Throwable + { + execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (c, d) > (0, 0)"); + } + + @Test + public void testSingleClusteringColumnEquality() throws Throwable + { + execute("INSERT INTO %s.single_clustering (a, b, c) VALUES (0, 0, 0)"); + execute("INSERT INTO %s.single_clustering (a, b, c) VALUES (0, 1, 0)"); + execute("INSERT INTO %s.single_clustering (a, b, c) VALUES (0, 2, 0)"); + UntypedResultSet results = execute("SELECT * FROM %s.single_clustering WHERE a=0 AND (b) = (1)"); + assertEquals(1, results.size()); + checkRow(0, results, 0, 1, 0); + + results = execute("SELECT * FROM %s.single_clustering WHERE a=0 AND (b) = (3)"); + assertEquals(0, results.size()); + } + + @Test + public void testMultipleClusteringColumnEquality() throws Throwable + { + execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 0, 0, 0)"); + execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 1, 0, 0)"); + execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 1, 1, 0)"); + execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 1, 1, 1)"); + execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 2, 0, 0)"); + UntypedResultSet results = execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b) = (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 WHERE a=0 AND (b, c) = (1, 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 WHERE a=0 AND (b, c, d) = (1, 1, 1)"); + assertEquals(1, results.size()); + checkRow(0, results, 0, 1, 1, 1); + execute("DELETE FROM %s.multiple_clustering WHERE a=0 AND b=2 and c=0 and d=0"); + } + + @Test(expected=InvalidRequestException.class) + public void testPartitionAndClusteringColumnEquality() throws Throwable + { + execute("SELECT * FROM %s.single_clustering WHERE (a, b) = (0, 0)"); + } + + @Test(expected=InvalidRequestException.class) + public void testClusteringColumnsOutOfOrderInEquality() throws Throwable + { + execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (d, c, b) = (3, 2, 1)"); + } + + @Test(expected=InvalidRequestException.class) + public void testBadType() throws Throwable + { + execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) = (1, 2, 'foobar')"); + } + + @Test(expected=SyntaxException.class) + public void testSingleColumnTupleRelation() throws Throwable + { + execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND b = (1, 2, 3)"); + } + + @Test + public void testMixSingleAndTupleInequalities() throws Throwable + { + String[] queries = new String[]{ + "SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) > (0, 1, 0) AND b < 1", + "SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) > (0, 1, 0) AND c < 1", + "SELECT * FROM %s.multiple_clustering WHERE a=0 AND b > 1 AND (b, c, d) < (1, 1, 0)", + "SELECT * FROM %s.multiple_clustering WHERE a=0 AND c > 1 AND (b, c, d) < (1, 1, 0)", + }; + + for (String query : queries) + { + try + { + execute(query); + fail(String.format("Expected query \"%s\" to throw an InvalidRequestException", query)); + } + catch (InvalidRequestException e) {} + } + } + + @Test + public void testSingleClusteringColumnInequality() throws Throwable + { + execute("INSERT INTO %s.single_clustering (a, b, c) VALUES (0, 0, 0)"); + execute("INSERT INTO %s.single_clustering (a, b, c) VALUES (0, 1, 0)"); + execute("INSERT INTO %s.single_clustering (a, b, c) VALUES (0, 2, 0)"); + + UntypedResultSet results = execute("SELECT * FROM %s.single_clustering WHERE a=0 AND (b) > (0)"); + assertEquals(2, results.size()); + checkRow(0, results, 0, 1, 0); + checkRow(1, results, 0, 2, 0); + + results = execute("SELECT * FROM %s.single_clustering WHERE a=0 AND (b) >= (1)"); + assertEquals(2, results.size()); + checkRow(0, results, 0, 1, 0); + checkRow(1, results, 0, 2, 0); + + results = execute("SELECT * FROM %s.single_clustering WHERE a=0 AND (b) < (2)"); + assertEquals(2, results.size()); + checkRow(0, results, 0, 0, 0); + checkRow(1, results, 0, 1, 0); + + results = execute("SELECT * FROM %s.single_clustering WHERE a=0 AND (b) <= (1)"); + assertEquals(2, results.size()); + checkRow(0, results, 0, 0, 0); + checkRow(1, results, 0, 1, 0); + + results = execute("SELECT * FROM %s.single_clustering WHERE a=0 AND (b) > (0) AND (b) < (2)"); + assertEquals(1, results.size()); + checkRow(0, results, 0, 1, 0); + } + + @Test + public void testMultipleClusteringColumnInequality() throws Throwable + { + execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 0, 0, 0)"); + execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 0, 1, 0)"); + execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 0, 1, 1)"); + + execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 1, 0, 0)"); + execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 1, 1, 0)"); + execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 1, 1, 1)"); + + UntypedResultSet results = execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b) > (0)"); + 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 WHERE a=0 AND (b) >= (0)"); + assertEquals(6, results.size()); + checkRow(0, results, 0, 0, 0, 0); + checkRow(1, results, 0, 0, 1, 0); + checkRow(2, results, 0, 0, 1, 1); + checkRow(3, results, 0, 1, 0, 0); + checkRow(4, results, 0, 1, 1, 0); + checkRow(5, results, 0, 1, 1, 1); + + results = execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c) > (1, 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 WHERE a=0 AND (b, c) >= (1, 0)"); + 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 WHERE a=0 AND (b, c, d) > (1, 1, 0)"); + assertEquals(1, results.size()); + checkRow(0, results, 0, 1, 1, 1); + + results = execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) >= (1, 1, 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 WHERE a=0 AND (b) < (1)"); + assertEquals(3, results.size()); + checkRow(0, results, 0, 0, 0, 0); + checkRow(1, results, 0, 0, 1, 0); + checkRow(2, results, 0, 0, 1, 1); + + results = execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b) <= (1)"); + assertEquals(6, results.size()); + checkRow(0, results, 0, 0, 0, 0); + checkRow(1, results, 0, 0, 1, 0); + checkRow(2, results, 0, 0, 1, 1); + checkRow(3, results, 0, 1, 0, 0); + checkRow(4, results, 0, 1, 1, 0); + checkRow(5, results, 0, 1, 1, 1); + + results = execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c) < (0, 1)"); + assertEquals(1, results.size()); + checkRow(0, results, 0, 0, 0, 0); + + results = execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c) <= (0, 1)"); + assertEquals(3, results.size()); + checkRow(0, results, 0, 0, 0, 0); + checkRow(1, results, 0, 0, 1, 0); + checkRow(2, results, 0, 0, 1, 1); + + results = execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) < (0, 1, 1)"); + assertEquals(2, results.size()); + checkRow(0, results, 0, 0, 0, 0); + checkRow(1, results, 0, 0, 1, 0); + + results = execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) <= (0, 1, 1)"); + checkRow(0, results, 0, 0, 0, 0); + checkRow(1, results, 0, 0, 1, 0); + checkRow(2, results, 0, 0, 1, 1); + + results = execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) > (0, 1, 0) AND (b) < (1)"); + assertEquals(1, results.size()); + checkRow(0, results, 0, 0, 1, 1); + + results = execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) > (0, 1, 1) AND (b, c) < (1, 1)"); + assertEquals(1, results.size()); + checkRow(0, results, 0, 1, 0, 0); + + results = execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) > (0, 1, 1) AND (b, c, d) < (1, 1, 0)"); + assertEquals(1, results.size()); + checkRow(0, results, 0, 1, 0, 0); + + // reversed + results = execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b) > (0) ORDER BY b DESC, c DESC, d DESC"); + assertEquals(3, results.size()); + checkRow(2, results, 0, 1, 0, 0); + checkRow(1, results, 0, 1, 1, 0); + checkRow(0, results, 0, 1, 1, 1); + + results = execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b) >= (0) ORDER BY b DESC, c DESC, d DESC"); + assertEquals(6, results.size()); + checkRow(5, results, 0, 0, 0, 0); + checkRow(4, results, 0, 0, 1, 0); + checkRow(3, results, 0, 0, 1, 1); + checkRow(2, results, 0, 1, 0, 0); + checkRow(1, results, 0, 1, 1, 0); + checkRow(0, results, 0, 1, 1, 1); + + results = execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c) > (1, 0) ORDER BY b DESC, c DESC, d DESC"); + assertEquals(2, results.size()); + checkRow(1, results, 0, 1, 1, 0); + checkRow(0, results, 0, 1, 1, 1); + + results = execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c) >= (1, 0) ORDER BY b DESC, c DESC, d DESC"); + assertEquals(3, results.size()); + checkRow(2, results, 0, 1, 0, 0); + checkRow(1, results, 0, 1, 1, 0); + checkRow(0, results, 0, 1, 1, 1); + + results = execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) > (1, 1, 0) ORDER BY b DESC, c DESC, d DESC"); + assertEquals(1, results.size()); + checkRow(0, results, 0, 1, 1, 1); + + results = execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) >= (1, 1, 0) ORDER BY b DESC, c DESC, d DESC"); + assertEquals(2, results.size()); + checkRow(1, results, 0, 1, 1, 0); + checkRow(0, results, 0, 1, 1, 1); + + results = execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b) < (1) ORDER BY b DESC, c DESC, d DESC"); + assertEquals(3, results.size()); + checkRow(2, results, 0, 0, 0, 0); + checkRow(1, results, 0, 0, 1, 0); + checkRow(0, results, 0, 0, 1, 1); + + results = execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b) <= (1) ORDER BY b DESC, c DESC, d DESC"); + assertEquals(6, results.size()); + checkRow(5, results, 0, 0, 0, 0); + checkRow(4, results, 0, 0, 1, 0); + checkRow(3, results, 0, 0, 1, 1); + checkRow(2, results, 0, 1, 0, 0); + checkRow(1, results, 0, 1, 1, 0); + checkRow(0, results, 0, 1, 1, 1); + + results = execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c) < (0, 1) ORDER BY b DESC, c DESC, d DESC"); + assertEquals(1, results.size()); + checkRow(0, results, 0, 0, 0, 0); + + results = execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c) <= (0, 1) ORDER BY b DESC, c DESC, d DESC"); + assertEquals(3, results.size()); + checkRow(2, results, 0, 0, 0, 0); + checkRow(1, results, 0, 0, 1, 0); + checkRow(0, results, 0, 0, 1, 1); + + results = execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) < (0, 1, 1) ORDER BY b DESC, c DESC, d DESC"); + assertEquals(2, results.size()); + checkRow(1, results, 0, 0, 0, 0); + checkRow(0, results, 0, 0, 1, 0); + + results = execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) <= (0, 1, 1) ORDER BY b DESC, c DESC, d DESC"); + checkRow(2, results, 0, 0, 0, 0); + checkRow(1, results, 0, 0, 1, 0); + checkRow(0, results, 0, 0, 1, 1); + + results = execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) > (0, 1, 0) AND (b) < (1) ORDER BY b DESC, c DESC, d DESC"); + assertEquals(1, results.size()); + checkRow(0, results, 0, 0, 1, 1); + + results = execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) > (0, 1, 1) AND (b, c) < (1, 1) ORDER BY b DESC, c DESC, d DESC"); + assertEquals(1, results.size()); + checkRow(0, results, 0, 1, 0, 0); + + results = execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) > (0, 1, 1) AND (b, c, d) < (1, 1, 0) ORDER BY b DESC, c DESC, d DESC"); + assertEquals(1, results.size()); + checkRow(0, results, 0, 1, 0, 0); + } + + @Test + public void testMultipleClusteringColumnInequalityReversedComponents() throws Throwable + { + // b and d are reversed in the clustering order + execute("INSERT INTO %s.multiple_clustering_reversed (a, b, c, d) VALUES (0, 1, 0, 0)"); + execute("INSERT INTO %s.multiple_clustering_reversed (a, b, c, d) VALUES (0, 1, 1, 1)"); + execute("INSERT INTO %s.multiple_clustering_reversed (a, b, c, d) VALUES (0, 1, 1, 0)"); + + execute("INSERT INTO %s.multiple_clustering_reversed (a, b, c, d) VALUES (0, 0, 0, 0)"); + execute("INSERT INTO %s.multiple_clustering_reversed (a, b, c, d) VALUES (0, 0, 1, 1)"); + execute("INSERT INTO %s.multiple_clustering_reversed (a, b, c, d) VALUES (0, 0, 1, 0)"); + + + UntypedResultSet results = execute("SELECT * FROM %s.multiple_clustering_reversed WHERE a=0 AND (b) > (0)"); + assertEquals(3, results.size()); + checkRow(0, results, 0, 1, 0, 0); + checkRow(1, results, 0, 1, 1, 1); + checkRow(2, results, 0, 1, 1, 0); + + results = execute("SELECT * FROM %s.multiple_clustering_reversed WHERE a=0 AND (b) >= (0)"); + assertEquals(6, results.size()); + checkRow(0, results, 0, 1, 0, 0); + checkRow(1, results, 0, 1, 1, 1); + checkRow(2, results, 0, 1, 1, 0); + checkRow(3, results, 0, 0, 0, 0); + checkRow(4, results, 0, 0, 1, 1); + checkRow(5, results, 0, 0, 1, 0); + + results = execute("SELECT * FROM %s.multiple_clustering_reversed WHERE a=0 AND (b) < (1)"); + assertEquals(3, results.size()); + checkRow(0, results, 0, 0, 0, 0); + checkRow(1, results, 0, 0, 1, 1); + checkRow(2, results, 0, 0, 1, 0); + + results = execute("SELECT * FROM %s.multiple_clustering_reversed WHERE a=0 AND (b) <= (1)"); + assertEquals(6, results.size()); + checkRow(0, results, 0, 1, 0, 0); + checkRow(1, results, 0, 1, 1, 1); + checkRow(2, results, 0, 1, 1, 0); + checkRow(3, results, 0, 0, 0, 0); + checkRow(4, results, 0, 0, 1, 1); + checkRow(5, results, 0, 0, 1, 0); + + // preserve pre-6875 behavior (even though the query result is technically incorrect) + results = execute("SELECT * FROM %s.multiple_clustering_reversed WHERE a=0 AND (b, c) > (1, 0)"); + assertEquals(0, results.size()); + } + + @Test + public void testLiteralIn() throws Throwable + { + execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 0, 0, 0)"); + execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 0, 1, 0)"); + execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 0, 1, 1)"); + + UntypedResultSet results = execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) IN ((0, 1, 0), (0, 1, 1))"); + assertEquals(2, results.size()); + checkRow(0, results, 0, 0, 1, 0); + checkRow(1, results, 0, 0, 1, 1); + + // same query, but reversed order for the IN values + results = execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) IN ((0, 1, 1), (0, 1, 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 WHERE a=0 and (b, c) IN ((0, 1))"); + 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 WHERE a=0 and (b) IN ((0))"); + assertEquals(3, results.size()); + checkRow(0, results, 0, 0, 0, 0); + checkRow(1, results, 0, 0, 1, 0); + checkRow(2, results, 0, 0, 1, 1); + } + + @Test(expected=InvalidRequestException.class) + public void testLiteralInWithShortTuple() throws Throwable + { + execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) IN ((0, 1))"); + } + + @Test(expected=InvalidRequestException.class) + public void testLiteralInWithLongTuple() throws Throwable + { + execute("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) IN ((0, 1, 2, 3, 4))"); + } + + @Test(expected=InvalidRequestException.class) + public void testLiteralInWithPartitionKey() throws Throwable + { + execute("SELECT * FROM %s.multiple_clustering WHERE (a, b, c, d) IN ((0, 1, 2, 3))"); + } + + @Test(expected=InvalidRequestException.class) + public void testLiteralInSkipsClusteringColumn() throws Throwable + { + execute("SELECT * FROM %s.multiple_clustering WHERE (c, d) IN ((0, 1))"); + } + @Test + public void testPartitionAndClusteringInClauses() throws Throwable + { + execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 0, 0, 0)"); + execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 0, 1, 0)"); + execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 0, 1, 1)"); + + execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (1, 0, 0, 0)"); + execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (1, 0, 1, 0)"); + execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (1, 0, 1, 1)"); + + UntypedResultSet results = execute("SELECT * FROM %s.multiple_clustering WHERE a IN (0, 1) AND (b, c, d) IN ((0, 1, 0), (0, 1, 1))"); + assertEquals(4, results.size()); + checkRow(0, results, 0, 0, 1, 0); + checkRow(1, results, 0, 0, 1, 1); + checkRow(2, results, 1, 0, 1, 0); + checkRow(3, results, 1, 0, 1, 1); + + // same query, but reversed order for the IN values + results = execute("SELECT * FROM %s.multiple_clustering WHERE a IN (1, 0) AND (b, c, d) IN ((0, 1, 1), (0, 1, 0))"); + assertEquals(4, results.size()); + checkRow(0, results, 1, 0, 1, 0); + checkRow(1, results, 1, 0, 1, 1); + checkRow(2, results, 0, 0, 1, 0); + checkRow(3, results, 0, 0, 1, 1); + + results = execute("SELECT * FROM %s.multiple_clustering WHERE a IN (0, 1) and (b, c) IN ((0, 1))"); + assertEquals(4, results.size()); + checkRow(0, results, 0, 0, 1, 0); + checkRow(1, results, 0, 0, 1, 1); + checkRow(2, results, 1, 0, 1, 0); + checkRow(3, results, 1, 0, 1, 1); + + results = execute("SELECT * FROM %s.multiple_clustering WHERE a IN (0, 1) and (b) IN ((0))"); + assertEquals(6, results.size()); + checkRow(0, results, 0, 0, 0, 0); + checkRow(1, results, 0, 0, 1, 0); + checkRow(2, results, 0, 0, 1, 1); + checkRow(3, results, 1, 0, 0, 0); + checkRow(4, results, 1, 0, 1, 0); + checkRow(5, results, 1, 0, 1, 1); + } + + // prepare statement tests + + @Test(expected=InvalidRequestException.class) + public void testPreparePartitionAndClusteringColumnEquality() throws Throwable + { + prepare("SELECT * FROM %s.single_clustering WHERE (a, b) = (?, ?)"); + } + + @Test(expected=InvalidRequestException.class) + public void testPrepareDifferentTupleLengths() throws Throwable + { + prepare("SELECT * FROM %s.multiple_clustering WHERE (b, c) > (?, ?, ?)"); + } + + @Test + public void testPrepareEmptyIN() throws Throwable + { + MD5Digest id = prepare("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) IN ()"); + UntypedResultSet results = executePrepared(id, makeIntOptions()); + assertTrue(results.isEmpty()); + } + + @Test(expected=InvalidRequestException.class) + public void testPreparePartitionKeyInequality() throws Throwable + { + prepare("SELECT * FROM %s.single_partition WHERE (a) > (?)"); + } + + @Test(expected=InvalidRequestException.class) + public void testPreparePartitionKeyEquality() throws Throwable + { + prepare("SELECT * FROM %s.single_partition WHERE (a) = (?)"); + } + + @Test(expected=InvalidRequestException.class) + public void testPrepareRestrictNonPrimaryKey() throws Throwable + { + prepare("SELECT * FROM %s.single_partition WHERE (b) = (?)"); + } + + @Test(expected=InvalidRequestException.class) + public void testPrepareMixEqualityAndInequality() throws Throwable + { + prepare("SELECT * FROM %s.single_clustering WHERE a=0 AND (b) = (?) AND (b) > (?)"); + } + + @Test(expected=InvalidRequestException.class) + public void testPrepareMixMultipleInequalitiesOnSameBound() throws Throwable + { + prepare("SELECT * FROM %s.single_clustering WHERE a=0 AND (b) > (?) AND (b) > (?)"); + } + + @Test(expected=InvalidRequestException.class) + public void testPrepareClusteringColumnsOutOfOrderInInequality() throws Throwable + { + prepare("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (d, c, b) > (?, ?, ?)"); + } + + @Test(expected=InvalidRequestException.class) + public void testPrepareSkipClusteringColumnInEquality() throws Throwable + { + prepare("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (c, d) = (?, ?)"); + } + + @Test(expected=InvalidRequestException.class) + public void testPrepareSkipClusteringColumnInInequality() throws Throwable + { + prepare("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (c, d) > (?, ?)"); + } + + @Test + public void testPreparedClusteringColumnEquality() throws Throwable + { + execute("INSERT INTO %s.single_clustering (a, b, c) VALUES (0, 0, 0)"); + execute("INSERT INTO %s.single_clustering (a, b, c) VALUES (0, 1, 0)"); + MD5Digest id = prepare("SELECT * FROM %s.single_clustering WHERE a=0 AND (b) = (?)"); + UntypedResultSet results = executePrepared(id, makeIntOptions(0)); + assertEquals(1, results.size()); + checkRow(0, results, 0, 0, 0); + } + + @Test + public void testPreparedClusteringColumnEqualitySingleMarker() throws Throwable + { + execute("INSERT INTO %s.single_clustering (a, b, c) VALUES (0, 0, 0)"); + execute("INSERT INTO %s.single_clustering (a, b, c) VALUES (0, 1, 0)"); + MD5Digest id = prepare("SELECT * FROM %s.single_clustering WHERE a=0 AND (b) = ?"); + UntypedResultSet results = executePrepared(id, options(tuple(0))); + assertEquals(1, results.size()); + checkRow(0, results, 0, 0, 0); + } + + @Test + public void testPreparedSingleClusteringColumnInequality() throws Throwable + { + execute("INSERT INTO %s.single_clustering (a, b, c) VALUES (0, 0, 0)"); + execute("INSERT INTO %s.single_clustering (a, b, c) VALUES (0, 1, 0)"); + execute("INSERT INTO %s.single_clustering (a, b, c) VALUES (0, 2, 0)"); + + MD5Digest id = prepare("SELECT * FROM %s.single_clustering WHERE a=0 AND (b) > (?)"); + UntypedResultSet results = executePrepared(id, makeIntOptions(0)); + assertEquals(2, results.size()); + checkRow(0, results, 0, 1, 0); + checkRow(1, results, 0, 2, 0); + + results = executePrepared(prepare("SELECT * FROM %s.single_clustering WHERE a=0 AND (b) >= (?)"), makeIntOptions(1)); + assertEquals(2, results.size()); + checkRow(0, results, 0, 1, 0); + checkRow(1, results, 0, 2, 0); + + results = executePrepared(prepare("SELECT * FROM %s.single_clustering WHERE a=0 AND (b) < (?)"), makeIntOptions(2)); + assertEquals(2, results.size()); + checkRow(0, results, 0, 0, 0); + checkRow(1, results, 0, 1, 0); + + results = executePrepared(prepare("SELECT * FROM %s.single_clustering WHERE a=0 AND (b) <= (?)"), makeIntOptions(1)); + assertEquals(2, results.size()); + checkRow(0, results, 0, 0, 0); + checkRow(1, results, 0, 1, 0); + + results = executePrepared(prepare("SELECT * FROM %s.single_clustering WHERE a=0 AND (b) > (?) AND (b) < (?)"), makeIntOptions(0, 2)); + assertEquals(1, results.size()); + checkRow(0, results, 0, 1, 0); + } + + @Test + public void testPreparedSingleClusteringColumnInequalitySingleMarker() throws Throwable + { + execute("INSERT INTO %s.single_clustering (a, b, c) VALUES (0, 0, 0)"); + execute("INSERT INTO %s.single_clustering (a, b, c) VALUES (0, 1, 0)"); + execute("INSERT INTO %s.single_clustering (a, b, c) VALUES (0, 2, 0)"); + + MD5Digest id = prepare("SELECT * FROM %s.single_clustering WHERE a=0 AND (b) > ?"); + UntypedResultSet results = executePrepared(id, options(tuple(0))); + assertEquals(2, results.size()); + checkRow(0, results, 0, 1, 0); + checkRow(1, results, 0, 2, 0); + + results = executePrepared(prepare("SELECT * FROM %s.single_clustering WHERE a=0 AND (b) >= ?"), options(tuple(1))); + assertEquals(2, results.size()); + checkRow(0, results, 0, 1, 0); + checkRow(1, results, 0, 2, 0); + + results = executePrepared(prepare("SELECT * FROM %s.single_clustering WHERE a=0 AND (b) < ?"), options(tuple(2))); + assertEquals(2, results.size()); + checkRow(0, results, 0, 0, 0); + checkRow(1, results, 0, 1, 0); + + results = executePrepared(prepare("SELECT * FROM %s.single_clustering WHERE a=0 AND (b) <= ?"), options(tuple(1))); + assertEquals(2, results.size()); + checkRow(0, results, 0, 0, 0); + checkRow(1, results, 0, 1, 0); + + + results = executePrepared(prepare("SELECT * FROM %s.single_clustering WHERE a=0 AND (b) > ? AND (b) < ?"), + options(tuple(0), tuple(2))); + assertEquals(1, results.size()); + checkRow(0, results, 0, 1, 0); + } + + @Test + public void testPrepareMultipleClusteringColumnInequality() throws Throwable + { + execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 0, 0, 0)"); + execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 0, 1, 0)"); + execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 0, 1, 1)"); + + execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 1, 0, 0)"); + execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 1, 1, 0)"); + execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 1, 1, 1)"); + + UntypedResultSet results = executePrepared(prepare( + "SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b) > (?)"), makeIntOptions(0)); + 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 = executePrepared(prepare( + "SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c) > (?, ?)"), makeIntOptions(1, 0)); + assertEquals(2, results.size()); + checkRow(0, results, 0, 1, 1, 0); + checkRow(1, results, 0, 1, 1, 1); + + results = executePrepared(prepare + ("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) > (?, ?, ?)"), makeIntOptions(1, 1, 0)); + assertEquals(1, results.size()); + checkRow(0, results, 0, 1, 1, 1); + + results = executePrepared(prepare( + "SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) > (?, ?, ?) AND (b) < (?)"), + makeIntOptions(0, 1, 0, 1)); + assertEquals(1, results.size()); + checkRow(0, results, 0, 0, 1, 1); + + results = executePrepared(prepare + ("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) > (?, ?, ?) AND (b, c) < (?, ?)"), + makeIntOptions(0, 1, 1, 1, 1)); + assertEquals(1, results.size()); + checkRow(0, results, 0, 1, 0, 0); + + results = executePrepared(prepare( + "SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) > (?, ?, ?) AND (b, c, d) < (?, ?, ?)"), + makeIntOptions(0, 1, 1, 1, 1, 0)); + assertEquals(1, results.size()); + checkRow(0, results, 0, 1, 0, 0); + + // reversed + results = executePrepared(prepare( + "SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b) > (?) ORDER BY b DESC, c DESC, d DESC"), + makeIntOptions(0)); + assertEquals(3, results.size()); + checkRow(2, results, 0, 1, 0, 0); + checkRow(1, results, 0, 1, 1, 0); + checkRow(0, results, 0, 1, 1, 1); + + results = executePrepared(prepare( + "SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) > (?, ?, ?) AND (b, c) < (?, ?) ORDER BY b DESC, c DESC, d DESC"), + makeIntOptions(0, 1, 1, 1, 1)); + assertEquals(1, results.size()); + checkRow(0, results, 0, 1, 0, 0); + } + + @Test + public void testPrepareMultipleClusteringColumnInequalitySingleMarker() throws Throwable + { + execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 0, 0, 0)"); + execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 0, 1, 0)"); + execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 0, 1, 1)"); + + execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 1, 0, 0)"); + execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 1, 1, 0)"); + execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 1, 1, 1)"); + + UntypedResultSet results = executePrepared(prepare( + "SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b) > ?"), options(tuple(0))); + 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 = executePrepared(prepare( + "SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c) > ?"), options(tuple(1, 0))); + assertEquals(2, results.size()); + checkRow(0, results, 0, 1, 1, 0); + checkRow(1, results, 0, 1, 1, 1); + + results = executePrepared(prepare + ("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) > ?"), options(tuple(1, 1, 0))); + assertEquals(1, results.size()); + checkRow(0, results, 0, 1, 1, 1); + + results = executePrepared(prepare( + "SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) > ? AND (b) < ?"), + options(tuple(0, 1, 0), tuple(1))); + assertEquals(1, results.size()); + checkRow(0, results, 0, 0, 1, 1); + + results = executePrepared(prepare + ("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) > ? AND (b, c) < ?"), + options(tuple(0, 1, 1), tuple(1, 1))); + assertEquals(1, results.size()); + checkRow(0, results, 0, 1, 0, 0); + + results = executePrepared(prepare( + "SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) > ? AND (b, c, d) < ?"), + options(tuple(0, 1, 1), tuple(1, 1, 0))); + assertEquals(1, results.size()); + checkRow(0, results, 0, 1, 0, 0); + + // reversed + results = executePrepared(prepare( + "SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b) > ? ORDER BY b DESC, c DESC, d DESC"), + options(tuple(0))); + assertEquals(3, results.size()); + checkRow(2, results, 0, 1, 0, 0); + checkRow(1, results, 0, 1, 1, 0); + checkRow(0, results, 0, 1, 1, 1); + + results = executePrepared(prepare( + "SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) > ? AND (b, c) < ? ORDER BY b DESC, c DESC, d DESC"), + options(tuple(0, 1, 1), tuple(1, 1))); + assertEquals(1, results.size()); + checkRow(0, results, 0, 1, 0, 0); + } + + @Test + public void testPrepareLiteralIn() throws Throwable + { + execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 0, 0, 0)"); + execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 0, 1, 0)"); + execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 0, 1, 1)"); + + UntypedResultSet results = executePrepared(prepare( + "SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) IN ((?, ?, ?), (?, ?, ?))"), + makeIntOptions(0, 1, 0, 0, 1, 1)); + assertEquals(2, results.size()); + checkRow(0, results, 0, 0, 1, 0); + checkRow(1, results, 0, 0, 1, 1); + + // same query, but reversed order for the IN values + results = executePrepared(prepare( + "SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) IN ((?, ?, ?), (?, ?, ?))"), + makeIntOptions(0, 1, 1, 0, 1, 0)); + assertEquals(2, results.size()); + checkRow(0, results, 0, 0, 1, 0); + checkRow(1, results, 0, 0, 1, 1); + + results = executePrepared(prepare("SELECT * FROM %s.multiple_clustering WHERE a=0 and (b, c) IN ((?, ?))"), + makeIntOptions(0, 1)); + assertEquals(2, results.size()); + checkRow(0, results, 0, 0, 1, 0); + checkRow(1, results, 0, 0, 1, 1); + + results = executePrepared(prepare("SELECT * FROM %s.multiple_clustering WHERE a=0 and (b) IN ((?))"), + makeIntOptions(0)); + assertEquals(3, results.size()); + checkRow(0, results, 0, 0, 0, 0); + checkRow(1, results, 0, 0, 1, 0); + checkRow(2, results, 0, 0, 1, 1); + } + + @Test + public void testPrepareInOneMarkerPerTuple() throws Throwable + { + execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 0, 0, 0)"); + execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 0, 1, 0)"); + execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 0, 1, 1)"); + + UntypedResultSet results = executePrepared(prepare( + "SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) IN (?, ?)"), + options(tuple(0, 1, 0), tuple(0, 1, 1))); + assertEquals(2, results.size()); + checkRow(0, results, 0, 0, 1, 0); + checkRow(1, results, 0, 0, 1, 1); + + // same query, but reversed order for the IN values + results = executePrepared(prepare( + "SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) IN (?, ?)"), + options(tuple(0, 1, 1), tuple(0, 1, 0))); + assertEquals(2, results.size()); + checkRow(0, results, 0, 0, 1, 0); + checkRow(1, results, 0, 0, 1, 1); + + + results = executePrepared(prepare("SELECT * FROM %s.multiple_clustering WHERE a=0 and (b, c) IN (?)"), + options(tuple(0, 1))); + assertEquals(2, results.size()); + checkRow(0, results, 0, 0, 1, 0); + checkRow(1, results, 0, 0, 1, 1); + + results = executePrepared(prepare("SELECT * FROM %s.multiple_clustering WHERE a=0 and (b) IN (?)"), + options(tuple(0))); + assertEquals(3, results.size()); + checkRow(0, results, 0, 0, 0, 0); + checkRow(1, results, 0, 0, 1, 0); + checkRow(2, results, 0, 0, 1, 1); + } + + @Test + public void testPrepareInOneMarker() throws Throwable + { + execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 0, 0, 0)"); + execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 0, 1, 0)"); + execute("INSERT INTO %s.multiple_clustering (a, b, c, d) VALUES (0, 0, 1, 1)"); + + UntypedResultSet results = executePrepared(prepare( + "SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) IN ?"), + options(list(tuple(0, 1, 0), tuple(0, 1, 1)))); + assertEquals(2, results.size()); + checkRow(0, results, 0, 0, 1, 0); + checkRow(1, results, 0, 0, 1, 1); + + // same query, but reversed order for the IN values + results = executePrepared(prepare( + "SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) IN ?"), + options(list(tuple(0, 1, 1), tuple(0, 1, 0)))); + assertEquals(2, results.size()); + checkRow(0, results, 0, 0, 1, 0); + checkRow(1, results, 0, 0, 1, 1); + + results = executePrepared(prepare( + "SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) IN ?"), + options(list())); + assertTrue(results.isEmpty()); + + results = executePrepared(prepare("SELECT * FROM %s.multiple_clustering WHERE a=0 and (b, c) IN ?"), + options(list(tuple(0, 1)))); + assertEquals(2, results.size()); + checkRow(0, results, 0, 0, 1, 0); + checkRow(1, results, 0, 0, 1, 1); + + results = executePrepared(prepare("SELECT * FROM %s.multiple_clustering WHERE a=0 and (b) IN ?"), + options(list(tuple(0)))); + assertEquals(3, results.size()); + checkRow(0, results, 0, 0, 0, 0); + checkRow(1, results, 0, 0, 1, 0); + checkRow(2, results, 0, 0, 1, 1); + + results = executePrepared(prepare("SELECT * FROM %s.multiple_clustering WHERE a=0 and (b) IN ?"), + options(list())); + assertTrue(results.isEmpty()); + } + + @Test(expected=InvalidRequestException.class) + public void testPrepareLiteralInWithShortTuple() throws Throwable + { + prepare("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) IN ((?, ?))"); + } + + @Test(expected=InvalidRequestException.class) + public void testPrepareLiteralInWithLongTuple() throws Throwable + { + prepare("SELECT * FROM %s.multiple_clustering WHERE a=0 AND (b, c, d) IN ((?, ?, ?, ?, ?))"); + } + + @Test(expected=InvalidRequestException.class) + public void testPrepareLiteralInWithPartitionKey() throws Throwable + { + prepare("SELECT * FROM %s.multiple_clustering WHERE (a, b, c, d) IN ((?, ?, ?, ?))"); + } + + @Test(expected=InvalidRequestException.class) + public void testPrepareLiteralInSkipsClusteringColumn() throws Throwable + { + prepare("SELECT * FROM %s.multiple_clustering WHERE (c, d) IN ((?, ?))"); + } + + private static QueryOptions makeIntOptions(Integer... values) + { + List buffers = new ArrayList<>(values.length); + for (int value : values) + buffers.add(ByteBufferUtil.bytes(value)); - return new QueryOptions(ConsistencyLevel.ONE, buffers); ++ return QueryOptions.forInternalCalls(buffers); + } + + private static ByteBuffer tuple(Integer... values) + { + List> types = new ArrayList<>(values.length); + ByteBuffer[] buffers = new ByteBuffer[values.length]; + for (int i = 0; i < values.length; i++) + { + types.add(Int32Type.instance); + buffers[i] = ByteBufferUtil.bytes(values[i]); + } + + TupleType type = new TupleType(types); + return type.buildValue(buffers); + } + + private static ByteBuffer list(ByteBuffer... values) + { - return CollectionType.pack(Arrays.asList(values), values.length); ++ return CollectionSerializer.pack(Arrays.asList(values), values.length, 3); + } + + private static QueryOptions options(ByteBuffer... buffers) + { - return new QueryOptions(ConsistencyLevel.ONE, Arrays.asList(buffers)); ++ return QueryOptions.forInternalCalls(Arrays.asList(buffers)); + } + + private static void checkRow(int rowIndex, UntypedResultSet results, Integer... expectedValues) + { + List rows = newArrayList(results.iterator()); + UntypedResultSet.Row row = rows.get(rowIndex); + Iterator columns = row.getColumns().iterator(); + for (Integer expected : expectedValues) + { + String columnName = columns.next().name.toString(); + int actual = row.getInt(columnName); + assertEquals(String.format("Expected value %d for column %s in row %d, but got %s", actual, columnName, rowIndex, expected), + (long) expected, actual); + } + } + }