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 2018/01/10 14:48:06 UTC

[1/3] cassandra git commit: Split CommitLogStressTest to avoid timeout

Repository: cassandra
Updated Branches:
  refs/heads/cassandra-3.11 4faa6e12f -> 62e46f719
  refs/heads/trunk 39837d479 -> ee907a321


Split CommitLogStressTest to avoid timeout

patch by Jay Zhuang; reviewed by jasobrown for CASSANDRA-14143


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

Branch: refs/heads/cassandra-3.11
Commit: 62e46f71903b339d962c4dcb3d2c04991c391a68
Parents: 4faa6e1
Author: Jay Zhuang <ja...@yahoo.com>
Authored: Thu Dec 28 11:56:09 2017 -0800
Committer: Jason Brown <ja...@gmail.com>
Committed: Wed Jan 10 06:44:26 2018 -0800

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 .../db/commitlog/BatchCommitLogStressTest.java  |  37 ++++++
 .../db/commitlog/CommitLogStressTest.java       | 128 ++++++-------------
 .../commitlog/PeriodicCommitLogStressTest.java  |  39 ++++++
 4 files changed, 117 insertions(+), 88 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/62e46f71/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 1b252f5..d6d8066 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 3.11.2
+ * Split CommitLogStressTest to avoid timeout (CASSANDRA-14143)
  * Avoid invalidating disk boundaries unnecessarily (CASSANDRA-14083)
  * Avoid exposing compaction strategy index externally (CASSANDRA-14082)
  * Prevent continuous schema exchange between 3.0 and 3.11 nodes (CASSANDRA-14109)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/62e46f71/test/long/org/apache/cassandra/db/commitlog/BatchCommitLogStressTest.java
----------------------------------------------------------------------
diff --git a/test/long/org/apache/cassandra/db/commitlog/BatchCommitLogStressTest.java b/test/long/org/apache/cassandra/db/commitlog/BatchCommitLogStressTest.java
new file mode 100644
index 0000000..3665882
--- /dev/null
+++ b/test/long/org/apache/cassandra/db/commitlog/BatchCommitLogStressTest.java
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.db.commitlog;
+
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import org.apache.cassandra.config.Config;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.config.ParameterizedClass;
+import org.apache.cassandra.security.EncryptionContext;
+
+@RunWith(Parameterized.class)
+public class BatchCommitLogStressTest extends CommitLogStressTest
+{
+    public BatchCommitLogStressTest(ParameterizedClass commitLogCompression, EncryptionContext encryptionContext)
+    {
+        super(commitLogCompression, encryptionContext);
+        DatabaseDescriptor.setCommitLogSync(Config.CommitLogSync.batch);
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/62e46f71/test/long/org/apache/cassandra/db/commitlog/CommitLogStressTest.java
----------------------------------------------------------------------
diff --git a/test/long/org/apache/cassandra/db/commitlog/CommitLogStressTest.java b/test/long/org/apache/cassandra/db/commitlog/CommitLogStressTest.java
index 3f5be03..2162d85 100644
--- a/test/long/org/apache/cassandra/db/commitlog/CommitLogStressTest.java
+++ b/test/long/org/apache/cassandra/db/commitlog/CommitLogStressTest.java
@@ -33,25 +33,30 @@ import com.google.common.util.concurrent.RateLimiter;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.BeforeClass;
+import org.junit.Ignore;
 import org.junit.Test;
+import org.junit.runners.Parameterized.Parameters;
 
 import io.netty.util.concurrent.FastThreadLocalThread;
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
 import org.apache.cassandra.UpdateBuilder;
-import org.apache.cassandra.config.Config.CommitLogSync;
 import org.apache.cassandra.config.*;
 import org.apache.cassandra.db.Mutation;
 import org.apache.cassandra.db.marshal.UTF8Type;
 import org.apache.cassandra.db.partitions.PartitionUpdate;
 import org.apache.cassandra.db.rows.*;
+import org.apache.cassandra.io.compress.DeflateCompressor;
+import org.apache.cassandra.io.compress.LZ4Compressor;
+import org.apache.cassandra.io.compress.SnappyCompressor;
 import org.apache.cassandra.io.util.DataInputBuffer;
 import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.security.EncryptionContext;
 import org.apache.cassandra.security.EncryptionContextGenerator;
 
 
-public class CommitLogStressTest
+@Ignore
+public abstract class CommitLogStressTest
 {
     static
     {
@@ -79,55 +84,19 @@ public class CommitLogStressTest
         return hash;
     }
 
-    public static void main(String[] args) throws Exception
-    {
-        try
-        {
-            if (args.length >= 1)
-            {
-                NUM_THREADS = Integer.parseInt(args[0]);
-                System.out.println("Setting num threads to: " + NUM_THREADS);
-            }
-
-            if (args.length >= 2)
-            {
-                numCells = Integer.parseInt(args[1]);
-                System.out.println("Setting num cells to: " + numCells);
-            }
-
-            if (args.length >= 3)
-            {
-                cellSize = Integer.parseInt(args[1]);
-                System.out.println("Setting cell size to: " + cellSize + " be aware the source corpus may be small");
-            }
+    private boolean failed = false;
+    private volatile boolean stop = false;
+    private boolean randomSize = false;
+    private boolean discardedRun = false;
+    private CommitLogPosition discardedPos;
 
-            if (args.length >= 4)
-            {
-                rateLimit = Integer.parseInt(args[1]);
-                System.out.println("Setting per thread rate limit to: " + rateLimit);
-            }
-            initialize();
-
-            CommitLogStressTest tester = new CommitLogStressTest();
-            tester.cleanDir();
-            tester.testFixedSize();
-        }
-        catch (Throwable e)
-        {
-            e.printStackTrace(System.err);
-        }
-        finally
-        {
-            System.exit(0);
-        }
+    public CommitLogStressTest(ParameterizedClass commitLogCompression, EncryptionContext encryptionContext)
+    {
+        DatabaseDescriptor.setCommitLogCompression(commitLogCompression);
+        DatabaseDescriptor.setEncryptionContext(encryptionContext);
+        DatabaseDescriptor.setCommitLogSegmentSize(32);
     }
 
-    boolean failed = false;
-    volatile boolean stop = false;
-    boolean randomSize = false;
-    boolean discardedRun = false;
-    CommitLogPosition discardedPos;
-
     @BeforeClass
     static public void initialize() throws IOException
     {
@@ -165,12 +134,23 @@ public class CommitLogStressTest
         }
     }
 
+    @Parameters()
+    public static Collection<Object[]> buildParameterizedVariants()
+    {
+        return Arrays.asList(new Object[][]{
+        {null, EncryptionContextGenerator.createDisabledContext()}, // No compression, no encryption
+        {null, EncryptionContextGenerator.createContext(true)}, // Encryption
+        { new ParameterizedClass(LZ4Compressor.class.getName(), Collections.emptyMap()), EncryptionContextGenerator.createDisabledContext()},
+        { new ParameterizedClass(SnappyCompressor.class.getName(), Collections.emptyMap()), EncryptionContextGenerator.createDisabledContext()},
+        { new ParameterizedClass(DeflateCompressor.class.getName(), Collections.emptyMap()), EncryptionContextGenerator.createDisabledContext()}});
+    }
+
     @Test
     public void testRandomSize() throws Exception
     {
         randomSize = true;
         discardedRun = false;
-        testAllLogConfigs();
+        testLog();
     }
 
     @Test
@@ -178,8 +158,7 @@ public class CommitLogStressTest
     {
         randomSize = false;
         discardedRun = false;
-
-        testAllLogConfigs();
+        testLog();
     }
 
     @Test
@@ -187,45 +166,18 @@ public class CommitLogStressTest
     {
         randomSize = true;
         discardedRun = true;
-
-        testAllLogConfigs();
-    }
-
-    public void testAllLogConfigs() throws IOException, InterruptedException
-    {
-        failed = false;
-        DatabaseDescriptor.setCommitLogSyncBatchWindow(1);
-        DatabaseDescriptor.setCommitLogSyncPeriod(30);
-        DatabaseDescriptor.setCommitLogSegmentSize(32);
-
-        // test plain vanilla commit logs (the choice of 98% of users)
-        testLog(null, EncryptionContextGenerator.createDisabledContext());
-
-        // test the compression types
-        testLog(new ParameterizedClass("LZ4Compressor", null), EncryptionContextGenerator.createDisabledContext());
-        testLog(new ParameterizedClass("SnappyCompressor", null), EncryptionContextGenerator.createDisabledContext());
-        testLog(new ParameterizedClass("DeflateCompressor", null), EncryptionContextGenerator.createDisabledContext());
-
-        // test the encrypted commit log
-        testLog(null, EncryptionContextGenerator.createContext(true));
+        testLog();
     }
 
-    public void testLog(ParameterizedClass compression, EncryptionContext encryptionContext) throws IOException, InterruptedException
+    private void testLog() throws IOException, InterruptedException
     {
-        DatabaseDescriptor.setCommitLogCompression(compression);
-        DatabaseDescriptor.setEncryptionContext(encryptionContext);
-
         String originalDir = DatabaseDescriptor.getCommitLogLocation();
         try
         {
             DatabaseDescriptor.setCommitLogLocation(location);
-            for (CommitLogSync sync : CommitLogSync.values())
-            {
-                DatabaseDescriptor.setCommitLogSync(sync);
-                CommitLog commitLog = new CommitLog(CommitLogArchiver.disabled()).start();
-                testLog(commitLog);
-                assert !failed;
-            }
+            CommitLog commitLog = new CommitLog(CommitLogArchiver.disabled()).start();
+            testLog(commitLog);
+            assert !failed;
         }
         finally
         {
@@ -233,7 +185,7 @@ public class CommitLogStressTest
         }
     }
 
-    public void testLog(CommitLog commitLog) throws IOException, InterruptedException {
+    private void testLog(CommitLog commitLog) throws IOException, InterruptedException {
         System.out.format("\nTesting commit log size %.0fmb, compressor: %s, encryption enabled: %b, sync %s%s%s\n",
                            mb(DatabaseDescriptor.getCommitLogSegmentSize()),
                            commitLog.configuration.getCompressorName(),
@@ -345,7 +297,7 @@ public class CommitLogStressTest
         Assert.assertTrue(ratios.isEmpty());
     }
 
-    public ScheduledExecutorService startThreads(final CommitLog commitLog, final List<CommitlogThread> threads)
+    private ScheduledExecutorService startThreads(final CommitLog commitLog, final List<CommitlogThread> threads)
     {
         stop = false;
         for (int ii = 0; ii < NUM_THREADS; ii++) {
@@ -404,7 +356,7 @@ public class CommitLogStressTest
         return maxMemory / (1024 * 1024);
     }
 
-    public static ByteBuffer randomBytes(int quantity, Random tlr)
+    private static ByteBuffer randomBytes(int quantity, Random tlr)
     {
         ByteBuffer slice = ByteBuffer.allocate(quantity);
         ByteBuffer source = dataSource.duplicate();
@@ -427,7 +379,7 @@ public class CommitLogStressTest
 
         volatile CommitLogPosition clsp;
 
-        public CommitlogThread(CommitLog commitLog, Random rand)
+        CommitlogThread(CommitLog commitLog, Random rand)
         {
             this.commitLog = commitLog;
             this.random = rand;
@@ -524,7 +476,7 @@ public class CommitLogStressTest
         }
     }
 
-    class DummyHandler implements CommitLogReadHandler
+    static class DummyHandler implements CommitLogReadHandler
     {
         public boolean shouldSkipSegmentOnError(CommitLogReadException exception) throws IOException { return false; }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/62e46f71/test/long/org/apache/cassandra/db/commitlog/PeriodicCommitLogStressTest.java
----------------------------------------------------------------------
diff --git a/test/long/org/apache/cassandra/db/commitlog/PeriodicCommitLogStressTest.java b/test/long/org/apache/cassandra/db/commitlog/PeriodicCommitLogStressTest.java
new file mode 100644
index 0000000..509d46a
--- /dev/null
+++ b/test/long/org/apache/cassandra/db/commitlog/PeriodicCommitLogStressTest.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.db.commitlog;
+
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import org.apache.cassandra.config.Config;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.config.ParameterizedClass;
+import org.apache.cassandra.security.EncryptionContext;
+
+@RunWith(Parameterized.class)
+public class PeriodicCommitLogStressTest extends CommitLogStressTest
+{
+    public PeriodicCommitLogStressTest(ParameterizedClass commitLogCompression, EncryptionContext encryptionContext)
+    {
+        super(commitLogCompression, encryptionContext);
+        DatabaseDescriptor.setCommitLogSync(Config.CommitLogSync.periodic);
+        DatabaseDescriptor.setCommitLogSyncPeriod(30);
+    }
+
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@cassandra.apache.org
For additional commands, e-mail: commits-help@cassandra.apache.org


[2/3] cassandra git commit: Split CommitLogStressTest to avoid timeout

Posted by ja...@apache.org.
Split CommitLogStressTest to avoid timeout

patch by Jay Zhuang; reviewed by jasobrown for CASSANDRA-14143


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

Branch: refs/heads/trunk
Commit: 62e46f71903b339d962c4dcb3d2c04991c391a68
Parents: 4faa6e1
Author: Jay Zhuang <ja...@yahoo.com>
Authored: Thu Dec 28 11:56:09 2017 -0800
Committer: Jason Brown <ja...@gmail.com>
Committed: Wed Jan 10 06:44:26 2018 -0800

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 .../db/commitlog/BatchCommitLogStressTest.java  |  37 ++++++
 .../db/commitlog/CommitLogStressTest.java       | 128 ++++++-------------
 .../commitlog/PeriodicCommitLogStressTest.java  |  39 ++++++
 4 files changed, 117 insertions(+), 88 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/62e46f71/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 1b252f5..d6d8066 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 3.11.2
+ * Split CommitLogStressTest to avoid timeout (CASSANDRA-14143)
  * Avoid invalidating disk boundaries unnecessarily (CASSANDRA-14083)
  * Avoid exposing compaction strategy index externally (CASSANDRA-14082)
  * Prevent continuous schema exchange between 3.0 and 3.11 nodes (CASSANDRA-14109)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/62e46f71/test/long/org/apache/cassandra/db/commitlog/BatchCommitLogStressTest.java
----------------------------------------------------------------------
diff --git a/test/long/org/apache/cassandra/db/commitlog/BatchCommitLogStressTest.java b/test/long/org/apache/cassandra/db/commitlog/BatchCommitLogStressTest.java
new file mode 100644
index 0000000..3665882
--- /dev/null
+++ b/test/long/org/apache/cassandra/db/commitlog/BatchCommitLogStressTest.java
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.db.commitlog;
+
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import org.apache.cassandra.config.Config;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.config.ParameterizedClass;
+import org.apache.cassandra.security.EncryptionContext;
+
+@RunWith(Parameterized.class)
+public class BatchCommitLogStressTest extends CommitLogStressTest
+{
+    public BatchCommitLogStressTest(ParameterizedClass commitLogCompression, EncryptionContext encryptionContext)
+    {
+        super(commitLogCompression, encryptionContext);
+        DatabaseDescriptor.setCommitLogSync(Config.CommitLogSync.batch);
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/62e46f71/test/long/org/apache/cassandra/db/commitlog/CommitLogStressTest.java
----------------------------------------------------------------------
diff --git a/test/long/org/apache/cassandra/db/commitlog/CommitLogStressTest.java b/test/long/org/apache/cassandra/db/commitlog/CommitLogStressTest.java
index 3f5be03..2162d85 100644
--- a/test/long/org/apache/cassandra/db/commitlog/CommitLogStressTest.java
+++ b/test/long/org/apache/cassandra/db/commitlog/CommitLogStressTest.java
@@ -33,25 +33,30 @@ import com.google.common.util.concurrent.RateLimiter;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.BeforeClass;
+import org.junit.Ignore;
 import org.junit.Test;
+import org.junit.runners.Parameterized.Parameters;
 
 import io.netty.util.concurrent.FastThreadLocalThread;
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
 import org.apache.cassandra.UpdateBuilder;
-import org.apache.cassandra.config.Config.CommitLogSync;
 import org.apache.cassandra.config.*;
 import org.apache.cassandra.db.Mutation;
 import org.apache.cassandra.db.marshal.UTF8Type;
 import org.apache.cassandra.db.partitions.PartitionUpdate;
 import org.apache.cassandra.db.rows.*;
+import org.apache.cassandra.io.compress.DeflateCompressor;
+import org.apache.cassandra.io.compress.LZ4Compressor;
+import org.apache.cassandra.io.compress.SnappyCompressor;
 import org.apache.cassandra.io.util.DataInputBuffer;
 import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.security.EncryptionContext;
 import org.apache.cassandra.security.EncryptionContextGenerator;
 
 
-public class CommitLogStressTest
+@Ignore
+public abstract class CommitLogStressTest
 {
     static
     {
@@ -79,55 +84,19 @@ public class CommitLogStressTest
         return hash;
     }
 
-    public static void main(String[] args) throws Exception
-    {
-        try
-        {
-            if (args.length >= 1)
-            {
-                NUM_THREADS = Integer.parseInt(args[0]);
-                System.out.println("Setting num threads to: " + NUM_THREADS);
-            }
-
-            if (args.length >= 2)
-            {
-                numCells = Integer.parseInt(args[1]);
-                System.out.println("Setting num cells to: " + numCells);
-            }
-
-            if (args.length >= 3)
-            {
-                cellSize = Integer.parseInt(args[1]);
-                System.out.println("Setting cell size to: " + cellSize + " be aware the source corpus may be small");
-            }
+    private boolean failed = false;
+    private volatile boolean stop = false;
+    private boolean randomSize = false;
+    private boolean discardedRun = false;
+    private CommitLogPosition discardedPos;
 
-            if (args.length >= 4)
-            {
-                rateLimit = Integer.parseInt(args[1]);
-                System.out.println("Setting per thread rate limit to: " + rateLimit);
-            }
-            initialize();
-
-            CommitLogStressTest tester = new CommitLogStressTest();
-            tester.cleanDir();
-            tester.testFixedSize();
-        }
-        catch (Throwable e)
-        {
-            e.printStackTrace(System.err);
-        }
-        finally
-        {
-            System.exit(0);
-        }
+    public CommitLogStressTest(ParameterizedClass commitLogCompression, EncryptionContext encryptionContext)
+    {
+        DatabaseDescriptor.setCommitLogCompression(commitLogCompression);
+        DatabaseDescriptor.setEncryptionContext(encryptionContext);
+        DatabaseDescriptor.setCommitLogSegmentSize(32);
     }
 
-    boolean failed = false;
-    volatile boolean stop = false;
-    boolean randomSize = false;
-    boolean discardedRun = false;
-    CommitLogPosition discardedPos;
-
     @BeforeClass
     static public void initialize() throws IOException
     {
@@ -165,12 +134,23 @@ public class CommitLogStressTest
         }
     }
 
+    @Parameters()
+    public static Collection<Object[]> buildParameterizedVariants()
+    {
+        return Arrays.asList(new Object[][]{
+        {null, EncryptionContextGenerator.createDisabledContext()}, // No compression, no encryption
+        {null, EncryptionContextGenerator.createContext(true)}, // Encryption
+        { new ParameterizedClass(LZ4Compressor.class.getName(), Collections.emptyMap()), EncryptionContextGenerator.createDisabledContext()},
+        { new ParameterizedClass(SnappyCompressor.class.getName(), Collections.emptyMap()), EncryptionContextGenerator.createDisabledContext()},
+        { new ParameterizedClass(DeflateCompressor.class.getName(), Collections.emptyMap()), EncryptionContextGenerator.createDisabledContext()}});
+    }
+
     @Test
     public void testRandomSize() throws Exception
     {
         randomSize = true;
         discardedRun = false;
-        testAllLogConfigs();
+        testLog();
     }
 
     @Test
@@ -178,8 +158,7 @@ public class CommitLogStressTest
     {
         randomSize = false;
         discardedRun = false;
-
-        testAllLogConfigs();
+        testLog();
     }
 
     @Test
@@ -187,45 +166,18 @@ public class CommitLogStressTest
     {
         randomSize = true;
         discardedRun = true;
-
-        testAllLogConfigs();
-    }
-
-    public void testAllLogConfigs() throws IOException, InterruptedException
-    {
-        failed = false;
-        DatabaseDescriptor.setCommitLogSyncBatchWindow(1);
-        DatabaseDescriptor.setCommitLogSyncPeriod(30);
-        DatabaseDescriptor.setCommitLogSegmentSize(32);
-
-        // test plain vanilla commit logs (the choice of 98% of users)
-        testLog(null, EncryptionContextGenerator.createDisabledContext());
-
-        // test the compression types
-        testLog(new ParameterizedClass("LZ4Compressor", null), EncryptionContextGenerator.createDisabledContext());
-        testLog(new ParameterizedClass("SnappyCompressor", null), EncryptionContextGenerator.createDisabledContext());
-        testLog(new ParameterizedClass("DeflateCompressor", null), EncryptionContextGenerator.createDisabledContext());
-
-        // test the encrypted commit log
-        testLog(null, EncryptionContextGenerator.createContext(true));
+        testLog();
     }
 
-    public void testLog(ParameterizedClass compression, EncryptionContext encryptionContext) throws IOException, InterruptedException
+    private void testLog() throws IOException, InterruptedException
     {
-        DatabaseDescriptor.setCommitLogCompression(compression);
-        DatabaseDescriptor.setEncryptionContext(encryptionContext);
-
         String originalDir = DatabaseDescriptor.getCommitLogLocation();
         try
         {
             DatabaseDescriptor.setCommitLogLocation(location);
-            for (CommitLogSync sync : CommitLogSync.values())
-            {
-                DatabaseDescriptor.setCommitLogSync(sync);
-                CommitLog commitLog = new CommitLog(CommitLogArchiver.disabled()).start();
-                testLog(commitLog);
-                assert !failed;
-            }
+            CommitLog commitLog = new CommitLog(CommitLogArchiver.disabled()).start();
+            testLog(commitLog);
+            assert !failed;
         }
         finally
         {
@@ -233,7 +185,7 @@ public class CommitLogStressTest
         }
     }
 
-    public void testLog(CommitLog commitLog) throws IOException, InterruptedException {
+    private void testLog(CommitLog commitLog) throws IOException, InterruptedException {
         System.out.format("\nTesting commit log size %.0fmb, compressor: %s, encryption enabled: %b, sync %s%s%s\n",
                            mb(DatabaseDescriptor.getCommitLogSegmentSize()),
                            commitLog.configuration.getCompressorName(),
@@ -345,7 +297,7 @@ public class CommitLogStressTest
         Assert.assertTrue(ratios.isEmpty());
     }
 
-    public ScheduledExecutorService startThreads(final CommitLog commitLog, final List<CommitlogThread> threads)
+    private ScheduledExecutorService startThreads(final CommitLog commitLog, final List<CommitlogThread> threads)
     {
         stop = false;
         for (int ii = 0; ii < NUM_THREADS; ii++) {
@@ -404,7 +356,7 @@ public class CommitLogStressTest
         return maxMemory / (1024 * 1024);
     }
 
-    public static ByteBuffer randomBytes(int quantity, Random tlr)
+    private static ByteBuffer randomBytes(int quantity, Random tlr)
     {
         ByteBuffer slice = ByteBuffer.allocate(quantity);
         ByteBuffer source = dataSource.duplicate();
@@ -427,7 +379,7 @@ public class CommitLogStressTest
 
         volatile CommitLogPosition clsp;
 
-        public CommitlogThread(CommitLog commitLog, Random rand)
+        CommitlogThread(CommitLog commitLog, Random rand)
         {
             this.commitLog = commitLog;
             this.random = rand;
@@ -524,7 +476,7 @@ public class CommitLogStressTest
         }
     }
 
-    class DummyHandler implements CommitLogReadHandler
+    static class DummyHandler implements CommitLogReadHandler
     {
         public boolean shouldSkipSegmentOnError(CommitLogReadException exception) throws IOException { return false; }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/62e46f71/test/long/org/apache/cassandra/db/commitlog/PeriodicCommitLogStressTest.java
----------------------------------------------------------------------
diff --git a/test/long/org/apache/cassandra/db/commitlog/PeriodicCommitLogStressTest.java b/test/long/org/apache/cassandra/db/commitlog/PeriodicCommitLogStressTest.java
new file mode 100644
index 0000000..509d46a
--- /dev/null
+++ b/test/long/org/apache/cassandra/db/commitlog/PeriodicCommitLogStressTest.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.db.commitlog;
+
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import org.apache.cassandra.config.Config;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.config.ParameterizedClass;
+import org.apache.cassandra.security.EncryptionContext;
+
+@RunWith(Parameterized.class)
+public class PeriodicCommitLogStressTest extends CommitLogStressTest
+{
+    public PeriodicCommitLogStressTest(ParameterizedClass commitLogCompression, EncryptionContext encryptionContext)
+    {
+        super(commitLogCompression, encryptionContext);
+        DatabaseDescriptor.setCommitLogSync(Config.CommitLogSync.periodic);
+        DatabaseDescriptor.setCommitLogSyncPeriod(30);
+    }
+
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@cassandra.apache.org
For additional commands, e-mail: commits-help@cassandra.apache.org


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

Posted by ja...@apache.org.
Merge branch 'cassandra-3.11' into trunk


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

Branch: refs/heads/trunk
Commit: ee907a3219b3f725b6f969d3a84afdeb11e1f397
Parents: 39837d4 62e46f7
Author: Jason Brown <ja...@gmail.com>
Authored: Wed Jan 10 06:47:16 2018 -0800
Committer: Jason Brown <ja...@gmail.com>
Committed: Wed Jan 10 06:47:16 2018 -0800

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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@cassandra.apache.org
For additional commands, e-mail: commits-help@cassandra.apache.org