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:10:00 UTC

[1/3] git commit: Disallow post-query re-ordering when paging

Repository: cassandra
Updated Branches:
  refs/heads/trunk 5d4ca6eb0 -> a896d0fe3


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/trunk
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;


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

Posted by sl...@apache.org.
Merge branch 'cassandra-2.1' into trunk


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

Branch: refs/heads/trunk
Commit: a896d0fe337b7b3b869dd31400bb7c08c9213a31
Parents: 5d4ca6e 143372b
Author: Sylvain Lebresne <sy...@datastax.com>
Authored: Fri Feb 21 14:09:52 2014 +0100
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Fri Feb 21 14:09:52 2014 +0100

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../cql3/statements/SelectStatement.java        | 79 ++++----------------
 2 files changed, 15 insertions(+), 65 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/a896d0fe/CHANGES.txt
----------------------------------------------------------------------


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

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

Conflicts:
	src/java/org/apache/cassandra/cql3/statements/SelectStatement.java


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

Branch: refs/heads/trunk
Commit: 143372bf7cba1ec25cfc08c5396ba0382b96b2c2
Parents: 3b4084b 4589656
Author: Sylvain Lebresne <sy...@datastax.com>
Authored: Fri Feb 21 14:09:26 2014 +0100
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Fri Feb 21 14:09:26 2014 +0100

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../cql3/statements/SelectStatement.java        | 79 ++++----------------
 2 files changed, 15 insertions(+), 65 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/143372bf/CHANGES.txt
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/143372bf/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
index 868d51c,9fbed03..dfeea84
--- a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
@@@ -78,9 -80,8 +78,8 @@@ public class SelectStatement implement
      private boolean isKeyRange;
      private boolean keyIsInRelation;
      private boolean usesSecondaryIndexing;
-     private boolean needOrderOnLastClustering;
  
 -    private Map<CFDefinition.Name, Integer> orderingIndexes;
 +    private Map<ColumnIdentifier, Integer> orderingIndexes;
  
      private boolean selectsStaticColumns;
      private boolean selectsOnlyStaticColumns;
@@@ -1114,82 -1108,92 +1124,32 @@@
      /**
       * Orders results when multiple keys are selected (using IN)
       */
 -    private void orderResults(ResultSet cqlRows)
 +    private void orderResults(ResultSet cqlRows, List<ByteBuffer> variables) throws InvalidRequestException
      {
-         if (cqlRows.size() == 0)
-             return;
- 
-         /*
-          * We need to do post-query ordering in 2 cases:
-          *   1) if the last clustering column is restricted by a IN and has no explicit ORDER BY on it.
-          *   2) if the partition key is restricted by a IN and there is some ORDER BY values
-          */
-         boolean needOrderOnPartitionKey = keyIsInRelation && !parameters.orderings.isEmpty();
-         if (!needOrderOnLastClustering && !needOrderOnPartitionKey)
+         if (cqlRows.size() == 0 || !needsPostQueryOrdering())
              return;
  
          assert orderingIndexes != null;
  
 -        // optimization when only *one* order condition was given
 -        // because there is no point of using composite comparator if there is only one order condition
 -        if (parameters.orderings.size() == 1)
 -        {
 -            CFDefinition.Name ordering = cfDef.get(parameters.orderings.keySet().iterator().next());
 -            Collections.sort(cqlRows.rows, new SingleColumnComparator(orderingIndexes.get(ordering), ordering.type));
 -            return;
 -        }
 +        List<Integer> idToSort = new ArrayList<Integer>();
 +        List<Comparator<ByteBuffer>> sorters = new ArrayList<Comparator<ByteBuffer>>();
  
-         // Note that we add the ORDER BY sorters first as they should prevail over ordering
-         // on the last clustering restriction.
 -        // builds a 'composite' type for multi-column comparison from the comparators of the ordering components
 -        // and passes collected position information and built composite comparator to CompositeComparator to do
 -        // an actual comparison of the CQL rows.
 -        List<AbstractType<?>> types = new ArrayList<AbstractType<?>>(parameters.orderings.size());
 -        int[] positions = new int[parameters.orderings.size()];
 -
 -        int idx = 0;
          for (ColumnIdentifier identifier : parameters.orderings.keySet())
          {
 -            CFDefinition.Name orderingColumn = cfDef.get(identifier);
 -            types.add(orderingColumn.type);
 -            positions[idx++] = orderingIndexes.get(orderingColumn);
 -        }
 -
 -        Collections.sort(cqlRows.rows, new CompositeComparator(types, positions));
 -    }
 -
 -    private void handleGroup(Selection selection,
 -                             Selection.ResultSetBuilder result,
 -                             ByteBuffer[] keyComponents,
 -                             ColumnGroupMap columns,
 -                             Map<CFDefinition.Name, ByteBuffer> staticValues) throws InvalidRequestException
 -    {
 -        // Respect requested order
 -        result.newRow();
 -        for (CFDefinition.Name name : selection.getColumnsList())
 -        {
 -            switch (name.kind)
 -            {
 -                case KEY_ALIAS:
 -                    result.add(keyComponents[name.position]);
 -                    break;
 -                case COLUMN_ALIAS:
 -                    result.add(columns.getKeyComponent(name.position));
 -                    break;
 -                case VALUE_ALIAS:
 -                    // This should not happen for SPARSE
 -                    throw new AssertionError();
 -                case COLUMN_METADATA:
 -                    if (name.type.isCollection())
 -                    {
 -                        result.add(getCollectionValue(name, columns));
 -                    }
 -                    else
 -                    {
 -                        result.add(columns.getSimple(name.name.key));
 -                    }
 -                    break;
 -                case STATIC:
 -                    result.add(staticValues.get(name));
 -                    break;
 -            }
 +            ColumnDefinition orderingColumn = cfm.getColumnDefinition(identifier);
 +            idToSort.add(orderingIndexes.get(orderingColumn.name));
 +            sorters.add(orderingColumn.type);
          }
 -    }
  
-         if (needOrderOnLastClustering)
-         {
-             List<ColumnDefinition> cc = cfm.clusteringColumns();
-             idToSort.add(orderingIndexes.get(cc.get(cc.size() - 1).name));
-             Restriction last = columnRestrictions[columnRestrictions.length - 1];
-             sorters.add(makeComparatorFor(last.values(variables), isReversed));
-         }
- 
 -    private static ByteBuffer getCollectionValue(CFDefinition.Name name, ColumnGroupMap columns)
 -    {
 -        List<Pair<ByteBuffer, Column>> collection = columns.getCollection(name.name.key);
 -        return collection == null ? null : ((CollectionType)name.type).serialize(collection);
 +        Comparator<List<ByteBuffer>> comparator = idToSort.size() == 1
 +                                                ? new SingleColumnComparator(idToSort.get(0), sorters.get(0))
 +                                                : new CompositeComparator(sorters, idToSort);
 +        Collections.sort(cqlRows.rows, comparator);
      }
  
-     // Comparator used when the last clustering key is an IN, to sort result
-     // rows in the order of the values provided for the IN.
-     private Comparator<ByteBuffer> makeComparatorFor(final List<ByteBuffer> vals, final boolean isReversed)
-     {
-         // This may not always be the most efficient, but it probably is if
-         // values is small, which is likely to be the most common case.
-         return new Comparator<ByteBuffer>()
-         {
-             private final List<ByteBuffer> values = isReversed ? com.google.common.collect.Lists.reverse(vals) : vals;
- 
-             public int compare(ByteBuffer b1, ByteBuffer b2)
-             {
-                 int idx1 = -1;
-                 int idx2 = -1;
-                 for (int i = 0; i < values.size(); i++)
-                 {
-                     ByteBuffer bb = values.get(i);
-                     if (bb.equals(b1))
-                         idx1 = i;
-                     if (bb.equals(b2))
-                         idx2 = i;
- 
-                     if (idx1 >= 0 && idx2 >= 0)
-                         break;
-                 }
-                 assert idx1 >= 0 && idx2 >= 0 : "Got CQL3 row that was not queried in resultset";
-                 return idx1 - idx2;
-             }
-         };
-     }
- 
 -    private static ByteBuffer getSimpleValue(CFDefinition.Name name, ColumnGroupMap columns)
 +    private static boolean isReversedType(ColumnDefinition def)
      {
 -        Column c = columns.getSimple(name.name.key);
 -        return c == null ? null : c.value();
 -    }
 -
 -    private static boolean isReversedType(CFDefinition.Name name)
 -    {
 -        return name.type instanceof ReversedType;
 +        return def.type instanceof ReversedType;
      }
  
      private boolean columnFilterIsIdentity()
@@@ -1441,22 -1448,12 +1401,12 @@@
                      // We only support IN for the last name and for compact storage so far
                      // TODO: #3885 allows us to extend to non compact as well, but that remains to be done
                      if (i != stmt.columnRestrictions.length - 1)
 -                        throw new InvalidRequestException(String.format("PRIMARY KEY part %s cannot be restricted by IN relation", cname));
 +                        throw new InvalidRequestException(String.format("PRIMARY KEY part %s cannot be restricted by IN relation", cdef.name));
                      else if (stmt.selectACollection())
 -                        throw new InvalidRequestException(String.format("Cannot restrict PRIMARY KEY part %s by IN relation as a collection is selected by the query", cname));
 +                        throw new InvalidRequestException(String.format("Cannot restrict PRIMARY KEY part %s by IN relation as a collection is selected by the query", cdef.name));
-                     // We will return rows in the order of the IN, unless that clustering column has a specific order set on.
-                     if (parameters.orderings.get(cdef.name) == null)
-                     {
-                         stmt.needOrderOnLastClustering = true;
-                         stmt.orderingIndexes = new HashMap<ColumnIdentifier, Integer>();
-                         int index = indexOf(cdef, stmt.selection);
-                         if (index < 0)
-                             index = stmt.selection.addColumnForOrdering(cdef);
-                         stmt.orderingIndexes.put(cdef.name, index);
-                     }
                  }
  
 -                previous = cname;
 +                previous = cdef;
              }
  
              // Covers indexes on the first clustering column (among others).
@@@ -1494,16 -1491,16 +1444,15 @@@
                  if (stmt.isKeyRange)
                      throw new InvalidRequestException("ORDER BY is only supported when the partition key is restricted by an EQ or an IN.");
  
 -                // If we order an IN query, we'll have to do a manual sort post-query. Currently, this sorting requires that we
 -                // have queried the column on which we sort (TODO: we should update it to add the column on which we sort to the one
 -                // queried automatically, and then removing it from the resultSet afterwards if needed)
 +                // If we order post-query (see orderResults), the sorted column needs to be in the ResultSet for sorting, even if we don't
 +                // ultimately ship them to the client (CASSANDRA-4911).
-                 if (stmt.keyIsInRelation || stmt.needOrderOnLastClustering)
+                 if (stmt.keyIsInRelation)
                  {
-                     if (stmt.orderingIndexes == null)
-                         stmt.orderingIndexes = new HashMap<ColumnIdentifier, Integer>();
 -                    stmt.orderingIndexes = new HashMap<CFDefinition.Name, Integer>();
++                    stmt.orderingIndexes = new HashMap<>();
                      for (ColumnIdentifier column : stmt.parameters.orderings.keySet())
                      {
 -                        final CFDefinition.Name name = cfDef.get(column);
 -                        if (name == null)
 +                        final ColumnDefinition def = cfm.getColumnDefinition(column);
 +                        if (def == null)
                          {
                              if (containsAlias(column))
                                  throw new InvalidRequestException(String.format("Aliases are not allowed in order by clause ('%s')", column));