cassandra-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From ble...@apache.org
Subject [4/5] cassandra git commit: Fix CommitLogSegmentManagerTest
Date Tue, 08 Nov 2016 17:23:52 GMT
Fix CommitLogSegmentManagerTest

patch by Benjamin Lerer; reviewed by Joshua McKenzie for CASSANDRA-12283


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

Branch: refs/heads/trunk
Commit: 8ae31392d66a9004b01bc40a267f0c8b34fc028f
Parents: 875c107
Author: Benjamin Lerer <b.lerer@gmail.com>
Authored: Tue Nov 8 18:19:33 2016 +0100
Committer: Benjamin Lerer <b.lerer@gmail.com>
Committed: Tue Nov 8 18:19:33 2016 +0100

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../commitlog/CommitLogSegmentManagerTest.java  | 93 ++++++++++++--------
 2 files changed, 55 insertions(+), 39 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/8ae31392/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 37e38e4..dd9088b 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 3.10
+ * Fix CommitLogSegmentManagerTest (CASSANDRA-12283)
  * Fix cassandra-stress truncate option (CASSANDRA-12695)
  * Fix crossNode value when receiving messages (CASSANDRA-12791)
  * Don't load MX4J beans twice (CASSANDRA-12869)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8ae31392/test/unit/org/apache/cassandra/db/commitlog/CommitLogSegmentManagerTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/commitlog/CommitLogSegmentManagerTest.java
b/test/unit/org/apache/cassandra/db/commitlog/CommitLogSegmentManagerTest.java
index af23821..cc31874 100644
--- a/test/unit/org/apache/cassandra/db/commitlog/CommitLogSegmentManagerTest.java
+++ b/test/unit/org/apache/cassandra/db/commitlog/CommitLogSegmentManagerTest.java
@@ -24,11 +24,9 @@ import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.Random;
 import java.util.concurrent.Semaphore;
-import javax.naming.ConfigurationException;
 
 import com.google.common.collect.ImmutableMap;
 import org.junit.Assert;
-import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 
@@ -46,23 +44,36 @@ import org.apache.cassandra.db.marshal.AsciiType;
 import org.apache.cassandra.db.marshal.BytesType;
 import org.apache.cassandra.schema.KeyspaceParams;
 import org.jboss.byteman.contrib.bmunit.BMRule;
+import org.jboss.byteman.contrib.bmunit.BMRules;
 import org.jboss.byteman.contrib.bmunit.BMUnitRunner;
 
 @RunWith(BMUnitRunner.class)
 public class CommitLogSegmentManagerTest
 {
     //Block commit log service from syncing
-    @SuppressWarnings("unused")
-    private static final Semaphore allowSync = new Semaphore(0);
+    private static final Semaphore allowSync = new Semaphore(1);
 
     private static final String KEYSPACE1 = "CommitLogTest";
     private static final String STANDARD1 = "Standard1";
     private static final String STANDARD2 = "Standard2";
 
     private final static byte[] entropy = new byte[1024 * 256];
-    @BeforeClass
-    public static void defineSchema()
+
+    @Test
+    @BMRules(rules = {@BMRule(name = "Acquire Semaphore before sync",
+                              targetClass = "AbstractCommitLogService$1",
+                              targetMethod = "run",
+                              targetLocation = "AT INVOKE org.apache.cassandra.db.commitlog.CommitLog.sync",
+                              action = "org.apache.cassandra.db.commitlog.CommitLogSegmentManagerTest.allowSync.acquire()"),
+                      @BMRule(name = "Release Semaphore after sync",
+                              targetClass = "AbstractCommitLogService$1",
+                              targetMethod = "run",
+                              targetLocation = "AFTER INVOKE org.apache.cassandra.db.commitlog.CommitLog.sync",
+                              action = "org.apache.cassandra.db.commitlog.CommitLogSegmentManagerTest.allowSync.release()")})
+    public void testCompressedCommitLogBackpressure() throws Throwable
     {
+        // Perform all initialization before making CommitLog.Sync blocking
+        // Doing the initialization within the method guarantee that Byteman has performed
its injections when we start
         new Random().nextBytes(entropy);
         DatabaseDescriptor.daemonInitialization();
         DatabaseDescriptor.setCommitLogCompression(new ParameterizedClass("LZ4Compressor",
ImmutableMap.of()));
@@ -77,50 +88,54 @@ public class CommitLogSegmentManagerTest
                                     SchemaLoader.standardCFMD(KEYSPACE1, STANDARD2, 0, AsciiType.instance,
BytesType.instance));
 
         CompactionManager.instance.disableAutoCompaction();
-    }
 
-    @Test
-    @BMRule(name = "Block AbstractCommitLogSegment segment flushing",
-            targetClass = "AbstractCommitLogService$1",
-            targetMethod = "run",
-            targetLocation = "AT INVOKE org.apache.cassandra.db.commitlog.CommitLog.sync",
-            action = "org.apache.cassandra.db.commitlog.CommitLogSegmentManagerTest.allowSync.acquire()")
-    public void testCompressedCommitLogBackpressure() throws Throwable
-    {
-        CommitLog.instance.resetUnsafe(true);
         ColumnFamilyStore cfs1 = Keyspace.open(KEYSPACE1).getColumnFamilyStore(STANDARD1);
 
-        final Mutation m = new RowUpdateBuilder(cfs1.metadata, 0, "k")
-                     .clustering("bytes")
-                     .add("val", ByteBuffer.wrap(entropy))
-                     .build();
+        final Mutation m = new RowUpdateBuilder(cfs1.metadata, 0, "k").clustering("bytes")
+                                                                      .add("val", ByteBuffer.wrap(entropy))
+                                                                      .build();
 
-        Thread dummyThread = new Thread( () ->
-        {
+        Thread dummyThread = new Thread(() -> {
             for (int i = 0; i < 20; i++)
                 CommitLog.instance.add(m);
         });
-        dummyThread.start();
-
-        AbstractCommitLogSegmentManager clsm = CommitLog.instance.segmentManager;
 
-        // Protect against delay, but still break out as fast as possible
-        long start = System.currentTimeMillis();
-        while (System.currentTimeMillis() - start < 5000)
+        try
         {
-            if (clsm.getActiveSegments().size() >= 3)
-                break;
-        }
-        Thread.sleep(1000);
+            // Makes sure any call to CommitLog.sync is blocking
+            allowSync.acquire();
+
+            dummyThread.start();
 
-        // Should only be able to create 3 segments not 7 because it blocks waiting for truncation
that never comes
-        Assert.assertEquals(3, clsm.getActiveSegments().size());
+            AbstractCommitLogSegmentManager clsm = CommitLog.instance.segmentManager;
 
-        // Discard the currently active segments so allocation can continue.
-        // Take snapshot of the list, otherwise this will also discard newly allocated segments.
-        new ArrayList<>(clsm.getActiveSegments()).forEach( clsm::archiveAndDiscard
);
+            Util.spinAssertEquals(3, () -> clsm.getActiveSegments().size(), 5);
 
-        // The allocated count should reach the limit again.
-        Util.spinAssertEquals(3, () -> clsm.getActiveSegments().size(), 5);
+            Thread.sleep(1000);
+
+            // Should only be able to create 3 segments not 7 because it blocks waiting for
truncation that never comes
+            Assert.assertEquals(3, clsm.getActiveSegments().size());
+
+            // Discard the currently active segments so allocation can continue.
+            // Take snapshot of the list, otherwise this will also discard newly allocated
segments.
+            new ArrayList<>(clsm.getActiveSegments()).forEach( clsm::archiveAndDiscard
);
+
+            // The allocated count should reach the limit again.
+            Util.spinAssertEquals(3, () -> clsm.getActiveSegments().size(), 5);
+        }
+        finally
+        {
+            // Allow the CommitLog.sync to perform normally.
+            allowSync.release();
+        }
+        try
+        {
+            // Wait for the dummy thread to die
+            dummyThread.join();
+        }
+        catch (InterruptedException e)
+        {
+            Thread.currentThread().interrupt();
+        }
     }
 }


Mime
View raw message