You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by ja...@apache.org on 2014/08/12 16:01:05 UTC

[1/3] git commit: Do not flush on truncate if durable_writes is false

Repository: cassandra
Updated Branches:
  refs/heads/cassandra-2.1.0 a1348aa29 -> c7834d3da


 Do not flush on truncate if durable_writes is false

Patch by Jeremiah Jordan; reviewed by tjake for CASSANDRA-7750


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

Branch: refs/heads/cassandra-2.1.0
Commit: 9be6576f24e52ca6553981976ac589bf6966e804
Parents: 52df514d
Author: Jake Luciani <ja...@apache.org>
Authored: Tue Aug 12 09:53:53 2014 -0400
Committer: Jake Luciani <ja...@apache.org>
Committed: Tue Aug 12 09:53:53 2014 -0400

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../apache/cassandra/db/ColumnFamilyStore.java  | 29 ++++++++++++++++----
 .../org/apache/cassandra/db/DataTracker.java    | 18 ++++++++++++
 .../org/apache/cassandra/db/CommitLogTest.java  | 29 ++++++++++++++++++++
 4 files changed, 72 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/9be6576f/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index ddf4627..fc32426 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 2.0.10
+ * Do not flush on truncate if durable_writes is false (CASSANDRA-7750)
  * Give CRR a default input_cql Statement (CASSANDRA-7226)
  * Better error message when adding a collection with the same name
    than a previously dropped one (CASSANDRA-6276)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9be6576f/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
index a3c080a..3da44de 100644
--- a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
+++ b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
@@ -2002,12 +2002,31 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
         // position in the System keyspace.
         logger.debug("truncating {}", name);
 
-        // flush the CF being truncated before forcing the new segment
-        forceBlockingFlush();
+        if (keyspace.metadata.durableWrites || DatabaseDescriptor.isAutoSnapshot())
+        {
+            // flush the CF being truncated before forcing the new segment
+            forceBlockingFlush();
 
-        // sleep a little to make sure that our truncatedAt comes after any sstable
-        // that was part of the flushed we forced; otherwise on a tie, it won't get deleted.
-        Uninterruptibles.sleepUninterruptibly(1, TimeUnit.MILLISECONDS);
+            // sleep a little to make sure that our truncatedAt comes after any sstable
+            // that was part of the flushed we forced; otherwise on a tie, it won't get deleted.
+            Uninterruptibles.sleepUninterruptibly(1, TimeUnit.MILLISECONDS);
+        }
+        else
+        {
+            Keyspace.switchLock.writeLock().lock();
+            try
+            {
+                for (ColumnFamilyStore cfs : concatWithIndexes())
+                {
+                    Memtable mt = cfs.getMemtableThreadSafe();
+                    if (!mt.isClean())
+                        mt.cfs.data.renewMemtable();
+                }
+            } finally
+            {
+                Keyspace.switchLock.writeLock().unlock();
+            }
+        }
 
         Runnable truncateRunnable = new Runnable()
         {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9be6576f/src/java/org/apache/cassandra/db/DataTracker.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/DataTracker.java b/src/java/org/apache/cassandra/db/DataTracker.java
index a0f880a..a9eef98 100644
--- a/src/java/org/apache/cassandra/db/DataTracker.java
+++ b/src/java/org/apache/cassandra/db/DataTracker.java
@@ -123,6 +123,24 @@ public class DataTracker
         return toFlushMemtable;
     }
 
+    /**
+     * Renew the current memtable without putting the old one for a flush.
+     * Used when we flush but a memtable is clean (in which case we must
+     * change it because it was frozen).
+     */
+    public void renewMemtable()
+    {
+        Memtable newMemtable = new Memtable(cfstore, view.get().memtable);
+        View currentView, newView;
+        do
+        {
+            currentView = view.get();
+            newView = currentView.renewMemtable(newMemtable);
+        }
+        while (!view.compareAndSet(currentView, newView));
+        notifyRenewed(currentView.memtable);
+    }
+
     public void replaceFlushed(Memtable memtable, SSTableReader sstable)
     {
         // sstable may be null if we flushed batchlog and nothing needed to be retained

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9be6576f/test/unit/org/apache/cassandra/db/CommitLogTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/CommitLogTest.java b/test/unit/org/apache/cassandra/db/CommitLogTest.java
index a7df871..1be29a6 100644
--- a/test/unit/org/apache/cassandra/db/CommitLogTest.java
+++ b/test/unit/org/apache/cassandra/db/CommitLogTest.java
@@ -32,13 +32,16 @@ import org.junit.Assert;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.Util;
 import org.apache.cassandra.config.Config;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.commitlog.CommitLog;
 import org.apache.cassandra.db.commitlog.CommitLogDescriptor;
 import org.apache.cassandra.db.commitlog.ReplayPosition;
+import org.apache.cassandra.db.filter.NamesQueryFilter;
 import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.FBUtilities;
 
 import static org.apache.cassandra.utils.ByteBufferUtil.bytes;
@@ -289,4 +292,30 @@ public class CommitLogTest extends SchemaLoader
         Assert.assertEquals(1, CommitLog.instance.activeSegments());
     }
 
+    @Test
+    public void testTruncateWithoutSnapshotNonDurable()  throws ExecutionException, InterruptedException
+    {
+        CommitLog.instance.resetUnsafe();
+        boolean prevAutoSnapshot = DatabaseDescriptor.isAutoSnapshot();
+        DatabaseDescriptor.setAutoSnapshot(false);
+        Keyspace notDurableKs = Keyspace.open("NoCommitlogSpace");
+        Assert.assertFalse(notDurableKs.metadata.durableWrites);
+        ColumnFamilyStore cfs = notDurableKs.getColumnFamilyStore("Standard1");
+        RowMutation rm;
+        DecoratedKey dk = Util.dk("key1");
+
+        // add data
+        rm = new RowMutation("NoCommitlogSpace", dk.key);
+        rm.add("Standard1", ByteBufferUtil.bytes("Column1"), ByteBufferUtil.bytes("abcd"), 0);
+        rm.apply();
+
+        ReadCommand command = new SliceByNamesReadCommand("Keyspace1", dk.key, "Standard1", System.currentTimeMillis(), new NamesQueryFilter(FBUtilities.singleton(ByteBufferUtil.bytes("Column1"), cfs.getComparator())));
+        Row row = command.getRow(notDurableKs);
+        Column col = row.cf.getColumn(ByteBufferUtil.bytes("Column1"));
+        Assert.assertEquals(col.value(), ByteBuffer.wrap("abcd".getBytes()));
+        cfs.truncateBlocking();
+        DatabaseDescriptor.setAutoSnapshot(prevAutoSnapshot);
+        row = command.getRow(notDurableKs);
+        Assert.assertEquals(null, row.cf);
+    }
 }


[3/3] git commit: Merge 2.0

Posted by ja...@apache.org.
Merge 2.0


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

Branch: refs/heads/cassandra-2.1.0
Commit: c7834d3dab82860ef8d87b043b8d6a7150419edb
Parents: 3e2e4dd
Author: Jake Luciani <ja...@apache.org>
Authored: Tue Aug 12 10:00:28 2014 -0400
Committer: Jake Luciani <ja...@apache.org>
Committed: Tue Aug 12 10:00:28 2014 -0400

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../apache/cassandra/db/ColumnFamilyStore.java  | 23 +++++++++++----
 .../org/apache/cassandra/db/CommitLogTest.java  | 30 ++++++++++++++++++++
 3 files changed, 49 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/c7834d3d/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index a180df9..342eb00 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -8,6 +8,7 @@
  * Fix UDT field selection with empty fields (CASSANDRA-7670)
  * Bogus deserialization of static cells from sstable (CASSANDRA-7684)
 Merged from 2.0:
+ * Do not flush on truncate if durable_writes is false (CASSANDRA-7750)
  * Give CRR a default input_cql Statement (CASSANDRA-7226)
  * Better error message when adding a collection with the same name
    than a previously dropped one (CASSANDRA-6276)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c7834d3d/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
index a1220df..a0860a7 100644
--- a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
+++ b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
@@ -2420,12 +2420,25 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
         // position in the System keyspace.
         logger.debug("truncating {}", name);
 
-        // flush the CF being truncated before forcing the new segment
-        forceBlockingFlush();
+        if (keyspace.metadata.durableWrites || DatabaseDescriptor.isAutoSnapshot())
+        {
+            // flush the CF being truncated before forcing the new segment
+            forceBlockingFlush();
 
-        // sleep a little to make sure that our truncatedAt comes after any sstable
-        // that was part of the flushed we forced; otherwise on a tie, it won't get deleted.
-        Uninterruptibles.sleepUninterruptibly(1, TimeUnit.MILLISECONDS);
+            // sleep a little to make sure that our truncatedAt comes after any sstable
+            // that was part of the flushed we forced; otherwise on a tie, it won't get deleted.
+            Uninterruptibles.sleepUninterruptibly(1, TimeUnit.MILLISECONDS);
+        }
+        else
+        {
+            // just nuke the memtable data w/o writing to disk first
+            synchronized (data)
+            {
+                final Flush flush = new Flush(true);
+                flushExecutor.execute(flush);
+                postFlushExecutor.submit(flush.postFlush);
+            }
+        }
 
         Runnable truncateRunnable = new Runnable()
         {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c7834d3d/test/unit/org/apache/cassandra/db/CommitLogTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/CommitLogTest.java b/test/unit/org/apache/cassandra/db/CommitLogTest.java
index a58549a..ed9601d 100644
--- a/test/unit/org/apache/cassandra/db/CommitLogTest.java
+++ b/test/unit/org/apache/cassandra/db/CommitLogTest.java
@@ -40,8 +40,11 @@ import org.apache.cassandra.db.commitlog.CommitLogDescriptor;
 import org.apache.cassandra.db.commitlog.ReplayPosition;
 import org.apache.cassandra.db.commitlog.CommitLogSegment;
 import org.apache.cassandra.db.composites.CellName;
+import org.apache.cassandra.db.composites.CellNameType;
+import org.apache.cassandra.db.filter.NamesQueryFilter;
 import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.FBUtilities;
 
 import static org.apache.cassandra.utils.ByteBufferUtil.bytes;
@@ -327,4 +330,31 @@ public class CommitLogTest extends SchemaLoader
         Assert.assertEquals(1, CommitLog.instance.activeSegments());
     }
 
+    @Test
+    public void testTruncateWithoutSnapshotNonDurable()  throws ExecutionException, InterruptedException
+    {
+        CommitLog.instance.resetUnsafe();
+        boolean prevAutoSnapshot = DatabaseDescriptor.isAutoSnapshot();
+        DatabaseDescriptor.setAutoSnapshot(false);
+        Keyspace notDurableKs = Keyspace.open("NoCommitlogSpace");
+        Assert.assertFalse(notDurableKs.metadata.durableWrites);
+        ColumnFamilyStore cfs = notDurableKs.getColumnFamilyStore("Standard1");
+        CellNameType type = notDurableKs.getColumnFamilyStore("Standard1").getComparator();
+        Mutation rm;
+        DecoratedKey dk = Util.dk("key1");
+
+        // add data
+        rm = new Mutation("NoCommitlogSpace", dk.getKey());
+        rm.add("Standard1", Util.cellname("Column1"), ByteBufferUtil.bytes("abcd"), 0);
+        rm.apply();
+
+        ReadCommand command = new SliceByNamesReadCommand("NoCommitlogSpace", dk.getKey(), "Standard1", System.currentTimeMillis(), new NamesQueryFilter(FBUtilities.singleton(Util.cellname("Column1"), type)));
+        Row row = command.getRow(notDurableKs);
+        Cell col = row.cf.getColumn(Util.cellname("Column1"));
+        Assert.assertEquals(col.value(), ByteBuffer.wrap("abcd".getBytes()));
+        cfs.truncateBlocking();
+        DatabaseDescriptor.setAutoSnapshot(prevAutoSnapshot);
+        row = command.getRow(notDurableKs);
+        Assert.assertEquals(null, row.cf);
+    }
 }


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

Posted by ja...@apache.org.
Merge branch 'cassandra-2.0' into cassandra-2.1.0


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

Branch: refs/heads/cassandra-2.1.0
Commit: 3e2e4dd907c934d4b15b17ed49ea0d47ca8fbc7b
Parents: a1348aa 9be6576
Author: Jake Luciani <ja...@apache.org>
Authored: Tue Aug 12 09:56:09 2014 -0400
Committer: Jake Luciani <ja...@apache.org>
Committed: Tue Aug 12 09:56:09 2014 -0400

----------------------------------------------------------------------

----------------------------------------------------------------------