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 2016/02/11 10:29:19 UTC

[3/3] cassandra git commit: Fix SELECT on tuple relations for mixed ASC/DESC clustering order

Fix SELECT on tuple relations for mixed ASC/DESC clustering order

patch by Marcin Szymaniuk and Benjamin Lerer; reviewed by Sylvain Lebresne for CASSANDRA-7281


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

Branch: refs/heads/cassandra-2.2
Commit: 90fc8969a656b183e81496e479db35f321c8a3aa
Parents: 984e174
Author: Marcin Szymaniuk <ma...@gmail.com>
Authored: Wed Feb 10 18:29:54 2016 +0100
Committer: Benjamin Lerer <b....@gmail.com>
Committed: Thu Feb 11 10:28:45 2016 +0100

----------------------------------------------------------------------
 CHANGES.txt                                     |    1 +
 .../cql3/restrictions/AbstractRestriction.java  |   15 +-
 .../ForwardingPrimaryKeyRestrictions.java       |    3 +-
 .../restrictions/MultiColumnRestriction.java    |   80 +-
 .../restrictions/PrimaryKeyRestrictionSet.java  |   95 +-
 .../cql3/restrictions/Restriction.java          |    3 +-
 .../cql3/restrictions/RestrictionSet.java       |    4 +-
 .../restrictions/SingleColumnRestriction.java   |    9 +-
 .../cql3/restrictions/TokenRestriction.java     |    3 +-
 .../cql3/statements/ModificationStatement.java  |   19 +-
 .../db/composites/CompositesBuilder.java        |   20 +-
 .../PrimaryKeyRestrictionSetTest.java           | 1164 +++++++++++++++---
 .../SelectMultiColumnRelationTest.java          |  859 ++++++++++++-
 13 files changed, 2043 insertions(+), 232 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/90fc8969/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 7565386..5674b9d 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 2.2.6
+ * Fix SELECT on tuple relations for mixed ASC/DESC clustering order (CASSANDRA-7281)
  * (cqlsh) Support utf-8/cp65001 encoding on Windows (CASSANDRA-11030)
  * Fix paging on DISTINCT queries repeats result when first row in partition changes (CASSANDRA-10010)
 Merged from 2.1:

http://git-wip-us.apache.org/repos/asf/cassandra/blob/90fc8969/src/java/org/apache/cassandra/cql3/restrictions/AbstractRestriction.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/restrictions/AbstractRestriction.java b/src/java/org/apache/cassandra/cql3/restrictions/AbstractRestriction.java
index 64c94f4..dac7203 100644
--- a/src/java/org/apache/cassandra/cql3/restrictions/AbstractRestriction.java
+++ b/src/java/org/apache/cassandra/cql3/restrictions/AbstractRestriction.java
@@ -19,12 +19,13 @@ package org.apache.cassandra.cql3.restrictions;
 
 import java.nio.ByteBuffer;
 
+import org.apache.cassandra.config.ColumnDefinition;
+
 import org.apache.cassandra.cql3.ColumnSpecification;
 import org.apache.cassandra.cql3.QueryOptions;
 import org.apache.cassandra.cql3.statements.Bound;
 import org.apache.cassandra.db.composites.CompositesBuilder;
 import org.apache.cassandra.exceptions.InvalidRequestException;
-
 import static org.apache.cassandra.cql3.statements.RequestValidations.checkBindValueSet;
 import static org.apache.cassandra.cql3.statements.RequestValidations.checkFalse;
 import static org.apache.cassandra.cql3.statements.RequestValidations.checkNotNull;
@@ -97,4 +98,16 @@ abstract class AbstractRestriction  implements Restriction
         checkFalse(value.remaining() > 0xFFFF, "Index expression values may not be larger than 64K");
         return value;
     }
+
+    /**
+     * Reverses the specified bound if the column type is a reversed one.
+     *
+     * @param columnDefinition the column definition
+     * @param bound the bound
+     * @return the bound reversed if the column type was a reversed one or the original bound
+     */
+    protected static Bound reverseBoundIfNeeded(ColumnDefinition columnDefinition, Bound bound)
+    {
+        return columnDefinition.isReversedType() ? bound.reverse() : bound;
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/90fc8969/src/java/org/apache/cassandra/cql3/restrictions/ForwardingPrimaryKeyRestrictions.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/restrictions/ForwardingPrimaryKeyRestrictions.java b/src/java/org/apache/cassandra/cql3/restrictions/ForwardingPrimaryKeyRestrictions.java
index 03c6cbc..71855a0 100644
--- a/src/java/org/apache/cassandra/cql3/restrictions/ForwardingPrimaryKeyRestrictions.java
+++ b/src/java/org/apache/cassandra/cql3/restrictions/ForwardingPrimaryKeyRestrictions.java
@@ -18,7 +18,6 @@
 package org.apache.cassandra.cql3.restrictions;
 
 import java.nio.ByteBuffer;
-import java.util.Collection;
 import java.util.List;
 
 import org.apache.cassandra.config.ColumnDefinition;
@@ -51,7 +50,7 @@ abstract class ForwardingPrimaryKeyRestrictions implements PrimaryKeyRestriction
     }
 
     @Override
-    public Collection<ColumnDefinition> getColumnDefs()
+    public List<ColumnDefinition> getColumnDefs()
     {
         return getDelegate().getColumnDefs();
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/90fc8969/src/java/org/apache/cassandra/cql3/restrictions/MultiColumnRestriction.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/restrictions/MultiColumnRestriction.java b/src/java/org/apache/cassandra/cql3/restrictions/MultiColumnRestriction.java
index 84a3952..96e6f2b 100644
--- a/src/java/org/apache/cassandra/cql3/restrictions/MultiColumnRestriction.java
+++ b/src/java/org/apache/cassandra/cql3/restrictions/MultiColumnRestriction.java
@@ -20,11 +20,9 @@ package org.apache.cassandra.cql3.restrictions;
 import java.nio.ByteBuffer;
 import java.util.*;
 
-import org.apache.cassandra.cql3.Term.Terminal;
-
-import org.apache.cassandra.cql3.Term;
 import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.cql3.*;
+import org.apache.cassandra.cql3.Term.Terminal;
 import org.apache.cassandra.cql3.functions.Function;
 import org.apache.cassandra.cql3.statements.Bound;
 import org.apache.cassandra.db.IndexExpression;
@@ -68,7 +66,7 @@ public abstract class MultiColumnRestriction extends AbstractRestriction
     }
 
     @Override
-    public Collection<ColumnDefinition> getColumnDefs()
+    public List<ColumnDefinition> getColumnDefs()
     {
         return columnDefs;
     }
@@ -361,14 +359,62 @@ public abstract class MultiColumnRestriction extends AbstractRestriction
         @Override
         public CompositesBuilder appendBoundTo(CompositesBuilder builder, Bound bound, QueryOptions options)
         {
-            List<ByteBuffer> vals = componentBounds(bound, options);
+            boolean reversed = getFirstColumn().isReversedType();
+
+            EnumMap<Bound, List<ByteBuffer>> componentBounds = new EnumMap<Bound, List<ByteBuffer>>(Bound.class);
+            componentBounds.put(Bound.START, componentBounds(Bound.START, options));
+            componentBounds.put(Bound.END, componentBounds(Bound.END, options));
 
-            for (int i = 0, m = vals.size(); i < m; i++)
+            List<List<ByteBuffer>> toAdd = new ArrayList<>();
+            List<ByteBuffer> values = new ArrayList<>();
+
+            for (int i = 0, m = columnDefs.size(); i < m; i++)
             {
-                ByteBuffer v = checkNotNull(vals.get(i), "Invalid null value in condition for column %s", columnDefs.get(i).name);
-                builder.addElementToAll(v);
+                ColumnDefinition column = columnDefs.get(i);
+                Bound b = reverseBoundIfNeeded(column, bound);
+
+                // For mixed order columns, we need to create additional slices when 2 columns are in reverse order
+                if (reversed != column.isReversedType())
+                {
+                    reversed = column.isReversedType();
+                    // As we are switching direction we need to add the current composite
+                    toAdd.add(values);
+
+                    // The new bound side has no value for this component.  just stop
+                    if (!hasComponent(b, i, componentBounds))
+                        continue;
+
+                    // The other side has still some components. We need to end the slice that we have just open.
+                    if (hasComponent(b.reverse(), i, componentBounds))
+                        toAdd.add(values);
+
+                    // We need to rebuild where we are in this bound side
+                    values = new ArrayList<ByteBuffer>();
+
+                    List<ByteBuffer> vals = componentBounds.get(b);
+
+                    int n = Math.min(i, vals.size());
+                    for (int j = 0; j < n; j++)
+                    {
+                        ByteBuffer v = checkNotNull(vals.get(j),
+                                                    "Invalid null value in condition for column %s",
+                                                    columnDefs.get(j).name);
+                        values.add(v);
+                    }
+                }
+
+                if (!hasComponent(b, i, componentBounds))
+                    continue;
+
+                ByteBuffer v = checkNotNull(componentBounds.get(b).get(i), "Invalid null value in condition for column %s", columnDefs.get(i).name);
+                values.add(v);
             }
-            return builder;
+            toAdd.add(values);
+
+            if (bound.isEnd())
+                Collections.reverse(toAdd);
+
+            return builder.addAllElementsToAll(toAdd);
         }
 
         @Override
@@ -378,9 +424,9 @@ public abstract class MultiColumnRestriction extends AbstractRestriction
         }
 
         @Override
-        public boolean hasBound(Bound b)
+        public boolean hasBound(Bound bound)
         {
-            return slice.hasBound(b);
+            return slice.hasBound(bound);
         }
 
         @Override
@@ -390,9 +436,9 @@ public abstract class MultiColumnRestriction extends AbstractRestriction
         }
 
         @Override
-        public boolean isInclusive(Bound b)
+        public boolean isInclusive(Bound bound)
         {
-            return slice.isInclusive(b);
+            return slice.isInclusive(bound);
         }
 
         @Override
@@ -448,6 +494,9 @@ public abstract class MultiColumnRestriction extends AbstractRestriction
          */
         private List<ByteBuffer> componentBounds(Bound b, QueryOptions options) throws InvalidRequestException
         {
+            if (!slice.hasBound(b))
+                return Collections.emptyList();
+
             Terminal terminal = slice.bound(b).bind(options);
 
             if (terminal instanceof Tuples.Value)
@@ -457,5 +506,10 @@ public abstract class MultiColumnRestriction extends AbstractRestriction
 
             return Collections.singletonList(terminal.get(options.getProtocolVersion()));
         }
+
+        private boolean hasComponent(Bound b, int index, EnumMap<Bound, List<ByteBuffer>> componentBounds)
+        {
+            return componentBounds.get(b).size() > index;
+        }
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/90fc8969/src/java/org/apache/cassandra/cql3/restrictions/PrimaryKeyRestrictionSet.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/restrictions/PrimaryKeyRestrictionSet.java b/src/java/org/apache/cassandra/cql3/restrictions/PrimaryKeyRestrictionSet.java
index 3a20f6a..936dbd6 100644
--- a/src/java/org/apache/cassandra/cql3/restrictions/PrimaryKeyRestrictionSet.java
+++ b/src/java/org/apache/cassandra/cql3/restrictions/PrimaryKeyRestrictionSet.java
@@ -18,10 +18,9 @@
 package org.apache.cassandra.cql3.restrictions;
 
 import java.nio.ByteBuffer;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.List;
+import java.util.*;
 
+import org.apache.cassandra.db.composites.Composite;
 import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.cql3.QueryOptions;
 import org.apache.cassandra.cql3.functions.Function;
@@ -31,7 +30,6 @@ import org.apache.cassandra.db.composites.*;
 import org.apache.cassandra.db.composites.Composite.EOC;
 import org.apache.cassandra.db.index.SecondaryIndexManager;
 import org.apache.cassandra.exceptions.InvalidRequestException;
-
 import static org.apache.cassandra.cql3.statements.RequestValidations.checkFalse;
 import static org.apache.cassandra.cql3.statements.RequestValidations.invalidRequest;
 
@@ -163,7 +161,7 @@ final class PrimaryKeyRestrictionSet extends AbstractPrimaryKeyRestrictions
     @Override
     public List<Composite> valuesAsComposites(QueryOptions options) throws InvalidRequestException
     {
-        return appendTo(new CompositesBuilder(ctype), options).build();
+        return filterAndSort(appendTo(new CompositesBuilder(ctype), options).build());
     }
 
     @Override
@@ -197,29 +195,16 @@ final class PrimaryKeyRestrictionSet extends AbstractPrimaryKeyRestrictions
         {
             ColumnDefinition def = r.getFirstColumn();
 
-            // In a restriction, we always have Bound.START < Bound.END for the "base" comparator.
-            // So if we're doing a reverse slice, we must inverse the bounds when giving them as start and end of the slice filter.
-            // But if the actual comparator itself is reversed, we must inversed the bounds too.
-            Bound b = !def.isReversedType() ? bound : bound.reverse();
             if (keyPosition != def.position() || r.isContains())
                 break;
 
             if (r.isSlice())
             {
-                if (!r.hasBound(b))
-                {
-                    // There wasn't any non EQ relation on that key, we select all records having the preceding component as prefix.
-                    // For composites, if there was preceding component and we're computing the end, we must change the last component
-                    // End-Of-Component, otherwise we would be selecting only one record.
-                    return builder.buildWithEOC(bound.isEnd() ? EOC.END : EOC.START);
-                }
-
-                r.appendBoundTo(builder, b, options);
-                Composite.EOC eoc = eocFor(r, bound, b);
-                return builder.buildWithEOC(eoc);
+                r.appendBoundTo(builder, bound, options);
+                return filterAndSort(setEocs(r, bound, builder.build()));
             }
 
-            r.appendBoundTo(builder, b, options);
+            r.appendBoundTo(builder, bound, options);
 
             if (builder.hasMissingElements())
                 return Collections.emptyList();
@@ -233,7 +218,71 @@ final class PrimaryKeyRestrictionSet extends AbstractPrimaryKeyRestrictions
         // case using the eoc would be bad, since for the random partitioner we have no guarantee that
         // prefix.end() will sort after prefix (see #5240).
         EOC eoc = !builder.hasRemaining() ? EOC.NONE : (bound.isEnd() ? EOC.END : EOC.START);
-        return builder.buildWithEOC(eoc);
+        return filterAndSort(builder.buildWithEOC(eoc));
+    }
+
+    /**
+     * Removes duplicates and sort the specified composites.
+     *
+     * @param composites the composites to filter and sort
+     * @return the composites sorted and without duplicates
+     */
+    private List<Composite> filterAndSort(List<Composite> composites)
+    {
+        if (composites.size() <= 1)
+            return composites;
+
+        TreeSet<Composite> set = new TreeSet<Composite>(ctype);
+        set.addAll(composites);
+        return new ArrayList<>(set);
+    }
+
+    /**
+     * Sets EOCs for the composites returned by the specified slice restriction for the given bound.
+     *
+     * @param r the slice restriction
+     * @param bound the bound
+     * @param composites the composites
+     * @return the composites with their EOCs properly set
+     */
+    private List<Composite> setEocs(Restriction r, Bound bound, List<Composite> composites)
+    {
+        List<Composite> list = new ArrayList<>(composites.size());
+
+        // The first column of the slice might not be the first clustering column (e.g. clustering_0 = ? AND (clustering_1, clustering_2) >= (?, ?)
+        int offset = r.getFirstColumn().position();
+
+        for (int i = 0, m = composites.size(); i < m; i++)
+        {
+            Composite composite = composites.get(i);
+
+            // Handle the no bound case
+            if (composite.size() == offset)
+            {
+                list.add(composite.withEOC(bound.isEnd() ? EOC.END : EOC.START));
+                continue;
+            }
+
+            // In the case of mixed order columns, we will have some extra slices where the columns change directions.
+            // For example: if we have clustering_0 DESC and clustering_1 ASC a slice like (clustering_0, clustering_1) > (1, 2)
+            // will produce 2 slices: [EMPTY, 1.START] and [1.2.END, 1.END]
+            // So, the END bound will return 2 composite with the same values 1
+            if (composite.size() <= r.getLastColumn().position() && i < m - 1 && composite.equals(composites.get(i + 1)))
+            {
+                list.add(composite.withEOC(EOC.START));
+                list.add(composites.get(i++).withEOC(EOC.END));
+                continue;
+            }
+
+            // Handle the normal bounds
+            ColumnDefinition column = r.getColumnDefs().get(composite.size() - 1 - offset);
+            Bound b = reverseBoundIfNeeded(column, bound);
+
+            Composite.EOC eoc = eocFor(r, bound, b);
+            list.add(composite.withEOC(eoc));
+        }
+
+        return list;
     }
 
     @Override
@@ -307,7 +356,7 @@ final class PrimaryKeyRestrictionSet extends AbstractPrimaryKeyRestrictions
     }
 
     @Override
-    public Collection<ColumnDefinition> getColumnDefs()
+    public List<ColumnDefinition> getColumnDefs()
     {
         return restrictions.getColumnDefs();
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/90fc8969/src/java/org/apache/cassandra/cql3/restrictions/Restriction.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/restrictions/Restriction.java b/src/java/org/apache/cassandra/cql3/restrictions/Restriction.java
index c115a3b..f0ea6a7 100644
--- a/src/java/org/apache/cassandra/cql3/restrictions/Restriction.java
+++ b/src/java/org/apache/cassandra/cql3/restrictions/Restriction.java
@@ -17,7 +17,6 @@
  */
 package org.apache.cassandra.cql3.restrictions;
 
-import java.util.Collection;
 import java.util.List;
 
 import org.apache.cassandra.config.ColumnDefinition;
@@ -60,7 +59,7 @@ public interface Restriction
      * Returns the column definitions in position order.
      * @return the column definitions in position order.
      */
-    public Collection<ColumnDefinition> getColumnDefs();
+    public List<ColumnDefinition> getColumnDefs();
 
     /**
      * Return an Iterable over all of the functions (both native and user-defined) used by any component

http://git-wip-us.apache.org/repos/asf/cassandra/blob/90fc8969/src/java/org/apache/cassandra/cql3/restrictions/RestrictionSet.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/restrictions/RestrictionSet.java b/src/java/org/apache/cassandra/cql3/restrictions/RestrictionSet.java
index 6bf7666..676ed13 100644
--- a/src/java/org/apache/cassandra/cql3/restrictions/RestrictionSet.java
+++ b/src/java/org/apache/cassandra/cql3/restrictions/RestrictionSet.java
@@ -75,9 +75,9 @@ final class RestrictionSet implements Restrictions, Iterable<Restriction>
     }
 
     @Override
-    public final Set<ColumnDefinition> getColumnDefs()
+    public final List<ColumnDefinition> getColumnDefs()
     {
-        return restrictions.keySet();
+        return new ArrayList<>(restrictions.keySet());
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/cassandra/blob/90fc8969/src/java/org/apache/cassandra/cql3/restrictions/SingleColumnRestriction.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/restrictions/SingleColumnRestriction.java b/src/java/org/apache/cassandra/cql3/restrictions/SingleColumnRestriction.java
index afa0419..735a2e2 100644
--- a/src/java/org/apache/cassandra/cql3/restrictions/SingleColumnRestriction.java
+++ b/src/java/org/apache/cassandra/cql3/restrictions/SingleColumnRestriction.java
@@ -52,7 +52,7 @@ public abstract class SingleColumnRestriction extends AbstractRestriction
     }
 
     @Override
-    public Collection<ColumnDefinition> getColumnDefs()
+    public List<ColumnDefinition> getColumnDefs()
     {
         return Collections.singletonList(columnDef);
     }
@@ -351,7 +351,12 @@ public abstract class SingleColumnRestriction extends AbstractRestriction
         @Override
         public CompositesBuilder appendBoundTo(CompositesBuilder builder, Bound bound, QueryOptions options)
         {
-            ByteBuffer value = slice.bound(bound).bindAndGet(options);
+            Bound b = reverseBoundIfNeeded(getFirstColumn(), bound);
+
+            if (!hasBound(b))
+                return builder;
+
+            ByteBuffer value = slice.bound(b).bindAndGet(options);
             checkBindValueSet(value, "Invalid unset value for column %s", columnDef.name);
             return builder.addElementToAll(value);
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/90fc8969/src/java/org/apache/cassandra/cql3/restrictions/TokenRestriction.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/restrictions/TokenRestriction.java b/src/java/org/apache/cassandra/cql3/restrictions/TokenRestriction.java
index f8cd0dc..3cd3304 100644
--- a/src/java/org/apache/cassandra/cql3/restrictions/TokenRestriction.java
+++ b/src/java/org/apache/cassandra/cql3/restrictions/TokenRestriction.java
@@ -18,7 +18,6 @@
 package org.apache.cassandra.cql3.restrictions;
 
 import java.nio.ByteBuffer;
-import java.util.Collection;
 import java.util.Collections;
 import java.util.List;
 
@@ -67,7 +66,7 @@ public abstract class TokenRestriction extends AbstractPrimaryKeyRestrictions
     }
 
     @Override
-    public Collection<ColumnDefinition> getColumnDefs()
+    public List<ColumnDefinition> getColumnDefs()
     {
         return columnDefs;
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/90fc8969/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java b/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
index 8b594dd..fbdfc0c 100644
--- a/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
@@ -48,7 +48,6 @@ import org.apache.cassandra.transport.messages.ResultMessage;
 import org.apache.cassandra.triggers.TriggerExecutor;
 import org.apache.cassandra.utils.Pair;
 import org.apache.cassandra.utils.UUIDGen;
-
 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;
@@ -310,7 +309,7 @@ public abstract class ModificationStatement implements CQLStatement
             r.appendTo(keyBuilder, options);
         }
 
-        return Lists.transform(keyBuilder.build(), new com.google.common.base.Function<Composite, ByteBuffer>()
+        return Lists.transform(filterAndSort(keyBuilder.build()), new com.google.common.base.Function<Composite, ByteBuffer>()
         {
             @Override
             public ByteBuffer apply(Composite composite)
@@ -404,6 +403,22 @@ public abstract class ModificationStatement implements CQLStatement
         return builder.build().get(0); // We only allow IN for row keys so far
     }
 
+    /**
+     * Removes duplicates and sort the specified composites.
+     *
+     * @param composites the composites to filter and sort
+     * @return the composites sorted and without duplicates
+     */
+    private List<Composite> filterAndSort(List<Composite> composites)
+    {
+        if (composites.size() <= 1)
+            return composites;
+
+        TreeSet<Composite> set = new TreeSet<Composite>(cfm.getKeyValidatorAsCType());
+        set.addAll(composites);
+        return new ArrayList<>(set);
+    }
+
     protected ColumnDefinition getFirstEmptyKey()
     {
         for (ColumnDefinition def : cfm.clusteringColumns())

http://git-wip-us.apache.org/repos/asf/cassandra/blob/90fc8969/src/java/org/apache/cassandra/db/composites/CompositesBuilder.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/composites/CompositesBuilder.java b/src/java/org/apache/cassandra/db/composites/CompositesBuilder.java
index 25a510f..9a4da9e 100644
--- a/src/java/org/apache/cassandra/db/composites/CompositesBuilder.java
+++ b/src/java/org/apache/cassandra/db/composites/CompositesBuilder.java
@@ -181,9 +181,6 @@ public final class CompositesBuilder
 
                     List<ByteBuffer> value = values.get(j);
 
-                    if (value.isEmpty())
-                        hasMissingElements = true;
-
                     if (value.contains(null))
                         containsNull = true;
                     if (value.contains(ByteBufferUtil.UNSET_BYTE_BUFFER))
@@ -283,26 +280,15 @@ public final class CompositesBuilder
         if (elementsList.isEmpty())
             return singletonList(builder.build().withEOC(eoc));
 
-        // Use a Set to sort if needed and eliminate duplicates
-        Set<Composite> set = newSet();
+        List<Composite> list = new ArrayList<>();
 
         for (int i = 0, m = elementsList.size(); i < m; i++)
         {
             List<ByteBuffer> elements = elementsList.get(i);
-            set.add(builder.buildWith(elements).withEOC(eoc));
+            list.add(builder.buildWith(elements).withEOC(eoc));
         }
 
-        return new ArrayList<>(set);
-    }
-
-    /**
-     * Returns a new <code>Set</code> instance that will be used to eliminate duplicates and sort the results.
-     *
-     * @return a new <code>Set</code> instance.
-     */
-    private Set<Composite> newSet()
-    {
-        return new TreeSet<>(ctype);
+        return list;
     }
 
     private void checkUpdateable()