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/04/02 21:38:15 UTC

[1/2] git commit: Fix collection element access in IF

Repository: cassandra
Updated Branches:
  refs/heads/cassandra-2.1 8234bc15f -> 13510d414


Fix collection element access in IF

patch by slebresne; reviewed by thobbs for CASSANDRA-6914


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

Branch: refs/heads/cassandra-2.1
Commit: 5aafa98768d0e309c2b19f3a20bdf46e114b0c94
Parents: d4ec31f
Author: Sylvain Lebresne <sy...@datastax.com>
Authored: Tue Mar 25 08:48:38 2014 +0100
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Wed Apr 2 21:28:03 2014 +0200

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 .../apache/cassandra/cql3/ColumnCondition.java  | 126 +++++++++++++++++--
 src/java/org/apache/cassandra/cql3/Cql.g        |   4 +-
 .../cql3/statements/CQL3CasConditions.java      |   5 +-
 4 files changed, 123 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/5aafa987/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 9bbcf07..3a08c33 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -43,6 +43,7 @@
  * Fix error with super columns in mixed 1.2-2.0 clusters (CASSANDRA-6966)
  * Fix bad skip of sstables on slice query with composite start/finish (CASSANDRA-6825)
  * Fix unintended update with conditional statement (CASSANDRA-6893)
+ * Fix map element access in IF (CASSANDRA-6914)
 Merged from 1.2:
  * Add UNLOGGED, COUNTER options to BATCH documentation (CASSANDRA-6816)
  * add extra SSL cipher suites (CASSANDRA-6613)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/5aafa987/src/java/org/apache/cassandra/cql3/ColumnCondition.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/ColumnCondition.java b/src/java/org/apache/cassandra/cql3/ColumnCondition.java
index e6cdf43..9fb3390 100644
--- a/src/java/org/apache/cassandra/cql3/ColumnCondition.java
+++ b/src/java/org/apache/cassandra/cql3/ColumnCondition.java
@@ -28,6 +28,7 @@ import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.filter.ColumnSlice;
 import org.apache.cassandra.db.marshal.*;
 import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.utils.ByteBufferUtil;
 
 /**
  * A CQL3 condition.
@@ -35,18 +36,27 @@ import org.apache.cassandra.exceptions.InvalidRequestException;
 public class ColumnCondition
 {
     public final CFDefinition.Name column;
+
+    // For collection, when testing the equality of a specific element, null otherwise.
+    private final Term collectionElement;
+
     private final Term value;
 
-    private ColumnCondition(CFDefinition.Name column, Term value)
+    private ColumnCondition(CFDefinition.Name column, Term collectionElement, Term value)
     {
         this.column = column;
+        this.collectionElement = collectionElement;
         this.value = value;
     }
 
-    // The only ones we support so far
     public static ColumnCondition equal(CFDefinition.Name column, Term value)
     {
-        return new ColumnCondition(column, value);
+        return new ColumnCondition(column, null, value);
+    }
+
+    public static ColumnCondition equal(CFDefinition.Name column, Term collectionElement, Term value)
+    {
+        return new ColumnCondition(column, collectionElement, value);
     }
 
     /**
@@ -57,6 +67,8 @@ public class ColumnCondition
      */
     public void collectMarkerSpecification(VariableSpecifications boundNames)
     {
+        if (collectionElement != null)
+            collectionElement.collectMarkerSpecification(boundNames);
         value.collectMarkerSpecification(boundNames);
     }
 
@@ -74,12 +86,26 @@ public class ColumnCondition
             this.variables = variables;
         }
 
-        // Not overriding equals() because we need the variables to have been attached when this is
-        // called and so having a non standard method name might help avoid mistakes
         public boolean equalsTo(WithVariables other) throws InvalidRequestException
         {
-            return column.equals(other.column())
-                && value.bindAndGet(variables).equals(other.value().bindAndGet(other.variables));
+            if (!column().equals(other.column()))
+                return false;
+
+            if ((collectionElement() == null) != (other.collectionElement() == null))
+                return false;
+
+            if (collectionElement() != null)
+            {
+                assert column.type instanceof ListType || column.type instanceof MapType;
+                AbstractType<?> comparator = column.type instanceof ListType
+                                           ? Int32Type.instance
+                                           : ((MapType)column.type).keys;
+
+                if (comparator.compare(collectionElement().bindAndGet(variables), other.collectionElement().bindAndGet(variables)) != 0)
+                    return false;
+            }
+
+            return value().bindAndGet(variables).equals(other.value().bindAndGet(other.variables));
         }
 
         private CFDefinition.Name column()
@@ -87,11 +113,21 @@ public class ColumnCondition
             return column;
         }
 
+        private Term collectionElement()
+        {
+            return collectionElement;
+        }
+
         private Term value()
         {
             return value;
         }
 
+        public ByteBuffer getCollectionElementValue() throws InvalidRequestException
+        {
+            return collectionElement == null ? null : collectionElement.bindAndGet(variables);
+        }
+
         private ColumnNameBuilder copyOrUpdatePrefix(CFMetaData cfm, ColumnNameBuilder rowPrefix)
         {
             return column.kind == CFDefinition.Name.Kind.STATIC ? cfm.getStaticColumnNameBuilder() : rowPrefix.copy();
@@ -105,6 +141,7 @@ public class ColumnCondition
             if (column.type instanceof CollectionType)
                 return collectionAppliesTo((CollectionType)column.type, rowPrefix, current, now);
 
+            assert collectionElement == null;
             ColumnNameBuilder prefix = copyOrUpdatePrefix(current.metadata(), rowPrefix);
             ByteBuffer columnName = column.kind == CFDefinition.Name.Kind.VALUE_ALIAS
                                   ? prefix.build()
@@ -120,6 +157,17 @@ public class ColumnCondition
         private boolean collectionAppliesTo(CollectionType type, ColumnNameBuilder rowPrefix, ColumnFamily current, final long now) throws InvalidRequestException
         {
             ColumnNameBuilder collectionPrefix = copyOrUpdatePrefix(current.metadata(), rowPrefix).add(column.name.key);
+            Term.Terminal v = value.bind(variables);
+
+            // For map element access, we won't iterate over the collection, so deal with that first. In other case, we do.
+            if (collectionElement != null && type instanceof MapType)
+            {
+                ByteBuffer e = collectionElement.bindAndGet(variables);
+                if (e == null)
+                    throw new InvalidRequestException("Invalid null value for map access");
+                return mapElementAppliesTo((MapType)type, current, collectionPrefix, e, v.get(), now);
+            }
+
             // We are testing for collection equality, so we need to have the expected values *and* only those.
             ColumnSlice[] collectionSlice = new ColumnSlice[]{ new ColumnSlice(collectionPrefix.build(), collectionPrefix.buildAsEndOfRange()) };
             // Filter live columns, this makes things simpler afterwards
@@ -132,10 +180,19 @@ public class ColumnCondition
                 }
             });
 
-            Term.Terminal v = value.bind(variables);
             if (v == null)
                 return !iter.hasNext();
 
+            if (collectionElement != null)
+            {
+                assert type instanceof ListType;
+                ByteBuffer e = collectionElement.bindAndGet(variables);
+                if (e == null)
+                    throw new InvalidRequestException("Invalid null value for list access");
+
+                return listElementAppliesTo((ListType)type, iter, e, v.get());
+            }
+
             switch (type.kind)
             {
                 case LIST: return listAppliesTo((ListType)type, current.metadata(), iter, ((Lists.Value)v).elements);
@@ -160,6 +217,19 @@ public class ColumnCondition
             return !iter.hasNext();
         }
 
+        private boolean listElementAppliesTo(ListType type, Iterator<Column> iter, ByteBuffer element, ByteBuffer value) throws InvalidRequestException
+        {
+            int idx = ByteBufferUtil.toInt(element);
+            if (idx < 0)
+                throw new InvalidRequestException(String.format("Invalid negative list index %d", idx));
+
+            int adv = Iterators.advance(iter, idx);
+            if (adv != idx || !iter.hasNext())
+                throw new InvalidRequestException(String.format("List index %d out of bound, list has size %d", idx, adv));
+
+            return type.elements.compare(iter.next().value(), value) == 0;
+        }
+
         private boolean setAppliesTo(SetType type, CFMetaData cfm, Iterator<Column> iter, Set<ByteBuffer> elements)
         {
             Set<ByteBuffer> remaining = new TreeSet<>(type.elements);
@@ -191,15 +261,36 @@ public class ColumnCondition
             }
             return remaining.isEmpty();
         }
+
+        private boolean mapElementAppliesTo(MapType type, ColumnFamily current, ColumnNameBuilder collectionPrefix, ByteBuffer element, ByteBuffer value, long now)
+        {
+            ByteBuffer name = collectionPrefix.add(element).build();
+            Column c = current.getColumn(name);
+            return c != null && c.isLive(now) && type.values.compare(c.value(), value) == 0;
+        }
     }
 
     public static class Raw
     {
         private final Term.Raw value;
 
-        public Raw(Term.Raw value)
+        // Can be null, only used with the syntax "IF m[e] = ..." (in which case it's 'e')
+        private final Term.Raw collectionElement;
+
+        private Raw(Term.Raw value, Term.Raw collectionElement)
         {
             this.value = value;
+            this.collectionElement = collectionElement;
+        }
+
+        public static Raw simpleEqual(Term.Raw value)
+        {
+            return new Raw(value, null);
+        }
+
+        public static Raw collectionEqual(Term.Raw value, Term.Raw collectionElement)
+        {
+            return new Raw(value, collectionElement);
         }
 
         public ColumnCondition prepare(CFDefinition.Name receiver) throws InvalidRequestException
@@ -207,7 +298,22 @@ public class ColumnCondition
             if (receiver.type instanceof CounterColumnType)
                 throw new InvalidRequestException("Condtions on counters are not supported");
 
-            return ColumnCondition.equal(receiver, value.prepare(receiver));
+            if (collectionElement == null)
+                return ColumnCondition.equal(receiver, value.prepare(receiver));
+
+            if (!(receiver.type.isCollection()))
+                throw new InvalidRequestException(String.format("Invalid element access syntax for non-collection column %s", receiver.name));
+
+            switch (((CollectionType)receiver.type).kind)
+            {
+                case LIST:
+                    return ColumnCondition.equal(receiver, collectionElement.prepare(Lists.indexSpecOf(receiver)), value.prepare(Lists.valueSpecOf(receiver)));
+                case SET:
+                    throw new InvalidRequestException(String.format("Invalid element access syntax for set column %s", receiver.name));
+                case MAP:
+                    return ColumnCondition.equal(receiver, collectionElement.prepare(Maps.keySpecOf(receiver)), value.prepare(Maps.valueSpecOf(receiver)));
+            }
+            throw new AssertionError();
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/5aafa987/src/java/org/apache/cassandra/cql3/Cql.g
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/Cql.g b/src/java/org/apache/cassandra/cql3/Cql.g
index ed482d0..04f9f59 100644
--- a/src/java/org/apache/cassandra/cql3/Cql.g
+++ b/src/java/org/apache/cassandra/cql3/Cql.g
@@ -852,7 +852,9 @@ columnOperation[List<Pair<ColumnIdentifier, Operation.RawUpdate>> operations]
     ;
 
 columnCondition[List<Pair<ColumnIdentifier, ColumnCondition.Raw>> conditions]
-    : key=cident '=' t=term { conditions.add(Pair.create(key, new ColumnCondition.Raw(t))); } // Note: we'll reject duplicates later
+    // Note: we'll reject duplicates later
+    : key=cident '=' t=term { conditions.add(Pair.create(key, ColumnCondition.Raw.simpleEqual(t))); }
+    | key=cident '[' element=term ']' '=' t=term { conditions.add(Pair.create(key, ColumnCondition.Raw.collectionEqual(t, element))); } 
     ;
 
 properties[PropertyDefinitions props]

http://git-wip-us.apache.org/repos/asf/cassandra/blob/5aafa987/src/java/org/apache/cassandra/cql3/statements/CQL3CasConditions.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/CQL3CasConditions.java b/src/java/org/apache/cassandra/cql3/statements/CQL3CasConditions.java
index 7d3c0f7..9f67bc0 100644
--- a/src/java/org/apache/cassandra/cql3/statements/CQL3CasConditions.java
+++ b/src/java/org/apache/cassandra/cql3/statements/CQL3CasConditions.java
@@ -26,6 +26,7 @@ import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.filter.*;
 import org.apache.cassandra.exceptions.InvalidRequestException;
 import org.apache.cassandra.service.CASConditions;
+import org.apache.cassandra.utils.Pair;
 
 /**
  * Processed CAS conditions on potentially multiple rows of the same partition.
@@ -166,7 +167,7 @@ public class CQL3CasConditions implements CASConditions
 
     private static class ColumnsConditions extends RowCondition
     {
-        private final Map<ColumnIdentifier, ColumnCondition.WithVariables> conditions = new HashMap<>();
+        private final Map<Pair<ColumnIdentifier, ByteBuffer>, ColumnCondition.WithVariables> conditions = new HashMap<>();
 
         private ColumnsConditions(ColumnNameBuilder rowPrefix, long now)
         {
@@ -180,7 +181,7 @@ public class CQL3CasConditions implements CASConditions
                 // We will need the variables in appliesTo but with protocol batches, each condition in this object can have a
                 // different list of variables.
                 ColumnCondition.WithVariables current = condition.with(variables);
-                ColumnCondition.WithVariables previous = conditions.put(condition.column.name, current);
+                ColumnCondition.WithVariables previous = conditions.put(Pair.create(condition.column.name, current.getCollectionElementValue()), current);
                 // If 2 conditions are actually equal, let it slide
                 if (previous != null && !previous.equalsTo(current))
                     throw new InvalidRequestException("Duplicate and incompatible conditions for column " + condition.column.name);


[2/2] 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/ColumnCondition.java


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

Branch: refs/heads/cassandra-2.1
Commit: 13510d414fbb0da489ecd062054aa49d059ba79d
Parents: 8234bc1 5aafa98
Author: Sylvain Lebresne <sy...@datastax.com>
Authored: Wed Apr 2 21:38:03 2014 +0200
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Wed Apr 2 21:38:03 2014 +0200

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 .../apache/cassandra/cql3/ColumnCondition.java  | 126 +++++++++++++++++--
 src/java/org/apache/cassandra/cql3/Cql.g        |   4 +-
 .../cql3/statements/CQL3CasConditions.java      |   5 +-
 4 files changed, 123 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


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

http://git-wip-us.apache.org/repos/asf/cassandra/blob/13510d41/src/java/org/apache/cassandra/cql3/ColumnCondition.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/ColumnCondition.java
index 64c2032,9fb3390..67e7174
--- a/src/java/org/apache/cassandra/cql3/ColumnCondition.java
+++ b/src/java/org/apache/cassandra/cql3/ColumnCondition.java
@@@ -36,19 -35,28 +37,28 @@@ import org.apache.cassandra.utils.ByteB
   */
  public class ColumnCondition
  {
 -    public final CFDefinition.Name column;
 +    public final ColumnDefinition column;
+ 
+     // For collection, when testing the equality of a specific element, null otherwise.
+     private final Term collectionElement;
+ 
      private final Term value;
  
-     private ColumnCondition(ColumnDefinition column, Term value)
 -    private ColumnCondition(CFDefinition.Name column, Term collectionElement, Term value)
++    private ColumnCondition(ColumnDefinition column, Term collectionElement, Term value)
      {
          this.column = column;
+         this.collectionElement = collectionElement;
          this.value = value;
      }
  
-     // The only ones we support so far
 -    public static ColumnCondition equal(CFDefinition.Name column, Term value)
 +    public static ColumnCondition equal(ColumnDefinition column, Term value)
      {
-         return new ColumnCondition(column, value);
+         return new ColumnCondition(column, null, value);
+     }
+ 
 -    public static ColumnCondition equal(CFDefinition.Name column, Term collectionElement, Term value)
++    public static ColumnCondition equal(ColumnDefinition column, Term collectionElement, Term value)
+     {
+         return new ColumnCondition(column, collectionElement, value);
      }
  
      /**
@@@ -76,15 -86,29 +88,29 @@@
              this.variables = variables;
          }
  
-         // Not overriding equals() because we need the variables to have been attached when this is
-         // called and so having a non standard method name might help avoid mistakes
          public boolean equalsTo(WithVariables other) throws InvalidRequestException
          {
-             return column.equals(other.column())
-                 && value.bindAndGet(variables).equals(other.value().bindAndGet(other.variables));
+             if (!column().equals(other.column()))
+                 return false;
+ 
+             if ((collectionElement() == null) != (other.collectionElement() == null))
+                 return false;
+ 
+             if (collectionElement() != null)
+             {
+                 assert column.type instanceof ListType || column.type instanceof MapType;
+                 AbstractType<?> comparator = column.type instanceof ListType
+                                            ? Int32Type.instance
+                                            : ((MapType)column.type).keys;
+ 
+                 if (comparator.compare(collectionElement().bindAndGet(variables), other.collectionElement().bindAndGet(variables)) != 0)
+                     return false;
+             }
+ 
+             return value().bindAndGet(variables).equals(other.value().bindAndGet(other.variables));
          }
  
 -        private CFDefinition.Name column()
 +        private ColumnDefinition column()
          {
              return column;
          }
@@@ -94,6 -123,16 +125,11 @@@
              return value;
          }
  
+         public ByteBuffer getCollectionElementValue() throws InvalidRequestException
+         {
+             return collectionElement == null ? null : collectionElement.bindAndGet(variables);
+         }
+ 
 -        private ColumnNameBuilder copyOrUpdatePrefix(CFMetaData cfm, ColumnNameBuilder rowPrefix)
 -        {
 -            return column.kind == CFDefinition.Name.Kind.STATIC ? cfm.getStaticColumnNameBuilder() : rowPrefix.copy();
 -        }
 -
          /**
           * Validates whether this condition applies to {@code current}.
           */
@@@ -102,22 -141,39 +138,34 @@@
              if (column.type instanceof CollectionType)
                  return collectionAppliesTo((CollectionType)column.type, rowPrefix, current, now);
  
+             assert collectionElement == null;
 -            ColumnNameBuilder prefix = copyOrUpdatePrefix(current.metadata(), rowPrefix);
 -            ByteBuffer columnName = column.kind == CFDefinition.Name.Kind.VALUE_ALIAS
 -                                  ? prefix.build()
 -                                  : prefix.add(column.name.key).build();
 -
 -            Column c = current.getColumn(columnName);
 +            Cell c = current.getColumn(current.metadata().comparator.create(rowPrefix, column));
              ByteBuffer v = value.bindAndGet(variables);
              return v == null
                   ? c == null || !c.isLive(now)
 -                 : c != null && c.isLive(now) && column.type.compare(c.value(), v) == 0;
 +                 : c != null && c.isLive(now) && c.value().equals(v);
          }
  
 -        private boolean collectionAppliesTo(CollectionType type, ColumnNameBuilder rowPrefix, ColumnFamily current, final long now) throws InvalidRequestException
 +        private boolean collectionAppliesTo(CollectionType type, Composite rowPrefix, ColumnFamily current, final long now) throws InvalidRequestException
          {
 -            ColumnNameBuilder collectionPrefix = copyOrUpdatePrefix(current.metadata(), rowPrefix).add(column.name.key);
+             Term.Terminal v = value.bind(variables);
+ 
+             // For map element access, we won't iterate over the collection, so deal with that first. In other case, we do.
+             if (collectionElement != null && type instanceof MapType)
+             {
+                 ByteBuffer e = collectionElement.bindAndGet(variables);
+                 if (e == null)
+                     throw new InvalidRequestException("Invalid null value for map access");
 -                return mapElementAppliesTo((MapType)type, current, collectionPrefix, e, v.get(), now);
++                return mapElementAppliesTo((MapType)type, current, rowPrefix, e, v.get(), now);
+             }
+ 
 +            CellName name = current.metadata().comparator.create(rowPrefix, column);
              // We are testing for collection equality, so we need to have the expected values *and* only those.
 -            ColumnSlice[] collectionSlice = new ColumnSlice[]{ new ColumnSlice(collectionPrefix.build(), collectionPrefix.buildAsEndOfRange()) };
 +            ColumnSlice[] collectionSlice = new ColumnSlice[]{ name.slice() };
              // Filter live columns, this makes things simpler afterwards
 -            Iterator<Column> iter = Iterators.filter(current.iterator(collectionSlice), new Predicate<Column>()
 +            Iterator<Cell> iter = Iterators.filter(current.iterator(collectionSlice), new Predicate<Cell>()
              {
 -                public boolean apply(Column c)
 +                public boolean apply(Cell c)
                  {
                      // we only care about live columns
                      return c.isLive(now);
@@@ -127,12 -183,21 +175,21 @@@
              if (v == null)
                  return !iter.hasNext();
  
+             if (collectionElement != null)
+             {
+                 assert type instanceof ListType;
+                 ByteBuffer e = collectionElement.bindAndGet(variables);
+                 if (e == null)
+                     throw new InvalidRequestException("Invalid null value for list access");
+ 
+                 return listElementAppliesTo((ListType)type, iter, e, v.get());
+             }
+ 
              switch (type.kind)
              {
 -                case LIST: return listAppliesTo((ListType)type, current.metadata(), iter, ((Lists.Value)v).elements);
 -                case SET: return setAppliesTo((SetType)type, current.metadata(), iter, ((Sets.Value)v).elements);
 -                case MAP: return mapAppliesTo((MapType)type, current.metadata(), iter, ((Maps.Value)v).map);
 +                case LIST: return listAppliesTo((ListType)type, iter, ((Lists.Value)v).elements);
 +                case SET: return setAppliesTo((SetType)type, iter, ((Sets.Value)v).elements);
 +                case MAP: return mapAppliesTo((MapType)type, iter, ((Maps.Value)v).map);
              }
              throw new AssertionError();
          }
@@@ -146,7 -217,20 +203,20 @@@
              return !iter.hasNext();
          }
  
 -        private boolean listElementAppliesTo(ListType type, Iterator<Column> iter, ByteBuffer element, ByteBuffer value) throws InvalidRequestException
++        private boolean listElementAppliesTo(ListType type, Iterator<Cell> iter, ByteBuffer element, ByteBuffer value) throws InvalidRequestException
+         {
+             int idx = ByteBufferUtil.toInt(element);
+             if (idx < 0)
+                 throw new InvalidRequestException(String.format("Invalid negative list index %d", idx));
+ 
+             int adv = Iterators.advance(iter, idx);
+             if (adv != idx || !iter.hasNext())
+                 throw new InvalidRequestException(String.format("List index %d out of bound, list has size %d", idx, adv));
+ 
+             return type.elements.compare(iter.next().value(), value) == 0;
+         }
+ 
 -        private boolean setAppliesTo(SetType type, CFMetaData cfm, Iterator<Column> iter, Set<ByteBuffer> elements)
 +        private boolean setAppliesTo(SetType type, Iterator<Cell> iter, Set<ByteBuffer> elements)
          {
              Set<ByteBuffer> remaining = new TreeSet<>(type.elements);
              remaining.addAll(elements);
@@@ -177,23 -261,59 +247,59 @@@
              }
              return remaining.isEmpty();
          }
+ 
 -        private boolean mapElementAppliesTo(MapType type, ColumnFamily current, ColumnNameBuilder collectionPrefix, ByteBuffer element, ByteBuffer value, long now)
++        private boolean mapElementAppliesTo(MapType type, ColumnFamily current, Composite rowPrefix, ByteBuffer element, ByteBuffer value, long now)
+         {
 -            ByteBuffer name = collectionPrefix.add(element).build();
 -            Column c = current.getColumn(name);
++            CellName name = current.getComparator().create(rowPrefix, column, element);
++            Cell c = current.getColumn(name);
+             return c != null && c.isLive(now) && type.values.compare(c.value(), value) == 0;
+         }
      }
  
      public static class Raw
      {
          private final Term.Raw value;
  
-         public Raw(Term.Raw value)
+         // Can be null, only used with the syntax "IF m[e] = ..." (in which case it's 'e')
+         private final Term.Raw collectionElement;
+ 
+         private Raw(Term.Raw value, Term.Raw collectionElement)
          {
              this.value = value;
+             this.collectionElement = collectionElement;
+         }
+ 
+         public static Raw simpleEqual(Term.Raw value)
+         {
+             return new Raw(value, null);
+         }
+ 
+         public static Raw collectionEqual(Term.Raw value, Term.Raw collectionElement)
+         {
+             return new Raw(value, collectionElement);
          }
  
 -        public ColumnCondition prepare(CFDefinition.Name receiver) throws InvalidRequestException
 +        public ColumnCondition prepare(String keyspace, ColumnDefinition receiver) throws InvalidRequestException
          {
              if (receiver.type instanceof CounterColumnType)
                  throw new InvalidRequestException("Condtions on counters are not supported");
  
-             return ColumnCondition.equal(receiver, value.prepare(keyspace, receiver));
+             if (collectionElement == null)
 -                return ColumnCondition.equal(receiver, value.prepare(receiver));
++                return ColumnCondition.equal(receiver, value.prepare(keyspace, receiver));
+ 
+             if (!(receiver.type.isCollection()))
+                 throw new InvalidRequestException(String.format("Invalid element access syntax for non-collection column %s", receiver.name));
+ 
+             switch (((CollectionType)receiver.type).kind)
+             {
+                 case LIST:
 -                    return ColumnCondition.equal(receiver, collectionElement.prepare(Lists.indexSpecOf(receiver)), value.prepare(Lists.valueSpecOf(receiver)));
++                    return ColumnCondition.equal(receiver, collectionElement.prepare(keyspace, Lists.indexSpecOf(receiver)), value.prepare(keyspace, Lists.valueSpecOf(receiver)));
+                 case SET:
+                     throw new InvalidRequestException(String.format("Invalid element access syntax for set column %s", receiver.name));
+                 case MAP:
 -                    return ColumnCondition.equal(receiver, collectionElement.prepare(Maps.keySpecOf(receiver)), value.prepare(Maps.valueSpecOf(receiver)));
++                    return ColumnCondition.equal(receiver, collectionElement.prepare(keyspace, Maps.keySpecOf(receiver)), value.prepare(keyspace, Maps.valueSpecOf(receiver)));
+             }
+             throw new AssertionError();
          }
      }
  }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/13510d41/src/java/org/apache/cassandra/cql3/Cql.g
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/13510d41/src/java/org/apache/cassandra/cql3/statements/CQL3CasConditions.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/statements/CQL3CasConditions.java
index fb842a2,9f67bc0..4003edc
--- a/src/java/org/apache/cassandra/cql3/statements/CQL3CasConditions.java
+++ b/src/java/org/apache/cassandra/cql3/statements/CQL3CasConditions.java
@@@ -166,9 -167,9 +167,9 @@@ public class CQL3CasConditions implemen
  
      private static class ColumnsConditions extends RowCondition
      {
-         private final Map<ColumnIdentifier, ColumnCondition.WithVariables> conditions = new HashMap<>();
+         private final Map<Pair<ColumnIdentifier, ByteBuffer>, ColumnCondition.WithVariables> conditions = new HashMap<>();
  
 -        private ColumnsConditions(ColumnNameBuilder rowPrefix, long now)
 +        private ColumnsConditions(Composite rowPrefix, long now)
          {
              super(rowPrefix, now);
          }