You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by bl...@apache.org on 2017/04/07 10:01:00 UTC
[3/9] cassandra git commit: Merge branch cassandra-2.1 into
cassandra-2.2
Merge branch cassandra-2.1 into cassandra-2.2
Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/194329d3
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/194329d3
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/194329d3
Branch: refs/heads/cassandra-3.11
Commit: 194329d3c3c3e70897fcfb20f81076c2c45269d8
Parents: 1e102b4 64d8a1d
Author: Benjamin Lerer <b....@gmail.com>
Authored: Fri Apr 7 11:24:20 2017 +0200
Committer: Benjamin Lerer <b....@gmail.com>
Committed: Fri Apr 7 11:32:12 2017 +0200
----------------------------------------------------------------------
CHANGES.txt | 1 +
.../restrictions/StatementRestrictions.java | 17 +++++++++
.../cql3/statements/SelectStatement.java | 8 ++---
.../validation/entities/SecondaryIndexTest.java | 38 ++++++++++++++++++++
4 files changed, 60 insertions(+), 4 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/cassandra/blob/194329d3/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index e01a63c,eacd275..6ea2d59
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,58 -1,10 +1,59 @@@
-2.1.18
+2.2.10
+ * Fix JVM metric paths (CASSANDRA-13103)
+ * Honor truststore-password parameter in cassandra-stress (CASSANDRA-12773)
+ * Discard in-flight shadow round responses (CASSANDRA-12653)
+ * Don't anti-compact repaired data to avoid inconsistencies (CASSANDRA-13153)
+ * Wrong logger name in AnticompactionTask (CASSANDRA-13343)
+ * Fix queries updating multiple time the same list (CASSANDRA-13130)
+ * Fix GRANT/REVOKE when keyspace isn't specified (CASSANDRA-13053)
+ * Avoid race on receiver by starting streaming sender thread after sending init message (CASSANDRA-12886)
+ * Fix "multiple versions of ant detected..." when running ant test (CASSANDRA-13232)
+ * Coalescing strategy sleeps too much (CASSANDRA-13090)
+ * Make sure compaction stats are updated when compaction is interrupted (Backport from 3.0, CASSANDRA-12100)
+ * Fix flaky LongLeveledCompactionStrategyTest (CASSANDRA-12202)
+ * Fix failing COPY TO STDOUT (CASSANDRA-12497)
+ * Fix ColumnCounter::countAll behaviour for reverse queries (CASSANDRA-13222)
+ * Exceptions encountered calling getSeeds() breaks OTC thread (CASSANDRA-13018)
+ * Commitlog replay may fail if last mutation is within 4 bytes of end of segment (CASSANDRA-13282)
+Merged from 2.1:
+ * Fix 2ndary index queries on partition keys for tables with static columns (CASSANDRA-13147)
* Fix ParseError unhashable type list in cqlsh copy from (CASSANDRA-13364)
- * Log stacktrace of uncaught exceptions (CASSANDRA-13108)
* Remove unused repositories (CASSANDRA-13278)
+ * Log stacktrace of uncaught exceptions (CASSANDRA-13108)
+
-2.1.17
+2.2.9
+ * Fix negative mean latency metric (CASSANDRA-12876)
+ * Use only one file pointer when creating commitlog segments (CASSANDRA-12539)
+ * Fix speculative retry bugs (CASSANDRA-13009)
+ * Fix handling of nulls and unsets in IN conditions (CASSANDRA-12981)
+ * Remove support for non-JavaScript UDFs (CASSANDRA-12883)
+ * Fix DynamicEndpointSnitch noop in multi-datacenter situations (CASSANDRA-13074)
+ * cqlsh copy-from: encode column names to avoid primary key parsing errors (CASSANDRA-12909)
+ * Temporarily fix bug that creates commit log when running offline tools (CASSANDRA-8616)
+ * Reduce granuality of OpOrder.Group during index build (CASSANDRA-12796)
+ * Test bind parameters and unset parameters in InsertUpdateIfConditionTest (CASSANDRA-12980)
+ * Do not specify local address on outgoing connection when listen_on_broadcast_address is set (CASSANDRA-12673)
+ * Use saved tokens when setting local tokens on StorageService.joinRing (CASSANDRA-12935)
+ * cqlsh: fix DESC TYPES errors (CASSANDRA-12914)
+ * Fix leak on skipped SSTables in sstableupgrade (CASSANDRA-12899)
+ * Avoid blocking gossip during pending range calculation (CASSANDRA-12281)
+ * Fix purgeability of tombstones with max timestamp (CASSANDRA-12792)
+ * Fail repair if participant dies during sync or anticompaction (CASSANDRA-12901)
+ * cqlsh COPY: unprotected pk values before converting them if not using prepared statements (CASSANDRA-12863)
+ * Fix Util.spinAssertEquals (CASSANDRA-12283)
+ * Fix potential NPE for compactionstats (CASSANDRA-12462)
+ * Prepare legacy authenticate statement if credentials table initialised after node startup (CASSANDRA-12813)
+ * Change cassandra.wait_for_tracing_events_timeout_secs default to 0 (CASSANDRA-12754)
+ * Clean up permissions when a UDA is dropped (CASSANDRA-12720)
+ * Limit colUpdateTimeDelta histogram updates to reasonable deltas (CASSANDRA-11117)
+ * Fix leak errors and execution rejected exceptions when draining (CASSANDRA-12457)
+ * Fix merkle tree depth calculation (CASSANDRA-12580)
+ * Make Collections deserialization more robust (CASSANDRA-12618)
+ * Better handle invalid system roles table (CASSANDRA-12700)
+ * Split consistent range movement flag correction (CASSANDRA-12786)
+ * CompactionTasks now correctly drops sstables out of compaction when not enough disk space is available (CASSANDRA-12979)
+Merged from 2.1:
* Use portable stderr for java error in startup (CASSANDRA-13211)
* Fix Thread Leak in OutboundTcpConnection (CASSANDRA-13204)
* Coalescing strategy can enter infinite loop (CASSANDRA-13159)
http://git-wip-us.apache.org/repos/asf/cassandra/blob/194329d3/src/java/org/apache/cassandra/cql3/restrictions/StatementRestrictions.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/restrictions/StatementRestrictions.java
index 8035877,0000000..2c396c4
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/cql3/restrictions/StatementRestrictions.java
+++ b/src/java/org/apache/cassandra/cql3/restrictions/StatementRestrictions.java
@@@ -1,656 -1,0 +1,673 @@@
+/*
+ * 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.restrictions;
+
+import java.nio.ByteBuffer;
+import java.util.*;
+
+import com.google.common.base.Joiner;
+import com.google.common.collect.Iterables;
+
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.cql3.*;
+import org.apache.cassandra.cql3.functions.Function;
+import org.apache.cassandra.cql3.statements.Bound;
+import org.apache.cassandra.db.*;
+import org.apache.cassandra.db.composites.Composite;
+import org.apache.cassandra.db.index.SecondaryIndexManager;
+import org.apache.cassandra.dht.*;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+import static org.apache.cassandra.config.ColumnDefinition.toIdentifiers;
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkFalse;
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkNotNull;
+import static org.apache.cassandra.cql3.statements.RequestValidations.invalidRequest;
+
+/**
+ * The restrictions corresponding to the relations specified on the where-clause of CQL query.
+ */
+public final class StatementRestrictions
+{
+ public static final String REQUIRES_ALLOW_FILTERING_MESSAGE =
+ "Cannot execute this query as it might involve data filtering and " +
+ "thus may have unpredictable performance. If you want to execute " +
+ "this query despite the performance unpredictability, use ALLOW FILTERING";
+
+ /**
+ * The Column Family meta data
+ */
+ public final CFMetaData cfm;
+
+ /**
+ * Restrictions on partitioning columns
+ */
+ private PrimaryKeyRestrictions partitionKeyRestrictions;
+
+ /**
+ * Restrictions on clustering columns
+ */
+ private PrimaryKeyRestrictions clusteringColumnsRestrictions;
+
+ /**
+ * Restriction on non-primary key columns (i.e. secondary index restrictions)
+ */
+ private RestrictionSet nonPrimaryKeyRestrictions;
+
+ /**
++ * <code>true</code> if nonPrimaryKeyRestrictions contains restriction on a regular column,
++ * <code>false</code> otherwise.
++ */
++ private boolean hasRegularColumnsRestriction = false;
++
++ /**
+ * The restrictions used to build the index expressions
+ */
+ private final List<Restrictions> indexRestrictions = new ArrayList<>();
+
+ /**
+ * <code>true</code> if the secondary index need to be queried, <code>false</code> otherwise
+ */
+ private boolean usesSecondaryIndexing;
+
+ /**
+ * Specify if the query will return a range of partition keys.
+ */
+ private boolean isKeyRange;
+
+ /**
+ * Creates a new empty <code>StatementRestrictions</code>.
+ *
+ * @param cfm the column family meta data
+ * @return a new empty <code>StatementRestrictions</code>.
+ */
+ public static StatementRestrictions empty(CFMetaData cfm)
+ {
+ return new StatementRestrictions(cfm);
+ }
+
+ private StatementRestrictions(CFMetaData cfm)
+ {
+ this.cfm = cfm;
+ this.partitionKeyRestrictions = new PrimaryKeyRestrictionSet(cfm.getKeyValidatorAsCType());
+ this.clusteringColumnsRestrictions = new PrimaryKeyRestrictionSet(cfm.comparator);
+ this.nonPrimaryKeyRestrictions = new RestrictionSet();
+ }
+
+ public StatementRestrictions(CFMetaData cfm,
+ List<Relation> whereClause,
+ VariableSpecifications boundNames,
+ boolean selectsOnlyStaticColumns,
+ boolean selectACollection,
+ boolean allowFiltering)
+ {
+ this.cfm = cfm;
+ this.partitionKeyRestrictions = new PrimaryKeyRestrictionSet(cfm.getKeyValidatorAsCType());
+ this.clusteringColumnsRestrictions = new PrimaryKeyRestrictionSet(cfm.comparator);
+ this.nonPrimaryKeyRestrictions = new RestrictionSet();
+
+ /*
+ * WHERE clause. For a given entity, rules are: - EQ relation conflicts with anything else (including a 2nd EQ)
+ * - Can't have more than one LT(E) relation (resp. GT(E) relation) - IN relation are restricted to row keys
+ * (for now) and conflicts with anything else (we could allow two IN for the same entity but that doesn't seem
+ * very useful) - The value_alias cannot be restricted in any way (we don't support wide rows with indexed value
+ * in CQL so far)
+ */
+ for (Relation relation : whereClause)
+ addRestriction(relation.toRestriction(cfm, boundNames));
+
+ SecondaryIndexManager secondaryIndexManager = Keyspace.open(cfm.ksName).getColumnFamilyStore(cfm.cfName).indexManager;
+ boolean hasQueriableClusteringColumnIndex = clusteringColumnsRestrictions.hasSupportingIndex(secondaryIndexManager);
+ boolean hasQueriableIndex = hasQueriableClusteringColumnIndex
+ || partitionKeyRestrictions.hasSupportingIndex(secondaryIndexManager)
+ || nonPrimaryKeyRestrictions.hasSupportingIndex(secondaryIndexManager);
+
+ // At this point, the select statement if fully constructed, but we still have a few things to validate
+ processPartitionKeyRestrictions(hasQueriableIndex);
+
+ // Some but not all of the partition key columns have been specified;
+ // hence we need turn these restrictions into index expressions.
+ if (usesSecondaryIndexing)
+ indexRestrictions.add(partitionKeyRestrictions);
+
+ checkFalse(selectsOnlyStaticColumns && hasClusteringColumnsRestriction(),
+ "Cannot restrict clustering columns when selecting only static columns");
+
+ processClusteringColumnsRestrictions(hasQueriableIndex, selectACollection);
+
+ // Covers indexes on the first clustering column (among others).
+ if (isKeyRange && hasQueriableClusteringColumnIndex)
+ usesSecondaryIndexing = true;
+
+ usesSecondaryIndexing = usesSecondaryIndexing || clusteringColumnsRestrictions.isContains();
+
+ if (usesSecondaryIndexing)
+ indexRestrictions.add(clusteringColumnsRestrictions);
+
+ // Even if usesSecondaryIndexing is false at this point, we'll still have to use one if
+ // there is restrictions not covered by the PK.
+ if (!nonPrimaryKeyRestrictions.isEmpty())
+ {
+ if (!hasQueriableIndex)
+ {
+ // Filtering for non-index query is only supported for thrift static CFs
+ if (cfm.comparator.isDense() || cfm.comparator.isCompound())
+ throw invalidRequest("Predicates on non-primary-key columns (%s) are not yet supported for non secondary index queries",
+ Joiner.on(", ").join(toIdentifiers(nonPrimaryKeyRestrictions.getColumnDefs())));
+
+ if (!allowFiltering)
+ throw invalidRequest(REQUIRES_ALLOW_FILTERING_MESSAGE);
+ }
+ usesSecondaryIndexing = true;
+ indexRestrictions.add(nonPrimaryKeyRestrictions);
+ }
+
+ if (usesSecondaryIndexing)
+ validateSecondaryIndexSelections(selectsOnlyStaticColumns);
+ }
+
+ private void addRestriction(Restriction restriction) throws InvalidRequestException
+ {
+ if (restriction.isMultiColumn())
+ clusteringColumnsRestrictions = clusteringColumnsRestrictions.mergeWith(restriction);
+ else if (restriction.isOnToken())
+ partitionKeyRestrictions = partitionKeyRestrictions.mergeWith(restriction);
+ else
+ addSingleColumnRestriction((SingleColumnRestriction) restriction);
+ }
+
+ public Iterable<Function> getFunctions()
+ {
+ return Iterables.concat(partitionKeyRestrictions.getFunctions(),
+ clusteringColumnsRestrictions.getFunctions(),
+ nonPrimaryKeyRestrictions.getFunctions());
+ }
+
+ private void addSingleColumnRestriction(SingleColumnRestriction restriction) throws InvalidRequestException
+ {
+ ColumnDefinition def = restriction.columnDef;
+ if (def.isPartitionKey())
+ partitionKeyRestrictions = partitionKeyRestrictions.mergeWith(restriction);
+ else if (def.isClusteringColumn())
+ clusteringColumnsRestrictions = clusteringColumnsRestrictions.mergeWith(restriction);
+ else
++ {
++ if (restriction.columnDef.kind == ColumnDefinition.Kind.REGULAR)
++ {
++ hasRegularColumnsRestriction = true;
++ }
+ nonPrimaryKeyRestrictions = nonPrimaryKeyRestrictions.addRestriction(restriction);
++ }
+ }
+
+ /**
+ * Checks if the restrictions on the partition key is an IN restriction.
+ *
+ * @return <code>true</code> the restrictions on the partition key is an IN restriction, <code>false</code>
+ * otherwise.
+ */
+ public boolean keyIsInRelation()
+ {
+ return partitionKeyRestrictions.isIN();
+ }
+
+ /**
+ * Checks if the query request a range of partition keys.
+ *
+ * @return <code>true</code> if the query request a range of partition keys, <code>false</code> otherwise.
+ */
+ public boolean isKeyRange()
+ {
+ return this.isKeyRange;
+ }
+
++ public boolean hasRegularColumnsRestriction()
++ {
++ return hasRegularColumnsRestriction;
++ }
++
+ /**
+ * Checks if the secondary index need to be queried.
+ *
+ * @return <code>true</code> if the secondary index need to be queried, <code>false</code> otherwise.
+ */
+ public boolean usesSecondaryIndexing()
+ {
+ return this.usesSecondaryIndexing;
+ }
+
+ private void processPartitionKeyRestrictions(boolean hasQueriableIndex) throws InvalidRequestException
+ {
+ // If there is a queriable index, no special condition are required on the other restrictions.
+ // But we still need to know 2 things:
+ // - If we don't have a queriable index, is the query ok
+ // - Is it queriable without 2ndary index, which is always more efficient
+ // If a component of the partition key is restricted by a relation, all preceding
+ // components must have a EQ. Only the last partition key component can be in IN relation.
+ if (partitionKeyRestrictions.isOnToken())
+ isKeyRange = true;
+
+ if (hasPartitionKeyUnrestrictedComponents())
+ {
+ if (!partitionKeyRestrictions.isEmpty())
+ {
+ if (!hasQueriableIndex)
+ throw invalidRequest("Partition key parts: %s must be restricted as other parts are",
+ Joiner.on(", ").join(getPartitionKeyUnrestrictedComponents()));
+ }
+
+ isKeyRange = true;
+ usesSecondaryIndexing = hasQueriableIndex;
+ }
+ }
+
+ /**
+ * Checks if the partition key has some unrestricted components.
+ * @return <code>true</code> if the partition key has some unrestricted components, <code>false</code> otherwise.
+ */
+ private boolean hasPartitionKeyUnrestrictedComponents()
+ {
+ return partitionKeyRestrictions.size() < cfm.partitionKeyColumns().size();
+ }
+
+ public boolean hasPartitionKeyRestrictions()
+ {
+ return !partitionKeyRestrictions.isEmpty();
+ }
+
+ /**
+ * Checks if the restrictions contain any non-primary key restrictions
+ * @return <code>true</code> if the restrictions contain any non-primary key restrictions, <code>false</code> otherwise.
+ */
+ public boolean hasNonPrimaryKeyRestrictions()
+ {
+ return !nonPrimaryKeyRestrictions.isEmpty();
+ }
+
+ /**
+ * Returns the partition key components that are not restricted.
+ * @return the partition key components that are not restricted.
+ */
+ private List<ColumnIdentifier> getPartitionKeyUnrestrictedComponents()
+ {
+ List<ColumnDefinition> list = new ArrayList<>(cfm.partitionKeyColumns());
+ list.removeAll(partitionKeyRestrictions.getColumnDefs());
+ return ColumnDefinition.toIdentifiers(list);
+ }
+
+ /**
+ * Processes the clustering column restrictions.
+ *
+ * @param hasQueriableIndex <code>true</code> if some of the queried data are indexed, <code>false</code> otherwise
+ * @param selectACollection <code>true</code> if the query should return a collection column
+ * @throws InvalidRequestException if the request is invalid
+ */
+ private void processClusteringColumnsRestrictions(boolean hasQueriableIndex,
+ boolean selectACollection) throws InvalidRequestException
+ {
+ validateClusteringRestrictions(hasQueriableIndex);
+
+ checkFalse(clusteringColumnsRestrictions.isIN() && selectACollection,
+ "Cannot restrict clustering columns by IN relations when a collection is selected by the query");
+ checkFalse(clusteringColumnsRestrictions.isContains() && !hasQueriableIndex,
+ "Cannot restrict clustering columns by a CONTAINS relation without a secondary index");
+
+ if (hasClusteringColumnsRestriction() && clusteringRestrictionsNeedFiltering())
+ {
+ if (hasQueriableIndex)
+ {
+ usesSecondaryIndexing = true;
+ return;
+ }
+
+ List<ColumnDefinition> clusteringColumns = cfm.clusteringColumns();
+ List<ColumnDefinition> restrictedColumns = new LinkedList<>(clusteringColumnsRestrictions.getColumnDefs());
+
+ for (int i = 0, m = restrictedColumns.size(); i < m; i++)
+ {
+ ColumnDefinition clusteringColumn = clusteringColumns.get(i);
+ ColumnDefinition restrictedColumn = restrictedColumns.get(i);
+
+ if (!clusteringColumn.equals(restrictedColumn))
+ {
+ throw invalidRequest(
+ "PRIMARY KEY column \"%s\" cannot be restricted as preceding column \"%s\" is not restricted",
+ restrictedColumn.name,
+ clusteringColumn.name);
+ }
+ }
+ }
+ }
+
+ /**
+ * Validates whether or not restrictions are allowed for execution when secondary index is not used.
+ */
+ public final void validateClusteringRestrictions(boolean hasQueriableIndex)
+ {
+ assert clusteringColumnsRestrictions instanceof PrimaryKeyRestrictionSet;
+
+ // If there's a queriable index, filtering will take care of clustering restrictions
+ if (hasQueriableIndex)
+ return;
+
+ Iterator<Restriction> iter = ((PrimaryKeyRestrictionSet)clusteringColumnsRestrictions).iterator();
+ Restriction previousRestriction = null;
+
+ while (iter.hasNext())
+ {
+ Restriction restriction = iter.next();
+
+ if (previousRestriction != null)
+ {
+ ColumnDefinition lastRestrictionStart = previousRestriction.getFirstColumn();
+ ColumnDefinition newRestrictionStart = restriction.getFirstColumn();
+
+ if (previousRestriction.isSlice() && newRestrictionStart.position() > lastRestrictionStart.position())
+ throw invalidRequest("Clustering column \"%s\" cannot be restricted (preceding column \"%s\" is restricted by a non-EQ relation)",
+ newRestrictionStart.name,
+ lastRestrictionStart.name);
+ }
+ previousRestriction = restriction;
+ }
+ }
+
+ public final boolean clusteringRestrictionsNeedFiltering()
+ {
+ assert clusteringColumnsRestrictions instanceof PrimaryKeyRestrictionSet;
+ return ((PrimaryKeyRestrictionSet) clusteringColumnsRestrictions).needsFiltering();
+ }
+
+ public List<IndexExpression> getIndexExpressions(SecondaryIndexManager indexManager,
+ QueryOptions options) throws InvalidRequestException
+ {
+ if (!usesSecondaryIndexing || indexRestrictions.isEmpty())
+ return Collections.emptyList();
+
+ List<IndexExpression> expressions = new ArrayList<>();
+ for (Restrictions restrictions : indexRestrictions)
+ restrictions.addIndexExpressionTo(expressions, indexManager, options);
+
+ return expressions;
+ }
+
+ /**
+ * Returns the partition keys for which the data is requested.
+ *
+ * @param options the query options
+ * @return the partition keys for which the data is requested.
+ * @throws InvalidRequestException if the partition keys cannot be retrieved
+ */
+ public Collection<ByteBuffer> getPartitionKeys(final QueryOptions options) throws InvalidRequestException
+ {
+ return partitionKeyRestrictions.values(cfm, options);
+ }
+
+ /**
+ * Returns the specified bound of the partition key.
+ *
+ * @param b the boundary type
+ * @param options the query options
+ * @return the specified bound of the partition key
+ * @throws InvalidRequestException if the boundary cannot be retrieved
+ */
+ private ByteBuffer getPartitionKeyBound(Bound b, QueryOptions options) throws InvalidRequestException
+ {
+ // Deal with unrestricted partition key components (special-casing is required to deal with 2i queries on the
+ // first
+ // component of a composite partition key).
+ if (hasPartitionKeyUnrestrictedComponents())
+ 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 partitionKeyRestrictions.bounds(cfm, b, options).get(0);
+ }
+
+ /**
+ * Returns the partition key bounds.
+ *
+ * @param options the query options
+ * @return the partition key bounds
+ * @throws InvalidRequestException if the query is invalid
+ */
+ public AbstractBounds<RowPosition> getPartitionKeyBounds(QueryOptions options) throws InvalidRequestException
+ {
+ IPartitioner p = StorageService.getPartitioner();
+
+ if (partitionKeyRestrictions.isOnToken())
+ {
+ return getPartitionKeyBoundsForTokenRestrictions(p, options);
+ }
+
+ return getPartitionKeyBounds(p, options);
+ }
+
+ private AbstractBounds<RowPosition> getPartitionKeyBounds(IPartitioner p,
+ QueryOptions options) throws InvalidRequestException
+ {
+ ByteBuffer startKeyBytes = getPartitionKeyBound(Bound.START, options);
+ ByteBuffer finishKeyBytes = getPartitionKeyBound(Bound.END, options);
+
+ RowPosition startKey = RowPosition.ForKey.get(startKeyBytes, p);
+ RowPosition finishKey = RowPosition.ForKey.get(finishKeyBytes, p);
+
+ if (startKey.compareTo(finishKey) > 0 && !finishKey.isMinimum())
+ return null;
+
+ if (partitionKeyRestrictions.isInclusive(Bound.START))
+ {
+ return partitionKeyRestrictions.isInclusive(Bound.END)
+ ? new Bounds<>(startKey, finishKey)
+ : new IncludingExcludingBounds<>(startKey, finishKey);
+ }
+
+ return partitionKeyRestrictions.isInclusive(Bound.END)
+ ? new Range<>(startKey, finishKey)
+ : new ExcludingBounds<>(startKey, finishKey);
+ }
+
+ private AbstractBounds<RowPosition> getPartitionKeyBoundsForTokenRestrictions(IPartitioner p,
+ QueryOptions options)
+ throws InvalidRequestException
+ {
+ Token startToken = getTokenBound(Bound.START, options, p);
+ Token endToken = getTokenBound(Bound.END, options, p);
+
+ boolean includeStart = partitionKeyRestrictions.isInclusive(Bound.START);
+ boolean includeEnd = partitionKeyRestrictions.isInclusive(Bound.END);
+
+ /*
+ * If we ask SP.getRangeSlice() for (token(200), token(200)], it will happily return the whole ring.
+ * However, wrapping range doesn't really make sense for CQL, and we want to return an empty result in that
+ * case (CASSANDRA-5573). So special case to create a range that is guaranteed to be empty.
+ *
+ * In practice, we want to return an empty result set if either startToken > endToken, or both are equal but
+ * one of the bound is excluded (since [a, a] can contains something, but not (a, a], [a, a) or (a, a)).
+ * Note though that in the case where startToken or endToken is the minimum token, then this special case
+ * rule should not apply.
+ */
+ int cmp = startToken.compareTo(endToken);
+ if (!startToken.isMinimum() && !endToken.isMinimum()
+ && (cmp > 0 || (cmp == 0 && (!includeStart || !includeEnd))))
+ return null;
+
+ RowPosition start = includeStart ? startToken.minKeyBound() : startToken.maxKeyBound();
+ RowPosition end = includeEnd ? endToken.maxKeyBound() : endToken.minKeyBound();
+
+ return new Range<>(start, end);
+ }
+
+ private Token getTokenBound(Bound b, QueryOptions options, IPartitioner p) throws InvalidRequestException
+ {
+ if (!partitionKeyRestrictions.hasBound(b))
+ return p.getMinimumToken();
+
+ ByteBuffer value = partitionKeyRestrictions.bounds(cfm, b, options).get(0);
+ checkNotNull(value, "Invalid null token value");
+ return p.getTokenFactory().fromByteArray(value);
+ }
+
+ /**
+ * Checks if the query does not contains any restriction on the clustering columns.
+ *
+ * @return <code>true</code> if the query does not contains any restriction on the clustering columns,
+ * <code>false</code> otherwise.
+ */
+ public boolean hasNoClusteringColumnsRestriction()
+ {
+ return clusteringColumnsRestrictions.isEmpty();
+ }
+
+ /**
+ * Checks if the query has some restrictions on the clustering columns.
+ *
+ * @return <code>true</code> if the query has some restrictions on the clustering columns,
+ * <code>false</code> otherwise.
+ */
+ public boolean hasClusteringColumnsRestriction()
+ {
+ return !clusteringColumnsRestrictions.isEmpty();
+ }
+
+ // For non-composite slices, we don't support internally the difference between exclusive and
+ // inclusive bounds, so we deal with it manually.
+ public boolean isNonCompositeSliceWithExclusiveBounds()
+ {
+ return !cfm.comparator.isCompound()
+ && clusteringColumnsRestrictions.isSlice()
+ && (!clusteringColumnsRestrictions.isInclusive(Bound.START) || !clusteringColumnsRestrictions.isInclusive(Bound.END));
+ }
+
+ /**
+ * Returns the requested clustering columns as <code>Composite</code>s.
+ *
+ * @param options the query options
+ * @return the requested clustering columns as <code>Composite</code>s
+ * @throws InvalidRequestException if the query is not valid
+ */
+ public List<Composite> getClusteringColumnsAsComposites(QueryOptions options) throws InvalidRequestException
+ {
+ return clusteringColumnsRestrictions.valuesAsComposites(cfm, options);
+ }
+
+ /**
+ * Returns the bounds (start or end) of the clustering columns as <code>Composites</code>.
+ *
+ * @param b the bound type
+ * @param options the query options
+ * @return the bounds (start or end) of the clustering columns as <code>Composites</code>
+ * @throws InvalidRequestException if the request is not valid
+ */
+ public List<Composite> getClusteringColumnsBoundsAsComposites(Bound b,
+ QueryOptions options) throws InvalidRequestException
+ {
+ List<Composite> bounds = clusteringColumnsRestrictions.boundsAsComposites(cfm, b, options);
+ for (Composite c : bounds) {
+ if (!c.isEmpty())
+ QueryProcessor.validateComposite(c, cfm.comparator);
+ }
+ return bounds;
+ }
+
+ /**
+ * Returns the bounds (start or end) of the clustering columns.
+ *
+ * @param b the bound type
+ * @param options the query options
+ * @return the bounds (start or end) of the clustering columns
+ * @throws InvalidRequestException if the request is not valid
+ */
+ public List<ByteBuffer> getClusteringColumnsBounds(Bound b, QueryOptions options) throws InvalidRequestException
+ {
+ return clusteringColumnsRestrictions.bounds(cfm, b, options);
+ }
+
+ /**
+ * Checks if the bounds (start or end) of the clustering columns are inclusive.
+ *
+ * @param bound the bound type
+ * @return <code>true</code> if the bounds (start or end) of the clustering columns are inclusive,
+ * <code>false</code> otherwise
+ */
+ public boolean areRequestedBoundsInclusive(Bound bound)
+ {
+ return clusteringColumnsRestrictions.isInclusive(bound);
+ }
+
+ /**
+ * Checks if the query returns a range of columns.
+ *
+ * @return <code>true</code> if the query returns a range of columns, <code>false</code> otherwise.
+ */
+ public boolean isColumnRange()
+ {
+ // Due to CASSANDRA-5762, we always do a slice for CQL3 tables (not dense, composite).
+ // Static CF (non dense but non composite) never entails a column slice however
+ if (!cfm.comparator.isDense())
+ return cfm.comparator.isCompound();
+
+ // Otherwise (i.e. for compact table where we don't have a row marker anyway and thus don't care about
+ // CASSANDRA-5762),
+ // it is a range query if it has at least one the column alias for which no relation is defined or is not EQ.
+ return clusteringColumnsRestrictions.size() < cfm.clusteringColumns().size() || clusteringColumnsRestrictions.isSlice();
+ }
+
+ /**
+ * Checks if the query need to use filtering.
+ * @return <code>true</code> if the query need to use filtering, <code>false</code> otherwise.
+ */
+ public boolean needFiltering()
+ {
+ int numberOfRestrictedColumns = 0;
+ for (Restrictions restrictions : indexRestrictions)
+ numberOfRestrictedColumns += restrictions.size();
+
+ return numberOfRestrictedColumns > 1
+ || (numberOfRestrictedColumns == 0 && !clusteringColumnsRestrictions.isEmpty())
+ || (numberOfRestrictedColumns != 0
+ && nonPrimaryKeyRestrictions.hasMultipleContains());
+ }
+
+ private void validateSecondaryIndexSelections(boolean selectsOnlyStaticColumns) throws InvalidRequestException
+ {
+ checkFalse(keyIsInRelation(),
+ "Select on indexed columns and with IN clause for the PRIMARY KEY are not supported");
+ // When the user only select static columns, the intent is that we don't query the whole partition but just
+ // the static parts. But 1) we don't have an easy way to do that with 2i and 2) since we don't support index on
+ // static columns
+ // so far, 2i means that you've restricted a non static column, so the query is somewhat non-sensical.
+ checkFalse(selectsOnlyStaticColumns, "Queries using 2ndary indexes don't support selecting only static columns");
+ }
+
+ public void reverse()
+ {
+ clusteringColumnsRestrictions = new ReversedPrimaryKeyRestrictions(clusteringColumnsRestrictions);
+ }
+
+ /**
+ * Checks if the query will never return any rows.
+ *
+ * @param options the query options
+ * @return {@code true} if the query will never return any rows, {@false} otherwise
+ */
+ public boolean isNotReturningAnyRows(QueryOptions options)
+ {
+ return clusteringColumnsRestrictions.isNotReturningAnyRows(cfm, options);
+ }
+}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/194329d3/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
index edf8e47,74fb201..13276c7
--- a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
@@@ -707,19 -1363,18 +707,19 @@@ public class SelectStatement implement
}
Iterator<Cell> cells = cf.getSortedColumns().iterator();
- if (sliceRestriction != null)
+ if (restrictions.isNonCompositeSliceWithExclusiveBounds())
cells = applySliceRestriction(cells, options);
+ int protocolVersion = options.getProtocolVersion();
CQL3Row.RowIterator iter = cfm.comparator.CQL3RowBuilder(cfm, now).group(cells);
- // If there is static columns but there is no non-static row, then provided the select was a full
- // partition selection (i.e. not a 2ndary index search and there was no condition on clustering columns)
- // then we want to include the static columns in the result set (and we're done).
+ // If there is static columns but there is no non-static row,
+ // and the select was a full partition selection (i.e. there was no condition on clustering or regular columns),
+ // we want to include the static columns in the result set (and we're done).
CQL3Row staticRow = iter.getStaticRow();
- if (staticRow != null && !iter.hasNext() && !restrictions.usesSecondaryIndexing() && restrictions.hasNoClusteringColumnsRestriction())
- if (staticRow != null && !iter.hasNext() && hasNoClusteringColumnsRestriction() && hasNoRegularColumnsRestriction())
++ if (staticRow != null && !iter.hasNext() && !restrictions.hasClusteringColumnsRestriction() && !restrictions.hasRegularColumnsRestriction())
{
- result.newRow();
+ result.newRow(protocolVersion);
for (ColumnDefinition def : selection.getColumns())
{
switch (def.kind)
http://git-wip-us.apache.org/repos/asf/cassandra/blob/194329d3/test/unit/org/apache/cassandra/cql3/validation/entities/SecondaryIndexTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/cql3/validation/entities/SecondaryIndexTest.java
index 11d2462,b723f60..b653f4e
--- a/test/unit/org/apache/cassandra/cql3/validation/entities/SecondaryIndexTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/SecondaryIndexTest.java
@@@ -822,109 -805,18 +845,124 @@@ public class SecondaryIndexTest extend
row(bytes("foo124"), EMPTY_BYTE_BUFFER));
}
+ @Test
+ public void testIndexOnRegularColumnWithPartitionWithoutRows() throws Throwable
+ {
+ createTable("CREATE TABLE %s (pk int, c int, s int static, v int, PRIMARY KEY(pk, c))");
+ createIndex("CREATE INDEX ON %s (v)");
+ execute("INSERT INTO %s (pk, c, s, v) VALUES (?, ?, ?, ?)", 1, 1, 9, 1);
+ execute("INSERT INTO %s (pk, c, s, v) VALUES (?, ?, ?, ?)", 1, 2, 9, 2);
+ execute("INSERT INTO %s (pk, s) VALUES (?, ?)", 2, 9);
+ execute("INSERT INTO %s (pk, c, s, v) VALUES (?, ?, ?, ?)", 3, 1, 9, 1);
+ flush();
+ execute("DELETE FROM %s WHERE pk = ? and c = ?", 3, 1);
+ assertRows(execute("SELECT * FROM %s WHERE v = ?", 1),
+ row(1, 1, 9, 1));
+ }
++
+ /**
+ * Custom index used to test the behavior of the system when the index is not ready.
+ * As Custom indices cannot by <code>PerColumnSecondaryIndex</code> we use a <code>PerRowSecondaryIndex</code>
+ * to avoid the check but return a <code>CompositesSearcher</code>.
+ */
+ public static class IndexBlockingOnInitialization extends PerRowSecondaryIndex
+ {
+ private volatile CountDownLatch latch = new CountDownLatch(1);
+
+ @Override
+ public void index(ByteBuffer rowKey, ColumnFamily cf)
+ {
+ try
+ {
+ latch.await();
+ }
+ catch (InterruptedException e)
+ {
+ Thread.interrupted();
+ }
+ }
+
+ @Override
+ public void delete(DecoratedKey key, Group opGroup)
+ {
+ }
+
+ @Override
+ public void init()
+ {
+ }
+
+ @Override
+ public void reload()
+ {
+ }
+
+ @Override
+ public void validateOptions() throws ConfigurationException
+ {
+ }
+
+ @Override
+ public String getIndexName()
+ {
+ return "testIndex";
+ }
+
+ @Override
+ protected SecondaryIndexSearcher createSecondaryIndexSearcher(Set<ByteBuffer> columns)
+ {
+ return new CompositesSearcher(baseCfs.indexManager, columns)
+ {
+ @Override
+ public boolean canHandleIndexClause(List<IndexExpression> clause)
+ {
+ return true;
+ }
+
+ @Override
+ public void validate(IndexExpression indexExpression) throws InvalidRequestException
+ {
+ }
+ };
+ }
+
+ @Override
+ public void forceBlockingFlush()
+ {
+ }
+
+ @Override
+ public ColumnFamilyStore getIndexCfs()
+ {
+ return baseCfs;
+ }
+
+ @Override
+ public void removeIndex(ByteBuffer columnName)
+ {
+ latch.countDown();
+ }
+
+ @Override
+ public void invalidate()
+ {
+ }
+
+ @Override
+ public void truncateBlocking(long truncatedAt)
+ {
+ }
+
+ @Override
+ public boolean indexes(CellName name)
+ {
+ return false;
+ }
+
+ @Override
+ public long estimateResultRows()
+ {
+ return 0;
+ }
+ }
}