You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by sl...@apache.org on 2014/02/21 14:03:22 UTC

git commit: Disallow post-query re-ordering when paging

Repository: cassandra
Updated Branches:
  refs/heads/cassandra-2.0 54a7e0034 -> 458965673


Disallow post-query re-ordering when paging

patch by slebresne; reviewed by iamaleksey for CASSANDRA-6722


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

Branch: refs/heads/cassandra-2.0
Commit: 4589656733cab73776ec16cf485dee984718a7e3
Parents: 54a7e00
Author: Sylvain Lebresne <sy...@datastax.com>
Authored: Fri Feb 21 14:02:15 2014 +0100
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Fri Feb 21 14:02:15 2014 +0100

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../cql3/statements/SelectStatement.java        | 69 ++++----------------
 2 files changed, 14 insertions(+), 56 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/45896567/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index a5e1016..b25ff47 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -20,6 +20,7 @@
  * Fix replaying pre-2.0 commit logs (CASSANDRA-6714)
  * Add static columns to CQL3 (CASSANDRA-6561)
  * Optimize single partition batch statements (CASSANDRA-6737)
+ * Disallow post-query re-ordering when paging (CASSANDRA-6722)
 Merged from 1.2:
  * Catch memtable flush exceptions during shutdown (CASSANDRA-6735)
  * Fix broken streams when replacing with same IP (CASSANDRA-6622)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/45896567/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
index 2636c83..9fbed03 100644
--- a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
@@ -219,6 +219,11 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
             if (parameters.isCount)
                 return pageCountQuery(pager, variables, pageSize, now);
 
+            // We can't properly do post-query ordering if we page (see #6722)
+            if (needsPostQueryOrdering())
+                throw new InvalidRequestException("Cannot page queries with both ORDER BY and a IN restriction on the partition key; you must either remove the "
+                                                + "ORDER BY or the IN and sort client side, or disable paging for this query");
+
             List<Row> page = pager.fetchPage(pageSize);
             ResultMessage.Rows msg = processResults(page, variables, limit, now);
             if (!pager.isExhausted())
@@ -931,56 +936,6 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
         return expressions;
     }
 
-    private Iterable<Column> columnsInOrder(final ColumnFamily cf, final List<ByteBuffer> variables) throws InvalidRequestException
-    {
-        if (columnRestrictions.length == 0)
-            return cf.getSortedColumns();
-
-        // If the restriction for the last column alias is an IN, respect
-        // requested order
-        Restriction last = columnRestrictions[columnRestrictions.length - 1];
-        if (last == null || last.isSlice())
-            return cf.getSortedColumns();
-
-        ColumnNameBuilder builder = cfDef.getColumnNameBuilder();
-        for (int i = 0; i < columnRestrictions.length - 1; i++)
-            builder.add(columnRestrictions[i].values(variables).get(0));
-
-
-        List<ByteBuffer> values = last.values(variables);
-        final List<ByteBuffer> requested = new ArrayList<ByteBuffer>(values.size());
-        Iterator<ByteBuffer> iter = values.iterator();
-        while (iter.hasNext())
-        {
-            ByteBuffer t = iter.next();
-            ColumnNameBuilder b = iter.hasNext() ? builder.copy() : builder;
-            requested.add(b.add(t).build());
-        }
-
-        return new Iterable<Column>()
-        {
-            public Iterator<Column> iterator()
-            {
-                return new AbstractIterator<Column>()
-                {
-                    // If the query is reversed, we'll reverse everything in the end, so return the
-                    // requested in reversed order so we do return values in requested order in the end
-                    Iterator<ByteBuffer> iter = (isReversed ? Lists.reverse(requested) : requested).iterator();
-                    public Column computeNext()
-                    {
-                        while (iter.hasNext())
-                        {
-                            Column column = cf.getColumn(iter.next());
-                            if (column != null)
-                                return column;
-                        }
-                        return endOfData();
-                    }
-                };
-            }
-        };
-    }
-
     private ResultSet process(List<Row> rows, List<ByteBuffer> variables, int limit, long now) throws InvalidRequestException
     {
         Selection.ResultSetBuilder result = selection.resultSetBuilder(now);
@@ -1027,7 +982,7 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
         else if (cfDef.isCompact)
         {
             // One cqlRow per column
-            for (Column c : columnsInOrder(cf, variables))
+            for (Column c : cf)
             {
                 if (c.isMarkedForDelete(now))
                     continue;
@@ -1144,16 +1099,18 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
         return true;
     }
 
+    private boolean needsPostQueryOrdering()
+    {
+        // We need post-query ordering only for queries with IN on the partition key and an ORDER BY.
+        return keyIsInRelation && !parameters.orderings.isEmpty();
+    }
+
     /**
      * Orders results when multiple keys are selected (using IN)
      */
     private void orderResults(ResultSet cqlRows)
     {
-        // There is nothing to do if
-        //   a. there are no results,
-        //   b. no ordering information where given,
-        //   c. key restriction is a Range or not an IN expression
-        if (cqlRows.size() == 0 || parameters.orderings.isEmpty() || isKeyRange || !keyIsInRelation)
+        if (cqlRows.size() == 0 || !needsPostQueryOrdering())
             return;
 
         assert orderingIndexes != null;