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/14 13:53:07 UTC

[1/3] git commit: IN on the last clustering columns + ORDER BY DESC yield no results

Updated Branches:
  refs/heads/trunk 5b0eb01c9 -> c36656c06


IN on the last clustering columns + ORDER BY DESC yield no results

patch by slebresne; reviewed by thobbs for CASSANDRA-6701


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

Branch: refs/heads/trunk
Commit: c9ade9dc67f1193c1c417a8709b6e0f85568fc68
Parents: de72e7f
Author: Sylvain Lebresne <sy...@datastax.com>
Authored: Fri Feb 14 13:42:27 2014 +0100
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Fri Feb 14 13:42:27 2014 +0100

----------------------------------------------------------------------
 CHANGES.txt                                                   | 1 +
 .../org/apache/cassandra/cql3/statements/SelectStatement.java | 7 +++++--
 2 files changed, 6 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/c9ade9dc/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 872934a..492cc2d 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -7,6 +7,7 @@
  * Log USING TTL/TIMESTAMP in a counter update warning (CASSANDRA-6649)
  * Don't exchange schema between nodes with different versions (CASSANDRA-6695)
  * Use real node messaging versions for schema exchange decisions (CASSANDRA-6700)
+ * IN on the last clustering columns + ORDER BY DESC yield no results (CASSANDRA-6701)
 
 
 1.2.15

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c9ade9dc/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 001c77a..e058cff 100644
--- a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
@@ -22,6 +22,7 @@ import java.util.*;
 import java.util.concurrent.ExecutionException;
 
 import com.google.common.collect.AbstractIterator;
+import com.google.common.collect.Lists;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.github.jamm.MemoryMeter;
@@ -625,7 +626,7 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
                             throw new InvalidRequestException(String.format("Invalid null clustering key part %s", name));
                         ColumnNameBuilder copy = builder.copy().add(val);
                         // See below for why this
-                        s.add((bound == Bound.END && copy.remainingCount() > 0) ? copy.buildAsEndOfRange() : copy.build());
+                        s.add((b == Bound.END && copy.remainingCount() > 0) ? copy.buildAsEndOfRange() : copy.build());
                     }
                     return new ArrayList<ByteBuffer>(s);
                 }
@@ -727,7 +728,9 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
             {
                 return new AbstractIterator<IColumn>()
                 {
-                    Iterator<ByteBuffer> iter = requested.iterator();
+                    // 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 IColumn computeNext()
                     {
                         while (iter.hasNext())


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

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

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/c36656c0
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/c36656c0
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/c36656c0

Branch: refs/heads/trunk
Commit: c36656c06bc10bccace324d40adc7d038de2f74f
Parents: 5b0eb01 44f9c86
Author: Sylvain Lebresne <sy...@datastax.com>
Authored: Fri Feb 14 13:52:57 2014 +0100
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Fri Feb 14 13:52:57 2014 +0100

----------------------------------------------------------------------
 CHANGES.txt                                                  | 1 +
 .../apache/cassandra/cql3/statements/SelectStatement.java    | 8 +++++---
 2 files changed, 6 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/c36656c0/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 42c8bd9,57eefac..d08cb93
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -56,8 -21,8 +56,9 @@@ Merged from 1.2
   * Log USING TTL/TIMESTAMP in a counter update warning (CASSANDRA-6649)
   * Don't exchange schema between nodes with different versions (CASSANDRA-6695)
   * Use real node messaging versions for schema exchange decisions (CASSANDRA-6700)
+  * IN on the last clustering columns + ORDER BY DESC yield no results (CASSANDRA-6701)
  
 +
  2.0.5
   * Reduce garbage generated by bloom filter lookups (CASSANDRA-6609)
   * Add ks.cf names to tombstone logging (CASSANDRA-6597)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c36656c0/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
index 3bf5906,307e668..60d13f4
--- a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
@@@ -694,12 -706,12 +694,12 @@@ public class SelectStatement implement
                      for (ByteBuffer val : values)
                      {
                          if (val == null)
 -                            throw new InvalidRequestException(String.format("Invalid null clustering key part %s", name));
 -                        ColumnNameBuilder copy = builder.copy().add(val);
 +                            throw new InvalidRequestException(String.format("Invalid null clustering key part %s", def.name));
 +                        Composite prefix = builder.buildWith(val);
                          // See below for why this
-                         s.add((bound == Bound.END && builder.remainingCount() > 0) ? prefix.end() : prefix);
 -                        s.add((b == Bound.END && copy.remainingCount() > 0) ? copy.buildAsEndOfRange() : copy.build());
++                        s.add((b == Bound.END && builder.remainingCount() > 0) ? prefix.end() : prefix);
                      }
 -                    return new ArrayList<ByteBuffer>(s);
 +                    return new ArrayList<Composite>(s);
                  }
  
                  ByteBuffer val = values.get(0);
@@@ -947,64 -986,70 +947,66 @@@
  
          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)
 +        List<Integer> idToSort = new ArrayList<Integer>();
 +        List<Comparator<ByteBuffer>> sorters = new ArrayList<Comparator<ByteBuffer>>();
 +
 +        // If the restriction for the last clustering key is an IN, respect requested order
 +        if (lastClusteringIsIn)
          {
 -            CFDefinition.Name ordering = cfDef.get(parameters.orderings.keySet().iterator().next());
 -            Collections.sort(cqlRows.rows, new SingleColumnComparator(orderingIndexes.get(ordering), ordering.type));
 -            return;
 +            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)));
++            sorters.add(makeComparatorFor(last.values(variables), isReversed));
          }
  
 -        // 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;
 +        // Then add the order by
          for (ColumnIdentifier identifier : parameters.orderings.keySet())
          {
 -            CFDefinition.Name orderingColumn = cfDef.get(identifier);
 -            types.add(orderingColumn.type);
 -            positions[idx++] = orderingIndexes.get(orderingColumn);
 +            ColumnDefinition orderingColumn = cfm.getColumnDefinition(identifier);
 +            idToSort.add(orderingIndexes.get(orderingColumn.name));
 +            sorters.add(orderingColumn.type);
          }
  
 -        Collections.sort(cqlRows.rows, new CompositeComparator(types, positions));
 +        Comparator<List<ByteBuffer>> comparator = idToSort.size() == 1
 +                                                ? new SingleColumnComparator(idToSort.get(0), sorters.get(0))
 +                                                : new CompositeComparator(sorters, idToSort);
 +        Collections.sort(cqlRows.rows, comparator);
      }
  
 -    private void handleGroup(Selection selection, Selection.ResultSetBuilder result, ByteBuffer[] keyComponents, ColumnGroupMap columns) throws InvalidRequestException
 +    // 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> values)
++    private Comparator<ByteBuffer> makeComparatorFor(final List<ByteBuffer> vals, final boolean isReversed)
      {
 -        // Respect requested order
 -        result.newRow();
 -        for (CFDefinition.Name name : selection.getColumnsList())
 +        // 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>()
          {
 -            switch (name.kind)
++            private final List<ByteBuffer> values = isReversed ? com.google.common.collect.Lists.reverse(vals) : vals;
++
 +            public int compare(ByteBuffer b1, ByteBuffer b2)
              {
 -                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())
 -                    {
 -                        List<Pair<ByteBuffer, Column>> collection = columns.getCollection(name.name.key);
 -                        ByteBuffer value = collection == null
 -                                         ? null
 -                                         : ((CollectionType)name.type).serialize(collection);
 -                        result.add(value);
 -                    }
 -                    else
 -                    {
 -                        result.add(columns.getSimple(name.name.key));
 -                    }
 -                    break;
 +                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 boolean isReversedType(CFDefinition.Name name)
 +    private static boolean isReversedType(ColumnDefinition def)
      {
 -        return name.type instanceof ReversedType;
 +        return def.type instanceof ReversedType;
      }
  
      private boolean columnFilterIsIdentity()


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

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

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/44f9c868
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/44f9c868
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/44f9c868

Branch: refs/heads/trunk
Commit: 44f9c8689507b6aabeeac4943d4aff4634117b73
Parents: 78df8a3 c9ade9d
Author: Sylvain Lebresne <sy...@datastax.com>
Authored: Fri Feb 14 13:48:46 2014 +0100
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Fri Feb 14 13:48:46 2014 +0100

----------------------------------------------------------------------
 CHANGES.txt                                                   | 1 +
 .../org/apache/cassandra/cql3/statements/SelectStatement.java | 7 +++++--
 2 files changed, 6 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/44f9c868/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 7425625,492cc2d..57eefac
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -21,33 -7,24 +21,34 @@@ Merged from 1.2
   * Log USING TTL/TIMESTAMP in a counter update warning (CASSANDRA-6649)
   * Don't exchange schema between nodes with different versions (CASSANDRA-6695)
   * Use real node messaging versions for schema exchange decisions (CASSANDRA-6700)
+  * IN on the last clustering columns + ORDER BY DESC yield no results (CASSANDRA-6701)
  
 -
 -1.2.15
 - * Move handling of migration event source to solve bootstrap race (CASSANDRA-6648)
 - * Make sure compaction throughput value doesn't overflow with int math (CASSANDRA-6647)
 -
 -
 -1.2.14
 - * Reverted code to limit CQL prepared statement cache by size (CASSANDRA-6592)
 - * add cassandra.default_messaging_version property to allow easier
 -   upgrading from 1.1 (CASSANDRA-6619)
 - * Allow executing CREATE statements multiple times (CASSANDRA-6471)
 - * Don't send confusing info with timeouts (CASSANDRA-6491)
 - * Don't resubmit counter mutation runnables internally (CASSANDRA-6427)
 - * Don't drop local mutations without a hint (CASSANDRA-6510)
 - * Don't allow null max_hint_window_in_ms (CASSANDRA-6419)
 - * Validate SliceRange start and finish lengths (CASSANDRA-6521)
 +2.0.5
 + * Reduce garbage generated by bloom filter lookups (CASSANDRA-6609)
 + * Add ks.cf names to tombstone logging (CASSANDRA-6597)
 + * Use LOCAL_QUORUM for LWT operations at LOCAL_SERIAL (CASSANDRA-6495)
 + * Wait for gossip to settle before accepting client connections (CASSANDRA-4288)
 + * Delete unfinished compaction incrementally (CASSANDRA-6086)
 + * Allow specifying custom secondary index options in CQL3 (CASSANDRA-6480)
 + * Improve replica pinning for cache efficiency in DES (CASSANDRA-6485)
 + * Fix LOCAL_SERIAL from thrift (CASSANDRA-6584)
 + * Don't special case received counts in CAS timeout exceptions (CASSANDRA-6595)
 + * Add support for 2.1 global counter shards (CASSANDRA-6505)
 + * Fix NPE when streaming connection is not yet established (CASSANDRA-6210)
 + * Avoid rare duplicate read repair triggering (CASSANDRA-6606)
 + * Fix paging discardFirst (CASSANDRA-6555)
 + * Fix ArrayIndexOutOfBoundsException in 2ndary index query (CASSANDRA-6470)
 + * Release sstables upon rebuilding 2i (CASSANDRA-6635)
 + * Add AbstractCompactionStrategy.startup() method (CASSANDRA-6637)
 + * SSTableScanner may skip rows during cleanup (CASSANDRA-6638)
 + * sstables from stalled repair sessions can resurrect deleted data (CASSANDRA-6503)
 + * Switch stress to use ITransportFactory (CASSANDRA-6641)
 + * Fix IllegalArgumentException during prepare (CASSANDRA-6592)
 + * Fix possible loss of 2ndary index entries during compaction (CASSANDRA-6517)
 + * Fix direct Memory on architectures that do not support unaligned long access
 +   (CASSANDRA-6628)
 + * Let scrub optionally skip broken counter partitions (CASSANDRA-5930)
 +Merged from 1.2:
   * fsync compression metadata (CASSANDRA-6531)
   * Validate CF existence on execution for prepared statement (CASSANDRA-6535)
   * Add ability to throttle batchlog replay (CASSANDRA-6550)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/44f9c868/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
index 29cafa2,e058cff..307e668
--- a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
@@@ -19,11 -19,12 +19,12 @@@ package org.apache.cassandra.cql3.state
  
  import java.nio.ByteBuffer;
  import java.util.*;
 -import java.util.concurrent.ExecutionException;
  
 +import com.google.common.base.Objects;
 +import com.google.common.base.Predicate;
  import com.google.common.collect.AbstractIterator;
 +import com.google.common.collect.Iterables;
+ import com.google.common.collect.Lists;
 -import org.slf4j.Logger;
 -import org.slf4j.LoggerFactory;
  import org.github.jamm.MemoryMeter;
  
  import org.apache.cassandra.auth.Permission;
@@@ -807,27 -711,27 +808,29 @@@ public class SelectStatement implement
  
          ColumnNameBuilder builder = cfDef.getColumnNameBuilder();
          for (int i = 0; i < columnRestrictions.length - 1; i++)
 -            builder.add(columnRestrictions[i].eqValues.get(0).bindAndGet(variables));
 +            builder.add(columnRestrictions[i].values(variables).get(0));
 +
  
 -        final List<ByteBuffer> requested = new ArrayList<ByteBuffer>(last.eqValues.size());
 -        Iterator<Term> iter = last.eqValues.iterator();
 +        List<ByteBuffer> values = last.values(variables);
 +        final List<ByteBuffer> requested = new ArrayList<ByteBuffer>(values.size());
 +        Iterator<ByteBuffer> iter = values.iterator();
          while (iter.hasNext())
          {
 -            Term t = iter.next();
 +            ByteBuffer t = iter.next();
              ColumnNameBuilder b = iter.hasNext() ? builder.copy() : builder;
 -            requested.add(b.add(t.bindAndGet(variables)).build());
 +            requested.add(b.add(t).build());
          }
  
 -        return new Iterable<IColumn>()
 +        return new Iterable<Column>()
          {
 -            public Iterator<IColumn> iterator()
 +            public Iterator<Column> iterator()
              {
 -                return new AbstractIterator<IColumn>()
 +                return new AbstractIterator<Column>()
                  {
-                     Iterator<ByteBuffer> iter = requested.iterator();
+                     // 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 IColumn computeNext()
 +                    public Column computeNext()
                      {
                          while (iter.hasNext())
                          {