You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by al...@apache.org on 2014/06/18 02:43:09 UTC

[1/4] git commit: Don't insert tombstones that hide indexed values into 2i

Repository: cassandra
Updated Branches:
  refs/heads/trunk 805a4aeeb -> 704469d95


Don't insert tombstones that hide indexed values into 2i

patch by Sam Tunnicliffe; reviewed by Aleksey Yeschenko for
CASSANDRA-7268


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

Branch: refs/heads/trunk
Commit: febf3854bfa507c092ad5d35e3fe2d536ca78ce1
Parents: 593bba9
Author: Sam Tunnicliffe <sa...@beobal.com>
Authored: Tue Jun 17 16:25:29 2014 -0700
Committer: Aleksey Yeschenko <al...@apache.org>
Committed: Tue Jun 17 16:26:43 2014 -0700

----------------------------------------------------------------------
 CHANGES.txt                                     |  4 +-
 .../db/index/SecondaryIndexManager.java         | 26 ++++++++-
 .../cassandra/db/ColumnFamilyStoreTest.java     | 55 ++++++++++++++++++++
 3 files changed, 82 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/febf3854/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 28b5f29..18929d5 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,6 +1,8 @@
 1.2.17
+ * Don't insert tombstones that hide indexed values into 2i (CASSANDRA-7268)
  * Track metrics at a keyspace level (CASSANDRA-6539)
- * Add replace_address_first_boot flag to only replace if not bootstrapped (CASSANDRA-7356)
+ * Add replace_address_first_boot flag to only replace if not bootstrapped
+   (CASSANDRA-7356)
  * Enable keepalive for native protocol (CASSANDRA-7380)
  * Check internal addresses for seeds (CASSANDRA-6523)
  * Fix potential / by 0 in HHOM page size calculation (CASSANDRA-7354)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/febf3854/src/java/org/apache/cassandra/db/index/SecondaryIndexManager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/index/SecondaryIndexManager.java b/src/java/org/apache/cassandra/db/index/SecondaryIndexManager.java
index c4e4129..7fefa13 100644
--- a/src/java/org/apache/cassandra/db/index/SecondaryIndexManager.java
+++ b/src/java/org/apache/cassandra/db/index/SecondaryIndexManager.java
@@ -29,7 +29,6 @@ import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.compaction.CompactionManager;
 import org.apache.cassandra.db.filter.IDiskAtomFilter;
-import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.dht.AbstractBounds;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.io.sstable.ReducingKeyIterator;
@@ -650,7 +649,14 @@ public class SecondaryIndexManager
                 // where the row is invisible to both queries (the opposite seems preferable); see CASSANDRA-5540
                 if (!column.isMarkedForDelete())
                     ((PerColumnSecondaryIndex) index).insert(key.key, column);
-                ((PerColumnSecondaryIndex) index).delete(key.key, oldColumn);
+
+                // Usually we want to delete the old value from the index, except when
+                // name/value/timestamp are all equal, but the columns themselves
+                // are not (as is the case when overwriting expiring columns with
+                // identical values and ttl) Then, we don't want to delete as the
+                // tombstone will hide the new value we just inserted; see CASSANDRA-7268
+                if (shouldCleanupOldValue(oldColumn, column))
+                    ((PerColumnSecondaryIndex) index).delete(key.key, oldColumn);
             }
         }
 
@@ -672,5 +678,21 @@ public class SecondaryIndexManager
             for (SecondaryIndex index : rowLevelIndexMap.values())
                 ((PerRowSecondaryIndex) index).index(key.key, cf);
         }
+
+        private boolean shouldCleanupOldValue(IColumn oldColumn, IColumn newColumn)
+        {
+            // If any one of name/value/timestamp are different, then we
+            // should delete from the index. If not, then we can infer that
+            // at least one of the columns is an ExpiringColumn and that the
+            // difference is in the expiry time. In this case, we don't want to
+            // delete the old value from the index as the tombstone we insert
+            // will just hide the inserted value.
+            // Completely identical columns (including expiring columns with
+            // identical ttl & localExpirationTime) will not get this far due
+            // to the oldColumn.equals(newColumn) in StandardUpdater.update
+            return !oldColumn.name().equals(newColumn.name())
+                || !oldColumn.value().equals(newColumn.value())
+                || oldColumn.timestamp() != newColumn.timestamp();
+        }
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/febf3854/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java b/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java
index cd30297..e5354ed 100644
--- a/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java
+++ b/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java
@@ -25,6 +25,7 @@ import java.nio.charset.CharacterCodingException;
 import java.util.*;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
 
 import com.google.common.base.Function;
 import com.google.common.collect.Iterables;
@@ -418,6 +419,60 @@ public class ColumnFamilyStoreTest extends SchemaLoader
     }
 
     @Test
+    public void testIndexUpdateOverwritingExpiringColumns() throws Exception
+    {
+        // see CASSANDRA-7268
+        Table table = Table.open("Keyspace2");
+
+        // create a row and update the birthdate value with an expiring column
+        RowMutation rm;
+        rm = new RowMutation("Keyspace2", ByteBufferUtil.bytes("k100"));
+        rm.add(new QueryPath("Indexed1", null, ByteBufferUtil.bytes("birthdate")), ByteBufferUtil.bytes(100L), 1, 1000);
+        rm.apply();
+
+        IndexExpression expr = new IndexExpression(ByteBufferUtil.bytes("birthdate"), IndexOperator.EQ, ByteBufferUtil.bytes(100L));
+        List<IndexExpression> clause = Arrays.asList(expr);
+        IDiskAtomFilter filter = new IdentityQueryFilter();
+        Range<RowPosition> range = Util.range("", "");
+        List<Row> rows = table.getColumnFamilyStore("Indexed1").search(clause, range, 100, filter);
+        assertEquals(1,rows.size());
+
+        // requires a 1s sleep because we calculate local expiry time as (now() / 1000) + ttl
+        TimeUnit.SECONDS.sleep(1);
+
+        // now overwrite with the same name/value/ttl, but the local expiry time will be different
+        rm = new RowMutation("Keyspace2", ByteBufferUtil.bytes("k100"));
+        rm.add(new QueryPath("Indexed1", null, ByteBufferUtil.bytes("birthdate")), ByteBufferUtil.bytes(100L), 1, 1000);
+        rm.apply();
+
+        rows = table.getColumnFamilyStore("Indexed1").search(clause, range, 100, filter);
+        assertEquals(1,rows.size());
+
+        // check that modifying the indexed value using the same timestamp behaves as expected
+        rm = new RowMutation("Keyspace2", ByteBufferUtil.bytes("k101"));
+        rm.add(new QueryPath("Indexed1", null, ByteBufferUtil.bytes("birthdate")), ByteBufferUtil.bytes(101L), 1, 1000);
+        rm.apply();
+
+        expr = new IndexExpression(ByteBufferUtil.bytes("birthdate"), IndexOperator.EQ, ByteBufferUtil.bytes(101L));
+        clause = Arrays.asList(expr);
+        rows = table.getColumnFamilyStore("Indexed1").search(clause, range, 100, filter);
+        assertEquals(1,rows.size());
+
+        TimeUnit.SECONDS.sleep(1);
+        rm = new RowMutation("Keyspace2", ByteBufferUtil.bytes("k101"));
+        rm.add(new QueryPath("Indexed1", null, ByteBufferUtil.bytes("birthdate")), ByteBufferUtil.bytes(102L), 1, 1000);
+        rm.apply();
+        // search for the old value
+        rows = table.getColumnFamilyStore("Indexed1").search(clause, range, 100, filter);
+        assertEquals(0,rows.size());
+        // and for the new
+        expr = new IndexExpression(ByteBufferUtil.bytes("birthdate"), IndexOperator.EQ, ByteBufferUtil.bytes(102L));
+        clause = Arrays.asList(expr);
+        rows = table.getColumnFamilyStore("Indexed1").search(clause, range, 100, filter);
+        assertEquals(1,rows.size());
+    }
+
+    @Test
     public void testDeleteOfInconsistentValuesInKeysIndex() throws Exception
     {
         String keySpace = "Keyspace2";


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

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

Conflicts:
	CHANGES.txt
	src/java/org/apache/cassandra/db/index/SecondaryIndexManager.java


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

Branch: refs/heads/trunk
Commit: 7801aab8c0945005735d81f64e248c85d057a4ec
Parents: cee1f67 febf385
Author: Aleksey Yeschenko <al...@apache.org>
Authored: Tue Jun 17 17:15:31 2014 -0700
Committer: Aleksey Yeschenko <al...@apache.org>
Committed: Tue Jun 17 17:15:31 2014 -0700

----------------------------------------------------------------------
 CHANGES.txt                                     |  4 +-
 .../db/index/SecondaryIndexManager.java         | 25 ++++++++-
 .../cassandra/db/ColumnFamilyStoreTest.java     | 55 ++++++++++++++++++++
 3 files changed, 82 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/7801aab8/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index e3bd64b,18929d5..6a50186
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,23 -1,8 +1,25 @@@
 -1.2.17
 +2.0.9
 + * Fix native protocol CAS batches (CASSANDRA-7337)
 + * Add per-CF range read request latency metrics (CASSANDRA-7338)
 + * Fix NPE in StreamTransferTask.createMessageForRetry() (CASSANDRA-7323)
 + * Add conditional CREATE/DROP USER support (CASSANDRA-7264)
 + * Swap local and global default read repair chances (CASSANDRA-7320)
 + * Add missing iso8601 patterns for date strings (CASSANDRA-6973)
 + * Support selecting multiple rows in a partition using IN (CASSANDRA-6875)
 + * cqlsh: always emphasize the partition key in DESC output (CASSANDRA-7274)
 + * Copy compaction options to make sure they are reloaded (CASSANDRA-7290)
 + * Add option to do more aggressive tombstone compactions (CASSANDRA-6563)
 + * Don't try to compact already-compacting files in HHOM (CASSANDRA-7288)
 + * Add authentication support to shuffle (CASSANDRA-6484)
 + * Cqlsh counts non-empty lines for "Blank lines" warning (CASSANDRA-7325)
 + * Make StreamSession#closeSession() idempotent (CASSANDRA-7262)
 + * Fix infinite loop on exception while streaming (CASSANDRA-7330)
 + * Reference sstables before populating key cache (CASSANDRA-7234)
 +Merged from 1.2:
+  * Don't insert tombstones that hide indexed values into 2i (CASSANDRA-7268)
   * Track metrics at a keyspace level (CASSANDRA-6539)
-  * Add replace_address_first_boot flag to only replace if not bootstrapped (CASSANDRA-7356)
+  * Add replace_address_first_boot flag to only replace if not bootstrapped
+    (CASSANDRA-7356)
   * Enable keepalive for native protocol (CASSANDRA-7380)
   * Check internal addresses for seeds (CASSANDRA-6523)
   * Fix potential / by 0 in HHOM page size calculation (CASSANDRA-7354)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7801aab8/src/java/org/apache/cassandra/db/index/SecondaryIndexManager.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/index/SecondaryIndexManager.java
index 9600099,7fefa13..2c0d611
--- a/src/java/org/apache/cassandra/db/index/SecondaryIndexManager.java
+++ b/src/java/org/apache/cassandra/db/index/SecondaryIndexManager.java
@@@ -623,17 -638,25 +623,24 @@@ public class SecondaryIndexManage
          {
              if (oldColumn.equals(column))
                  return;
 -
 -            SecondaryIndex index = indexFor(column.name());
 -            if (index == null)
 -                return;
 -
 -            if (index instanceof PerColumnSecondaryIndex)
 +            
 +            for (SecondaryIndex index : indexFor(column.name()))
              {
 -                // insert the new value before removing the old one, so we never have a period
 -                // where the row is invisible to both queries (the opposite seems preferable); see CASSANDRA-5540
 -                if (!column.isMarkedForDelete())
 -                    ((PerColumnSecondaryIndex) index).insert(key.key, column);
 -
 -                // Usually we want to delete the old value from the index, except when
 -                // name/value/timestamp are all equal, but the columns themselves
 -                // are not (as is the case when overwriting expiring columns with
 -                // identical values and ttl) Then, we don't want to delete as the
 -                // tombstone will hide the new value we just inserted; see CASSANDRA-7268
 -                if (shouldCleanupOldValue(oldColumn, column))
 -                    ((PerColumnSecondaryIndex) index).delete(key.key, oldColumn);
 +                if (index instanceof PerColumnSecondaryIndex)
 +                {
 +                    // insert the new value before removing the old one, so we never have a period
 +                    // where the row is invisible to both queries (the opposite seems preferable); see CASSANDRA-5540
 +                    if (!column.isMarkedForDelete(System.currentTimeMillis()))
 +                        ((PerColumnSecondaryIndex) index).insert(key.key, column);
-                     ((PerColumnSecondaryIndex) index).delete(key.key, oldColumn);
++
++                    // Usually we want to delete the old value from the index, except when
++                    // name/value/timestamp are all equal, but the columns themselves
++                    // are not (as is the case when overwriting expiring columns with
++                    // identical values and ttl) Then, we don't want to delete as the
++                    // tombstone will hide the new value we just inserted; see CASSANDRA-7268
++                    if (shouldCleanupOldValue(oldColumn, column))
++                        ((PerColumnSecondaryIndex) index).delete(key.key, oldColumn);
 +                }
              }
          }
  
@@@ -652,5 -678,21 +659,21 @@@
              for (SecondaryIndex index : rowLevelIndexMap.values())
                  ((PerRowSecondaryIndex) index).index(key.key, cf);
          }
+ 
 -        private boolean shouldCleanupOldValue(IColumn oldColumn, IColumn newColumn)
++        private boolean shouldCleanupOldValue(Column oldColumn, Column newColumn)
+         {
+             // If any one of name/value/timestamp are different, then we
+             // should delete from the index. If not, then we can infer that
+             // at least one of the columns is an ExpiringColumn and that the
+             // difference is in the expiry time. In this case, we don't want to
+             // delete the old value from the index as the tombstone we insert
+             // will just hide the inserted value.
+             // Completely identical columns (including expiring columns with
+             // identical ttl & localExpirationTime) will not get this far due
+             // to the oldColumn.equals(newColumn) in StandardUpdater.update
+             return !oldColumn.name().equals(newColumn.name())
+                 || !oldColumn.value().equals(newColumn.value())
+                 || oldColumn.timestamp() != newColumn.timestamp();
+         }
      }
  }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7801aab8/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java
index bc39ad6,e5354ed..611d6af
--- a/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java
+++ b/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java
@@@ -424,6 -419,60 +425,60 @@@ public class ColumnFamilyStoreTest exte
      }
  
      @Test
+     public void testIndexUpdateOverwritingExpiringColumns() throws Exception
+     {
+         // see CASSANDRA-7268
 -        Table table = Table.open("Keyspace2");
++        Keyspace keyspace = Keyspace.open("Keyspace2");
+ 
+         // create a row and update the birthdate value with an expiring column
+         RowMutation rm;
+         rm = new RowMutation("Keyspace2", ByteBufferUtil.bytes("k100"));
 -        rm.add(new QueryPath("Indexed1", null, ByteBufferUtil.bytes("birthdate")), ByteBufferUtil.bytes(100L), 1, 1000);
++        rm.add("Indexed1", ByteBufferUtil.bytes("birthdate"), ByteBufferUtil.bytes(100L), 1, 1000);
+         rm.apply();
+ 
+         IndexExpression expr = new IndexExpression(ByteBufferUtil.bytes("birthdate"), IndexOperator.EQ, ByteBufferUtil.bytes(100L));
+         List<IndexExpression> clause = Arrays.asList(expr);
+         IDiskAtomFilter filter = new IdentityQueryFilter();
+         Range<RowPosition> range = Util.range("", "");
 -        List<Row> rows = table.getColumnFamilyStore("Indexed1").search(clause, range, 100, filter);
 -        assertEquals(1,rows.size());
++        List<Row> rows = keyspace.getColumnFamilyStore("Indexed1").search(range, clause, filter, 100);
++        assertEquals(1, rows.size());
+ 
+         // requires a 1s sleep because we calculate local expiry time as (now() / 1000) + ttl
+         TimeUnit.SECONDS.sleep(1);
+ 
+         // now overwrite with the same name/value/ttl, but the local expiry time will be different
+         rm = new RowMutation("Keyspace2", ByteBufferUtil.bytes("k100"));
 -        rm.add(new QueryPath("Indexed1", null, ByteBufferUtil.bytes("birthdate")), ByteBufferUtil.bytes(100L), 1, 1000);
++        rm.add("Indexed1", ByteBufferUtil.bytes("birthdate"), ByteBufferUtil.bytes(100L), 1, 1000);
+         rm.apply();
+ 
 -        rows = table.getColumnFamilyStore("Indexed1").search(clause, range, 100, filter);
 -        assertEquals(1,rows.size());
++        rows = keyspace.getColumnFamilyStore("Indexed1").search(range, clause, filter, 100);
++        assertEquals(1, rows.size());
+ 
+         // check that modifying the indexed value using the same timestamp behaves as expected
+         rm = new RowMutation("Keyspace2", ByteBufferUtil.bytes("k101"));
 -        rm.add(new QueryPath("Indexed1", null, ByteBufferUtil.bytes("birthdate")), ByteBufferUtil.bytes(101L), 1, 1000);
++        rm.add("Indexed1", ByteBufferUtil.bytes("birthdate"), ByteBufferUtil.bytes(101L), 1, 1000);
+         rm.apply();
+ 
+         expr = new IndexExpression(ByteBufferUtil.bytes("birthdate"), IndexOperator.EQ, ByteBufferUtil.bytes(101L));
+         clause = Arrays.asList(expr);
 -        rows = table.getColumnFamilyStore("Indexed1").search(clause, range, 100, filter);
 -        assertEquals(1,rows.size());
++        rows = keyspace.getColumnFamilyStore("Indexed1").search(range, clause, filter, 100);
++        assertEquals(1, rows.size());
+ 
+         TimeUnit.SECONDS.sleep(1);
+         rm = new RowMutation("Keyspace2", ByteBufferUtil.bytes("k101"));
 -        rm.add(new QueryPath("Indexed1", null, ByteBufferUtil.bytes("birthdate")), ByteBufferUtil.bytes(102L), 1, 1000);
++        rm.add("Indexed1", ByteBufferUtil.bytes("birthdate"), ByteBufferUtil.bytes(102L), 1, 1000);
+         rm.apply();
+         // search for the old value
 -        rows = table.getColumnFamilyStore("Indexed1").search(clause, range, 100, filter);
 -        assertEquals(0,rows.size());
++        rows = keyspace.getColumnFamilyStore("Indexed1").search(range, clause, filter, 100);
++        assertEquals(0, rows.size());
+         // and for the new
+         expr = new IndexExpression(ByteBufferUtil.bytes("birthdate"), IndexOperator.EQ, ByteBufferUtil.bytes(102L));
+         clause = Arrays.asList(expr);
 -        rows = table.getColumnFamilyStore("Indexed1").search(clause, range, 100, filter);
 -        assertEquals(1,rows.size());
++        rows = keyspace.getColumnFamilyStore("Indexed1").search(range, clause, filter, 100);
++        assertEquals(1, rows.size());
+     }
+ 
+     @Test
      public void testDeleteOfInconsistentValuesInKeysIndex() throws Exception
      {
          String keySpace = "Keyspace2";


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

Posted by al...@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/704469d9
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/704469d9
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/704469d9

Branch: refs/heads/trunk
Commit: 704469d95f3e2842eb174d887330eea682774e4e
Parents: 805a4ae 5e90091
Author: Aleksey Yeschenko <al...@apache.org>
Authored: Tue Jun 17 17:42:59 2014 -0700
Committer: Aleksey Yeschenko <al...@apache.org>
Committed: Tue Jun 17 17:42:59 2014 -0700

----------------------------------------------------------------------
 CHANGES.txt                                     |  4 +-
 .../db/index/SecondaryIndexManager.java         | 28 +++++++++-
 .../cassandra/db/ColumnFamilyStoreTest.java     | 55 ++++++++++++++++++++
 3 files changed, 85 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


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


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

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

Conflicts:
	src/java/org/apache/cassandra/db/index/SecondaryIndexManager.java


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

Branch: refs/heads/trunk
Commit: 5e90091d1ef940bcba65a87bf14fc1e809503627
Parents: a3dc7b8 7801aab
Author: Aleksey Yeschenko <al...@apache.org>
Authored: Tue Jun 17 17:42:32 2014 -0700
Committer: Aleksey Yeschenko <al...@apache.org>
Committed: Tue Jun 17 17:42:32 2014 -0700

----------------------------------------------------------------------
 CHANGES.txt                                     |  4 +-
 .../db/index/SecondaryIndexManager.java         | 28 +++++++++-
 .../cassandra/db/ColumnFamilyStoreTest.java     | 55 ++++++++++++++++++++
 3 files changed, 85 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/5e90091d/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index fd7c62b,6a50186..4c9e69d
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,26 -1,25 +1,28 @@@
 -2.0.9
 +2.1.0
 + * Avoid incremental compaction on Windows (CASSANDRA-7365)
 + * Fix exception when querying a composite-keyed table with a collection index
 +   (CASSANDRA-7372)
 + * Use node's host id in place of counter ids (CASSANDRA-7366)
   * Fix native protocol CAS batches (CASSANDRA-7337)
 + * Reduce likelihood of contention on local paxos locking (CASSANDRA-7359)
 + * Upgrade to Pig 0.12.1 (CASSANDRA-6556)
 + * Make sure we clear out repair sessions from netstats (CASSANDRA-7329)
 + * Don't fail streams on failure detector downs (CASSANDRA-3569)
 + * Add optional keyspace to DROP INDEX statement (CASSANDRA-7314)
 + * Reduce run time for CQL tests (CASSANDRA-7327)
 + * Fix heap size calculation on Windows (CASSANDRA-7352)
 + * RefCount native frames from netty (CASSANDRA-7245)
 + * Use tarball dir instead of /var for default paths (CASSANDRA-7136)
 +Merged from 2.0:
   * Add per-CF range read request latency metrics (CASSANDRA-7338)
   * Fix NPE in StreamTransferTask.createMessageForRetry() (CASSANDRA-7323)
 - * Add conditional CREATE/DROP USER support (CASSANDRA-7264)
 - * Swap local and global default read repair chances (CASSANDRA-7320)
 - * Add missing iso8601 patterns for date strings (CASSANDRA-6973)
 - * Support selecting multiple rows in a partition using IN (CASSANDRA-6875)
 - * cqlsh: always emphasize the partition key in DESC output (CASSANDRA-7274)
 - * Copy compaction options to make sure they are reloaded (CASSANDRA-7290)
 - * Add option to do more aggressive tombstone compactions (CASSANDRA-6563)
 - * Don't try to compact already-compacting files in HHOM (CASSANDRA-7288)
 - * Add authentication support to shuffle (CASSANDRA-6484)
 - * Cqlsh counts non-empty lines for "Blank lines" warning (CASSANDRA-7325)
   * Make StreamSession#closeSession() idempotent (CASSANDRA-7262)
   * Fix infinite loop on exception while streaming (CASSANDRA-7330)
 - * Reference sstables before populating key cache (CASSANDRA-7234)
  Merged from 1.2:
+  * Don't insert tombstones that hide indexed values into 2i (CASSANDRA-7268)
   * Track metrics at a keyspace level (CASSANDRA-6539)
-  * Add replace_address_first_boot flag to only replace if not bootstrapped (CASSANDRA-7356)
+  * Add replace_address_first_boot flag to only replace if not bootstrapped
+    (CASSANDRA-7356)
   * Enable keepalive for native protocol (CASSANDRA-7380)
   * Check internal addresses for seeds (CASSANDRA-6523)
   * Fix potential / by 0 in HHOM page size calculation (CASSANDRA-7354)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/5e90091d/src/java/org/apache/cassandra/db/index/SecondaryIndexManager.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/index/SecondaryIndexManager.java
index 36c7e1e,2c0d611..f78dc86
--- a/src/java/org/apache/cassandra/db/index/SecondaryIndexManager.java
+++ b/src/java/org/apache/cassandra/db/index/SecondaryIndexManager.java
@@@ -710,10 -628,18 +710,20 @@@ public class SecondaryIndexManage
              {
                  if (index instanceof PerColumnSecondaryIndex)
                  {
 -                    // insert the new value before removing the old one, so we never have a period
 -                    // where the row is invisible to both queries (the opposite seems preferable); see CASSANDRA-5540
 -                    if (!column.isMarkedForDelete(System.currentTimeMillis()))
 -                        ((PerColumnSecondaryIndex) index).insert(key.key, column);
 -
 -                    // Usually we want to delete the old value from the index, except when
 -                    // name/value/timestamp are all equal, but the columns themselves
 -                    // are not (as is the case when overwriting expiring columns with
 -                    // identical values and ttl) Then, we don't want to delete as the
 -                    // tombstone will hide the new value we just inserted; see CASSANDRA-7268
 -                    if (shouldCleanupOldValue(oldColumn, column))
 -                        ((PerColumnSecondaryIndex) index).delete(key.key, oldColumn);
 +                    if (cell.isLive())
++                    {
 +                        ((PerColumnSecondaryIndex) index).update(key.getKey(), oldCell, cell, opGroup);
++                    }
 +                    else
-                         ((PerColumnSecondaryIndex) index).delete(key.getKey(), oldCell, opGroup);
++                    {
++                        // Usually we want to delete the old value from the index, except when
++                        // name/value/timestamp are all equal, but the columns themselves
++                        // are not (as is the case when overwriting expiring columns with
++                        // identical values and ttl) Then, we don't want to delete as the
++                        // tombstone will hide the new value we just inserted; see CASSANDRA-7268
++                        if (shouldCleanupOldValue(oldCell, cell))
++                            ((PerColumnSecondaryIndex) index).delete(key.getKey(), oldCell, opGroup);
++                    }
                  }
              }
          }
@@@ -731,7 -657,23 +741,23 @@@
          public void updateRowLevelIndexes()
          {
              for (SecondaryIndex index : rowLevelIndexMap.values())
 -                ((PerRowSecondaryIndex) index).index(key.key, cf);
 +                ((PerRowSecondaryIndex) index).index(key.getKey(), cf);
          }
+ 
 -        private boolean shouldCleanupOldValue(Column oldColumn, Column newColumn)
++        private boolean shouldCleanupOldValue(Cell oldCell, Cell newCell)
+         {
+             // If any one of name/value/timestamp are different, then we
+             // should delete from the index. If not, then we can infer that
 -            // at least one of the columns is an ExpiringColumn and that the
++            // at least one of the cells is an ExpiringColumn and that the
+             // difference is in the expiry time. In this case, we don't want to
+             // delete the old value from the index as the tombstone we insert
+             // will just hide the inserted value.
 -            // Completely identical columns (including expiring columns with
++            // Completely identical cells (including expiring columns with
+             // identical ttl & localExpirationTime) will not get this far due
 -            // to the oldColumn.equals(newColumn) in StandardUpdater.update
 -            return !oldColumn.name().equals(newColumn.name())
 -                || !oldColumn.value().equals(newColumn.value())
 -                || oldColumn.timestamp() != newColumn.timestamp();
++            // to the oldCell.equals(newColumn) in StandardUpdater.update
++            return !oldCell.name().equals(newCell.name())
++                || !oldCell.value().equals(newCell.value())
++                || oldCell.timestamp() != newCell.timestamp();
+         }
      }
  }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/5e90091d/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java
index b178e48,611d6af..dda9b65
--- a/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java
+++ b/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java
@@@ -22,22 -22,10 +22,23 @@@ import java.io.File
  import java.io.IOException;
  import java.nio.ByteBuffer;
  import java.nio.charset.CharacterCodingException;
 -import java.util.*;
 +import java.util.Arrays;
 +import java.util.Collection;
 +import java.util.Collections;
 +import java.util.HashMap;
 +import java.util.HashSet;
 +import java.util.Iterator;
 +import java.util.LinkedList;
 +import java.util.List;
 +import java.util.Map;
 +import java.util.Random;
 +import java.util.Set;
 +import java.util.SortedSet;
 +import java.util.TreeSet;
 +import java.util.UUID;
  import java.util.concurrent.ExecutionException;
  import java.util.concurrent.Future;
+ import java.util.concurrent.TimeUnit;
  
  import com.google.common.base.Function;
  import com.google.common.collect.Iterables;
@@@ -453,6 -425,60 +454,60 @@@ public class ColumnFamilyStoreTest exte
      }
  
      @Test
+     public void testIndexUpdateOverwritingExpiringColumns() throws Exception
+     {
+         // see CASSANDRA-7268
+         Keyspace keyspace = Keyspace.open("Keyspace2");
+ 
+         // create a row and update the birthdate value with an expiring column
 -        RowMutation rm;
 -        rm = new RowMutation("Keyspace2", ByteBufferUtil.bytes("k100"));
 -        rm.add("Indexed1", ByteBufferUtil.bytes("birthdate"), ByteBufferUtil.bytes(100L), 1, 1000);
++        Mutation rm;
++        rm = new Mutation("Keyspace2", ByteBufferUtil.bytes("k100"));
++        rm.add("Indexed1", cellname("birthdate"), ByteBufferUtil.bytes(100L), 1, 1000);
+         rm.apply();
+ 
 -        IndexExpression expr = new IndexExpression(ByteBufferUtil.bytes("birthdate"), IndexOperator.EQ, ByteBufferUtil.bytes(100L));
++        IndexExpression expr = new IndexExpression(ByteBufferUtil.bytes("birthdate"), IndexExpression.Operator.EQ, ByteBufferUtil.bytes(100L));
+         List<IndexExpression> clause = Arrays.asList(expr);
+         IDiskAtomFilter filter = new IdentityQueryFilter();
+         Range<RowPosition> range = Util.range("", "");
+         List<Row> rows = keyspace.getColumnFamilyStore("Indexed1").search(range, clause, filter, 100);
+         assertEquals(1, rows.size());
+ 
+         // requires a 1s sleep because we calculate local expiry time as (now() / 1000) + ttl
+         TimeUnit.SECONDS.sleep(1);
+ 
+         // now overwrite with the same name/value/ttl, but the local expiry time will be different
 -        rm = new RowMutation("Keyspace2", ByteBufferUtil.bytes("k100"));
 -        rm.add("Indexed1", ByteBufferUtil.bytes("birthdate"), ByteBufferUtil.bytes(100L), 1, 1000);
++        rm = new Mutation("Keyspace2", ByteBufferUtil.bytes("k100"));
++        rm.add("Indexed1", cellname("birthdate"), ByteBufferUtil.bytes(100L), 1, 1000);
+         rm.apply();
+ 
+         rows = keyspace.getColumnFamilyStore("Indexed1").search(range, clause, filter, 100);
+         assertEquals(1, rows.size());
+ 
+         // check that modifying the indexed value using the same timestamp behaves as expected
 -        rm = new RowMutation("Keyspace2", ByteBufferUtil.bytes("k101"));
 -        rm.add("Indexed1", ByteBufferUtil.bytes("birthdate"), ByteBufferUtil.bytes(101L), 1, 1000);
++        rm = new Mutation("Keyspace2", ByteBufferUtil.bytes("k101"));
++        rm.add("Indexed1", cellname("birthdate"), ByteBufferUtil.bytes(101L), 1, 1000);
+         rm.apply();
+ 
 -        expr = new IndexExpression(ByteBufferUtil.bytes("birthdate"), IndexOperator.EQ, ByteBufferUtil.bytes(101L));
++        expr = new IndexExpression(ByteBufferUtil.bytes("birthdate"), IndexExpression.Operator.EQ, ByteBufferUtil.bytes(101L));
+         clause = Arrays.asList(expr);
+         rows = keyspace.getColumnFamilyStore("Indexed1").search(range, clause, filter, 100);
+         assertEquals(1, rows.size());
+ 
+         TimeUnit.SECONDS.sleep(1);
 -        rm = new RowMutation("Keyspace2", ByteBufferUtil.bytes("k101"));
 -        rm.add("Indexed1", ByteBufferUtil.bytes("birthdate"), ByteBufferUtil.bytes(102L), 1, 1000);
++        rm = new Mutation("Keyspace2", ByteBufferUtil.bytes("k101"));
++        rm.add("Indexed1", cellname("birthdate"), ByteBufferUtil.bytes(102L), 1, 1000);
+         rm.apply();
+         // search for the old value
+         rows = keyspace.getColumnFamilyStore("Indexed1").search(range, clause, filter, 100);
+         assertEquals(0, rows.size());
+         // and for the new
 -        expr = new IndexExpression(ByteBufferUtil.bytes("birthdate"), IndexOperator.EQ, ByteBufferUtil.bytes(102L));
++        expr = new IndexExpression(ByteBufferUtil.bytes("birthdate"), IndexExpression.Operator.EQ, ByteBufferUtil.bytes(102L));
+         clause = Arrays.asList(expr);
+         rows = keyspace.getColumnFamilyStore("Indexed1").search(range, clause, filter, 100);
+         assertEquals(1, rows.size());
+     }
+ 
+     @Test
      public void testDeleteOfInconsistentValuesInKeysIndex() throws Exception
      {
          String keySpace = "Keyspace2";