You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by jb...@apache.org on 2013/09/20 00:45:53 UTC

[1/6] git commit: Revert CASSANDRA-5577 to fix CASSANDRA-6025 and add test patch by jbellis

Updated Branches:
  refs/heads/cassandra-2.0 023b55f3b -> f9f5af09d
  refs/heads/trunk 681312393 -> 388e1b236


Revert CASSANDRA-5577 to fix CASSANDRA-6025 and add test
patch by jbellis


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

Branch: refs/heads/cassandra-2.0
Commit: e5cbde9d62d7cb3a4755099750107aea77b157be
Parents: 023b55f
Author: Jonathan Ellis <jb...@apache.org>
Authored: Thu Sep 19 17:43:21 2013 -0500
Committer: Jonathan Ellis <jb...@apache.org>
Committed: Thu Sep 19 17:43:21 2013 -0500

----------------------------------------------------------------------
 .../cassandra/db/CollationController.java       | 32 ++-------
 .../unit/org/apache/cassandra/SchemaLoader.java |  1 +
 .../cassandra/db/CollationControllerTest.java   | 73 +++++++++++++++-----
 3 files changed, 63 insertions(+), 43 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5cbde9d/src/java/org/apache/cassandra/db/CollationController.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/CollationController.java b/src/java/org/apache/cassandra/db/CollationController.java
index 4ae27a2..859135d 100644
--- a/src/java/org/apache/cassandra/db/CollationController.java
+++ b/src/java/org/apache/cassandra/db/CollationController.java
@@ -85,11 +85,7 @@ public class CollationController
                     iterators.add(iter);
                     temp.delete(iter.getColumnFamily());
                     while (iter.hasNext())
-                    {
-                        OnDiskAtom atom = iter.next();
-                        if (atom.getLocalDeletionTime() >= gcBefore)
-                            temp.addAtom(atom);
-                    }
+                        temp.addAtom(iter.next());
                 }
 
                 container.addAll(temp, HeapAllocator.instance);
@@ -131,11 +127,7 @@ public class CollationController
                     temp.delete(cf);
                     sstablesIterated++;
                     while (iter.hasNext())
-                    {
-                        OnDiskAtom atom = iter.next();
-                        if (atom.getLocalDeletionTime() >= gcBefore)
-                            temp.addAtom(atom);
-                    }
+                        temp.addAtom(iter.next());
                 }
 
                 container.addAll(temp, HeapAllocator.instance);
@@ -147,22 +139,10 @@ public class CollationController
             if (iterators.isEmpty())
                 return null;
 
-            // We may have added columns that are shadowed by range or row tombstones, since we don't know what
-            // tombstones we may encounter in older sstables (and we don't know how many older sstables we'll have
-            // to open, without processing newer ones first).  So, make one more pass if necessary to clean those out.
-            ColumnFamily returnCF;
-            if (container.isMarkedForDelete())
-            {
-                returnCF = container.cloneMeShallow();
-                Tracing.trace("Removing shadowed cells");
-                filter.collateOnDiskAtom(returnCF, container.iterator(), gcBefore);
-            }
-            else
-            {
-                // skipping the collate is safe because we only do this time-ordered path for NameQueryFilter;
-                // for SQF, the collate is also what limits us to the requested number of columns.
-                returnCF = container;
-            }
+            // do a final collate.  toCollate is boilerplate required to provide a CloseableIterator
+            ColumnFamily returnCF = container.cloneMeShallow();
+            Tracing.trace("Collating all results");
+            filter.collateOnDiskAtom(returnCF, container.iterator(), gcBefore);
 
             // "hoist up" the requested data into a more recent sstable
             if (sstablesIterated > cfs.getMinimumCompactionThreshold()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5cbde9d/test/unit/org/apache/cassandra/SchemaLoader.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/SchemaLoader.java b/test/unit/org/apache/cassandra/SchemaLoader.java
index 1bd192b..f933b6f 100644
--- a/test/unit/org/apache/cassandra/SchemaLoader.java
+++ b/test/unit/org/apache/cassandra/SchemaLoader.java
@@ -149,6 +149,7 @@ public class SchemaLoader
                                            standardCFMD(ks1, "Standard2"),
                                            standardCFMD(ks1, "Standard3"),
                                            standardCFMD(ks1, "Standard4"),
+                                           standardCFMD(ks1, "StandardGCGS0").gcGraceSeconds(0),
                                            standardCFMD(ks1, "StandardLong1"),
                                            standardCFMD(ks1, "StandardLong2"),
                                            new CFMetaData(ks1,

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5cbde9d/test/unit/org/apache/cassandra/db/CollationControllerTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/CollationControllerTest.java b/test/unit/org/apache/cassandra/db/CollationControllerTest.java
index 0c23272..8763a41 100644
--- a/test/unit/org/apache/cassandra/db/CollationControllerTest.java
+++ b/test/unit/org/apache/cassandra/db/CollationControllerTest.java
@@ -21,6 +21,7 @@ package org.apache.cassandra.db;
 import static org.junit.Assert.assertEquals;
 
 import java.io.IOException;
+import java.nio.ByteBuffer;
 import java.util.concurrent.ExecutionException;
 
 import org.apache.cassandra.SchemaLoader;
@@ -35,51 +36,89 @@ public class CollationControllerTest extends SchemaLoader
 {
     @Test
     public void getTopLevelColumnsSkipsSSTablesModifiedBeforeRowDelete() 
-            throws IOException, ExecutionException, InterruptedException
+    throws IOException, ExecutionException, InterruptedException
     {
         Keyspace keyspace = Keyspace.open("Keyspace1");
-        ColumnFamilyStore store = keyspace.getColumnFamilyStore("Standard1");
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore("Standard1");
         RowMutation rm;
         DecoratedKey dk = Util.dk("key1");
         
         // add data
-        rm = new RowMutation("Keyspace1", dk.key);
-        rm.add("Standard1", ByteBufferUtil.bytes("Column1"), ByteBufferUtil.bytes("asdf"), 0);
+        rm = new RowMutation(keyspace.getName(), dk.key);
+        rm.add(cfs.name, ByteBufferUtil.bytes("Column1"), ByteBufferUtil.bytes("asdf"), 0);
         rm.apply();
-        store.forceBlockingFlush();
+        cfs.forceBlockingFlush();
         
         // remove
-        rm = new RowMutation("Keyspace1", dk.key);
-        rm.delete("Standard1", 10);
+        rm = new RowMutation(keyspace.getName(), dk.key);
+        rm.delete(cfs.name, 10);
         rm.apply();
         
         // add another mutation because sstable maxtimestamp isn't set
         // correctly during flush if the most recent mutation is a row delete
-        rm = new RowMutation("Keyspace1", Util.dk("key2").key);
-        rm.add("Standard1", ByteBufferUtil.bytes("Column1"), ByteBufferUtil.bytes("zxcv"), 20);
+        rm = new RowMutation(keyspace.getName(), Util.dk("key2").key);
+        rm.add(cfs.name, ByteBufferUtil.bytes("Column1"), ByteBufferUtil.bytes("zxcv"), 20);
         rm.apply();
         
-        store.forceBlockingFlush();
+        cfs.forceBlockingFlush();
 
         // add yet one more mutation
-        rm = new RowMutation("Keyspace1", dk.key);
-        rm.add("Standard1", ByteBufferUtil.bytes("Column1"), ByteBufferUtil.bytes("foobar"), 30);
+        rm = new RowMutation(keyspace.getName(), dk.key);
+        rm.add(cfs.name, ByteBufferUtil.bytes("Column1"), ByteBufferUtil.bytes("foobar"), 30);
         rm.apply();
-        store.forceBlockingFlush();
+        cfs.forceBlockingFlush();
 
         // A NamesQueryFilter goes down one code path (through collectTimeOrderedData())
         // It should only iterate the last flushed sstable, since it probably contains the most recent value for Column1
-        QueryFilter filter = QueryFilter.getNamesFilter(dk, "Standard1", ByteBufferUtil.bytes("Column1"), System.currentTimeMillis());
-        CollationController controller = new CollationController(store, filter, Integer.MIN_VALUE);
+        QueryFilter filter = QueryFilter.getNamesFilter(dk, cfs.name, ByteBufferUtil.bytes("Column1"), System.currentTimeMillis());
+        CollationController controller = new CollationController(cfs, filter, Integer.MIN_VALUE);
         controller.getTopLevelColumns();
         assertEquals(1, controller.getSstablesIterated());
 
         // SliceQueryFilter goes down another path (through collectAllData())
         // We will read "only" the last sstable in that case, but because the 2nd sstable has a tombstone that is more
         // recent than the maxTimestamp of the very first sstable we flushed, we should only read the 2 first sstables.
-        filter = QueryFilter.getIdentityFilter(dk, "Standard1", System.currentTimeMillis());
-        controller = new CollationController(store, filter, Integer.MIN_VALUE);
+        filter = QueryFilter.getIdentityFilter(dk, cfs.name, System.currentTimeMillis());
+        controller = new CollationController(cfs, filter, Integer.MIN_VALUE);
         controller.getTopLevelColumns();
         assertEquals(2, controller.getSstablesIterated());
     }
+
+    @Test
+    public void ensureTombstonesAppliedAfterGCGS()
+    throws IOException, ExecutionException, InterruptedException
+    {
+        Keyspace keyspace = Keyspace.open("Keyspace1");
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore("StandardGCGS0");
+        cfs.disableAutoCompaction();
+
+        RowMutation rm;
+        DecoratedKey dk = Util.dk("key1");
+        ByteBuffer cellName = ByteBufferUtil.bytes("Column1");
+
+        // add data
+        rm = new RowMutation(keyspace.getName(), dk.key);
+        rm.add(cfs.name, cellName, ByteBufferUtil.bytes("asdf"), 0);
+        rm.apply();
+        cfs.forceBlockingFlush();
+
+        // remove
+        rm = new RowMutation(keyspace.getName(), dk.key);
+        rm.delete(cfs.name, cellName, 0);
+        rm.apply();
+        cfs.forceBlockingFlush();
+
+        // use "realistic" query times since we'll compare these numbers to the local deletion time of the tombstone
+        QueryFilter filter;
+        long queryAt = System.currentTimeMillis() + 1000;
+        int gcBefore = cfs.gcBefore(queryAt);
+
+        filter = QueryFilter.getNamesFilter(dk, cfs.name, cellName, queryAt);
+        CollationController controller = new CollationController(cfs, filter, gcBefore);
+        assert ColumnFamilyStore.removeDeleted(controller.getTopLevelColumns(), gcBefore) == null;
+
+        filter = QueryFilter.getIdentityFilter(dk, cfs.name, queryAt);
+        controller = new CollationController(cfs, filter, gcBefore);
+        assert ColumnFamilyStore.removeDeleted(controller.getTopLevelColumns(), gcBefore) == null;
+    }
 }


[2/6] git commit: Revert CASSANDRA-5577 to fix CASSANDRA-6025 and add test patch by jbellis

Posted by jb...@apache.org.
Revert CASSANDRA-5577 to fix CASSANDRA-6025 and add test
patch by jbellis


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

Branch: refs/heads/trunk
Commit: e5cbde9d62d7cb3a4755099750107aea77b157be
Parents: 023b55f
Author: Jonathan Ellis <jb...@apache.org>
Authored: Thu Sep 19 17:43:21 2013 -0500
Committer: Jonathan Ellis <jb...@apache.org>
Committed: Thu Sep 19 17:43:21 2013 -0500

----------------------------------------------------------------------
 .../cassandra/db/CollationController.java       | 32 ++-------
 .../unit/org/apache/cassandra/SchemaLoader.java |  1 +
 .../cassandra/db/CollationControllerTest.java   | 73 +++++++++++++++-----
 3 files changed, 63 insertions(+), 43 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5cbde9d/src/java/org/apache/cassandra/db/CollationController.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/CollationController.java b/src/java/org/apache/cassandra/db/CollationController.java
index 4ae27a2..859135d 100644
--- a/src/java/org/apache/cassandra/db/CollationController.java
+++ b/src/java/org/apache/cassandra/db/CollationController.java
@@ -85,11 +85,7 @@ public class CollationController
                     iterators.add(iter);
                     temp.delete(iter.getColumnFamily());
                     while (iter.hasNext())
-                    {
-                        OnDiskAtom atom = iter.next();
-                        if (atom.getLocalDeletionTime() >= gcBefore)
-                            temp.addAtom(atom);
-                    }
+                        temp.addAtom(iter.next());
                 }
 
                 container.addAll(temp, HeapAllocator.instance);
@@ -131,11 +127,7 @@ public class CollationController
                     temp.delete(cf);
                     sstablesIterated++;
                     while (iter.hasNext())
-                    {
-                        OnDiskAtom atom = iter.next();
-                        if (atom.getLocalDeletionTime() >= gcBefore)
-                            temp.addAtom(atom);
-                    }
+                        temp.addAtom(iter.next());
                 }
 
                 container.addAll(temp, HeapAllocator.instance);
@@ -147,22 +139,10 @@ public class CollationController
             if (iterators.isEmpty())
                 return null;
 
-            // We may have added columns that are shadowed by range or row tombstones, since we don't know what
-            // tombstones we may encounter in older sstables (and we don't know how many older sstables we'll have
-            // to open, without processing newer ones first).  So, make one more pass if necessary to clean those out.
-            ColumnFamily returnCF;
-            if (container.isMarkedForDelete())
-            {
-                returnCF = container.cloneMeShallow();
-                Tracing.trace("Removing shadowed cells");
-                filter.collateOnDiskAtom(returnCF, container.iterator(), gcBefore);
-            }
-            else
-            {
-                // skipping the collate is safe because we only do this time-ordered path for NameQueryFilter;
-                // for SQF, the collate is also what limits us to the requested number of columns.
-                returnCF = container;
-            }
+            // do a final collate.  toCollate is boilerplate required to provide a CloseableIterator
+            ColumnFamily returnCF = container.cloneMeShallow();
+            Tracing.trace("Collating all results");
+            filter.collateOnDiskAtom(returnCF, container.iterator(), gcBefore);
 
             // "hoist up" the requested data into a more recent sstable
             if (sstablesIterated > cfs.getMinimumCompactionThreshold()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5cbde9d/test/unit/org/apache/cassandra/SchemaLoader.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/SchemaLoader.java b/test/unit/org/apache/cassandra/SchemaLoader.java
index 1bd192b..f933b6f 100644
--- a/test/unit/org/apache/cassandra/SchemaLoader.java
+++ b/test/unit/org/apache/cassandra/SchemaLoader.java
@@ -149,6 +149,7 @@ public class SchemaLoader
                                            standardCFMD(ks1, "Standard2"),
                                            standardCFMD(ks1, "Standard3"),
                                            standardCFMD(ks1, "Standard4"),
+                                           standardCFMD(ks1, "StandardGCGS0").gcGraceSeconds(0),
                                            standardCFMD(ks1, "StandardLong1"),
                                            standardCFMD(ks1, "StandardLong2"),
                                            new CFMetaData(ks1,

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5cbde9d/test/unit/org/apache/cassandra/db/CollationControllerTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/CollationControllerTest.java b/test/unit/org/apache/cassandra/db/CollationControllerTest.java
index 0c23272..8763a41 100644
--- a/test/unit/org/apache/cassandra/db/CollationControllerTest.java
+++ b/test/unit/org/apache/cassandra/db/CollationControllerTest.java
@@ -21,6 +21,7 @@ package org.apache.cassandra.db;
 import static org.junit.Assert.assertEquals;
 
 import java.io.IOException;
+import java.nio.ByteBuffer;
 import java.util.concurrent.ExecutionException;
 
 import org.apache.cassandra.SchemaLoader;
@@ -35,51 +36,89 @@ public class CollationControllerTest extends SchemaLoader
 {
     @Test
     public void getTopLevelColumnsSkipsSSTablesModifiedBeforeRowDelete() 
-            throws IOException, ExecutionException, InterruptedException
+    throws IOException, ExecutionException, InterruptedException
     {
         Keyspace keyspace = Keyspace.open("Keyspace1");
-        ColumnFamilyStore store = keyspace.getColumnFamilyStore("Standard1");
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore("Standard1");
         RowMutation rm;
         DecoratedKey dk = Util.dk("key1");
         
         // add data
-        rm = new RowMutation("Keyspace1", dk.key);
-        rm.add("Standard1", ByteBufferUtil.bytes("Column1"), ByteBufferUtil.bytes("asdf"), 0);
+        rm = new RowMutation(keyspace.getName(), dk.key);
+        rm.add(cfs.name, ByteBufferUtil.bytes("Column1"), ByteBufferUtil.bytes("asdf"), 0);
         rm.apply();
-        store.forceBlockingFlush();
+        cfs.forceBlockingFlush();
         
         // remove
-        rm = new RowMutation("Keyspace1", dk.key);
-        rm.delete("Standard1", 10);
+        rm = new RowMutation(keyspace.getName(), dk.key);
+        rm.delete(cfs.name, 10);
         rm.apply();
         
         // add another mutation because sstable maxtimestamp isn't set
         // correctly during flush if the most recent mutation is a row delete
-        rm = new RowMutation("Keyspace1", Util.dk("key2").key);
-        rm.add("Standard1", ByteBufferUtil.bytes("Column1"), ByteBufferUtil.bytes("zxcv"), 20);
+        rm = new RowMutation(keyspace.getName(), Util.dk("key2").key);
+        rm.add(cfs.name, ByteBufferUtil.bytes("Column1"), ByteBufferUtil.bytes("zxcv"), 20);
         rm.apply();
         
-        store.forceBlockingFlush();
+        cfs.forceBlockingFlush();
 
         // add yet one more mutation
-        rm = new RowMutation("Keyspace1", dk.key);
-        rm.add("Standard1", ByteBufferUtil.bytes("Column1"), ByteBufferUtil.bytes("foobar"), 30);
+        rm = new RowMutation(keyspace.getName(), dk.key);
+        rm.add(cfs.name, ByteBufferUtil.bytes("Column1"), ByteBufferUtil.bytes("foobar"), 30);
         rm.apply();
-        store.forceBlockingFlush();
+        cfs.forceBlockingFlush();
 
         // A NamesQueryFilter goes down one code path (through collectTimeOrderedData())
         // It should only iterate the last flushed sstable, since it probably contains the most recent value for Column1
-        QueryFilter filter = QueryFilter.getNamesFilter(dk, "Standard1", ByteBufferUtil.bytes("Column1"), System.currentTimeMillis());
-        CollationController controller = new CollationController(store, filter, Integer.MIN_VALUE);
+        QueryFilter filter = QueryFilter.getNamesFilter(dk, cfs.name, ByteBufferUtil.bytes("Column1"), System.currentTimeMillis());
+        CollationController controller = new CollationController(cfs, filter, Integer.MIN_VALUE);
         controller.getTopLevelColumns();
         assertEquals(1, controller.getSstablesIterated());
 
         // SliceQueryFilter goes down another path (through collectAllData())
         // We will read "only" the last sstable in that case, but because the 2nd sstable has a tombstone that is more
         // recent than the maxTimestamp of the very first sstable we flushed, we should only read the 2 first sstables.
-        filter = QueryFilter.getIdentityFilter(dk, "Standard1", System.currentTimeMillis());
-        controller = new CollationController(store, filter, Integer.MIN_VALUE);
+        filter = QueryFilter.getIdentityFilter(dk, cfs.name, System.currentTimeMillis());
+        controller = new CollationController(cfs, filter, Integer.MIN_VALUE);
         controller.getTopLevelColumns();
         assertEquals(2, controller.getSstablesIterated());
     }
+
+    @Test
+    public void ensureTombstonesAppliedAfterGCGS()
+    throws IOException, ExecutionException, InterruptedException
+    {
+        Keyspace keyspace = Keyspace.open("Keyspace1");
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore("StandardGCGS0");
+        cfs.disableAutoCompaction();
+
+        RowMutation rm;
+        DecoratedKey dk = Util.dk("key1");
+        ByteBuffer cellName = ByteBufferUtil.bytes("Column1");
+
+        // add data
+        rm = new RowMutation(keyspace.getName(), dk.key);
+        rm.add(cfs.name, cellName, ByteBufferUtil.bytes("asdf"), 0);
+        rm.apply();
+        cfs.forceBlockingFlush();
+
+        // remove
+        rm = new RowMutation(keyspace.getName(), dk.key);
+        rm.delete(cfs.name, cellName, 0);
+        rm.apply();
+        cfs.forceBlockingFlush();
+
+        // use "realistic" query times since we'll compare these numbers to the local deletion time of the tombstone
+        QueryFilter filter;
+        long queryAt = System.currentTimeMillis() + 1000;
+        int gcBefore = cfs.gcBefore(queryAt);
+
+        filter = QueryFilter.getNamesFilter(dk, cfs.name, cellName, queryAt);
+        CollationController controller = new CollationController(cfs, filter, gcBefore);
+        assert ColumnFamilyStore.removeDeleted(controller.getTopLevelColumns(), gcBefore) == null;
+
+        filter = QueryFilter.getIdentityFilter(dk, cfs.name, queryAt);
+        controller = new CollationController(cfs, filter, gcBefore);
+        assert ColumnFamilyStore.removeDeleted(controller.getTopLevelColumns(), gcBefore) == null;
+    }
 }


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

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


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

Branch: refs/heads/trunk
Commit: 1257868e33827b6e367c6dc51cca4f67f948c9ae
Parents: 6813123 e5cbde9
Author: Jonathan Ellis <jb...@apache.org>
Authored: Thu Sep 19 17:44:33 2013 -0500
Committer: Jonathan Ellis <jb...@apache.org>
Committed: Thu Sep 19 17:44:33 2013 -0500

----------------------------------------------------------------------
 .../cassandra/db/CollationController.java       | 32 ++-------
 .../unit/org/apache/cassandra/SchemaLoader.java |  1 +
 .../cassandra/db/CollationControllerTest.java   | 73 +++++++++++++++-----
 3 files changed, 63 insertions(+), 43 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/1257868e/test/unit/org/apache/cassandra/SchemaLoader.java
----------------------------------------------------------------------


[4/6] git commit: CHANGES

Posted by jb...@apache.org.
CHANGES


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

Branch: refs/heads/trunk
Commit: f9f5af09d406cb4596da3be080e69e44d891dc08
Parents: e5cbde9
Author: Jonathan Ellis <jb...@apache.org>
Authored: Thu Sep 19 17:45:36 2013 -0500
Committer: Jonathan Ellis <jb...@apache.org>
Committed: Thu Sep 19 17:45:36 2013 -0500

----------------------------------------------------------------------
 CHANGES.txt | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/f9f5af09/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index a6938f4..7f30117 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 2.0.1
+ * Fix bug that could allow reading deleted data temporarily (CASSANDRA-6025)
  * Improve memory use defaults (CASSANDRA-5069)
  * Make ThriftServer more easlly extensible (CASSANDRA-6058)
  * Remove Hadoop dependency from ITransportFactory (CASSANDRA-6062)


[5/6] git commit: CHANGES

Posted by jb...@apache.org.
CHANGES


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

Branch: refs/heads/cassandra-2.0
Commit: f9f5af09d406cb4596da3be080e69e44d891dc08
Parents: e5cbde9
Author: Jonathan Ellis <jb...@apache.org>
Authored: Thu Sep 19 17:45:36 2013 -0500
Committer: Jonathan Ellis <jb...@apache.org>
Committed: Thu Sep 19 17:45:36 2013 -0500

----------------------------------------------------------------------
 CHANGES.txt | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/f9f5af09/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index a6938f4..7f30117 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 2.0.1
+ * Fix bug that could allow reading deleted data temporarily (CASSANDRA-6025)
  * Improve memory use defaults (CASSANDRA-5069)
  * Make ThriftServer more easlly extensible (CASSANDRA-6058)
  * Remove Hadoop dependency from ITransportFactory (CASSANDRA-6062)


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

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


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

Branch: refs/heads/trunk
Commit: 388e1b2363c1c5857a6ad58472677e26ca692867
Parents: 1257868 f9f5af0
Author: Jonathan Ellis <jb...@apache.org>
Authored: Thu Sep 19 17:45:41 2013 -0500
Committer: Jonathan Ellis <jb...@apache.org>
Committed: Thu Sep 19 17:45:41 2013 -0500

----------------------------------------------------------------------
 CHANGES.txt | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/388e1b23/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 578da35,7f30117..72488e8
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,12 -1,5 +1,13 @@@
 +2.1
 + * change logging from log4j to logback (CASSANDRA-5883)
 + * switch to LZ4 compression for internode communication (CASSANDRA-5887)
 + * Stop using Thrift-generated Index* classes internally (CASSANDRA-5971)
 + * Remove 1.2 network compatibility code (CASSANDRA-5960)
 + * Remove leveled json manifest migration code (CASSANDRA-5996)
 +
 +
  2.0.1
+  * Fix bug that could allow reading deleted data temporarily (CASSANDRA-6025)
   * Improve memory use defaults (CASSANDRA-5069)
   * Make ThriftServer more easlly extensible (CASSANDRA-6058)
   * Remove Hadoop dependency from ITransportFactory (CASSANDRA-6062)