Repository: cassandra Updated Branches: refs/heads/cassandra-2.2 58bee04cc -> c645b1193 refs/heads/cassandra-3.0 b294058b6 -> da8d48a79 refs/heads/trunk 8517e13fe -> 793bf45fb
Extracted commit log failure policy tests from CommitLogTest to CommitLogFailurePolicyTest Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/c645b119 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/c645b119 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/c645b119 Branch: refs/heads/cassandra-2.2 Commit: c645b1193444fdfae7ded2ce01bddc0de2813429 Parents: 58bee04 Author: Paulo Motta <[email protected]> Authored: Mon Aug 10 16:28:06 2015 -0300 Committer: Benedict Elliott Smith <[email protected]> Committed: Fri Aug 14 20:28:24 2015 +0100 ---------------------------------------------------------------------- .../db/CommitLogFailurePolicyTest.java | 112 ++++++++++++++ .../org/apache/cassandra/db/CommitLogTest.java | 147 ------------------- 2 files changed, 112 insertions(+), 147 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/cassandra/blob/c645b119/test/unit/org/apache/cassandra/db/CommitLogFailurePolicyTest.java ---------------------------------------------------------------------- diff --git a/test/unit/org/apache/cassandra/db/CommitLogFailurePolicyTest.java b/test/unit/org/apache/cassandra/db/CommitLogFailurePolicyTest.java index 7dabd5f..cca6503 100644 --- a/test/unit/org/apache/cassandra/db/CommitLogFailurePolicyTest.java +++ b/test/unit/org/apache/cassandra/db/CommitLogFailurePolicyTest.java @@ -19,6 +19,8 @@ package org.apache.cassandra.db; +import java.io.File; + import org.junit.Assert; import org.junit.BeforeClass; import org.junit.Test; @@ -26,8 +28,10 @@ import org.apache.cassandra.SchemaLoader; 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.CommitLogSegmentManager; import org.apache.cassandra.exceptions.ConfigurationException; import org.apache.cassandra.gms.Gossiper; +import org.apache.cassandra.service.CassandraDaemon; import org.apache.cassandra.service.StorageService; import org.apache.cassandra.utils.JVMStabilityInspector; import org.apache.cassandra.utils.KillerForTests; @@ -45,6 +49,10 @@ public class CommitLogFailurePolicyTest @Test public void testCommitFailurePolicy_stop() throws ConfigurationException { + CassandraDaemon daemon = new CassandraDaemon(); + daemon.completeSetup(); //startup must be completed, otherwise commit log failure must kill JVM regardless of failure policy + StorageService.instance.registerDaemon(daemon); + // Need storage service active so stop policy can shutdown gossip StorageService.instance.initServer(); Assert.assertTrue(Gossiper.instance.isEnabled()); @@ -65,6 +73,10 @@ public class CommitLogFailurePolicyTest @Test public void testCommitFailurePolicy_die() { + CassandraDaemon daemon = new CassandraDaemon(); + daemon.completeSetup(); //startup must be completed, otherwise commit log failure must kill JVM regardless of failure policy + StorageService.instance.registerDaemon(daemon); + KillerForTests killerForTests = new KillerForTests(); JVMStabilityInspector.Killer originalKiller = JVMStabilityInspector.replaceKiller(killerForTests); Config.CommitFailurePolicy oldPolicy = DatabaseDescriptor.getCommitFailurePolicy(); @@ -73,11 +85,111 @@ public class CommitLogFailurePolicyTest DatabaseDescriptor.setCommitFailurePolicy(Config.CommitFailurePolicy.die); CommitLog.handleCommitError("Testing die policy", new Throwable()); Assert.assertTrue(killerForTests.wasKilled()); + Assert.assertFalse(killerForTests.wasKilledQuietly()); //only killed quietly on startup failure + } + finally + { + DatabaseDescriptor.setCommitFailurePolicy(oldPolicy); + JVMStabilityInspector.replaceKiller(originalKiller); + } + } + + @Test + public void testCommitFailurePolicy_mustDieIfNotStartedUp() + { + //startup was not completed successfuly (since method completeSetup() was not called) + CassandraDaemon daemon = new CassandraDaemon(); + StorageService.instance.registerDaemon(daemon); + + KillerForTests killerForTests = new KillerForTests(); + JVMStabilityInspector.Killer originalKiller = JVMStabilityInspector.replaceKiller(killerForTests); + Config.CommitFailurePolicy oldPolicy = DatabaseDescriptor.getCommitFailurePolicy(); + try + { + //even though policy is ignore, JVM must die because Daemon has not finished initializing + DatabaseDescriptor.setCommitFailurePolicy(Config.CommitFailurePolicy.ignore); + CommitLog.handleCommitError("Testing die policy", new Throwable()); + Assert.assertTrue(killerForTests.wasKilled()); + Assert.assertTrue(killerForTests.wasKilledQuietly()); //killed quietly due to startup failure + } + finally + { + DatabaseDescriptor.setCommitFailurePolicy(oldPolicy); + JVMStabilityInspector.replaceKiller(originalKiller); + } + } + + @Test + public void testCommitLogFailureBeforeInitialization_mustKillJVM() throws Exception + { + //startup was not completed successfuly (since method completeSetup() was not called) + CassandraDaemon daemon = new CassandraDaemon(); + StorageService.instance.registerDaemon(daemon); + + //let's make the commit log directory non-writable + File commitLogDir = new File(DatabaseDescriptor.getCommitLogLocation()); + commitLogDir.setWritable(false); + + KillerForTests killerForTests = new KillerForTests(); + JVMStabilityInspector.Killer originalKiller = JVMStabilityInspector.replaceKiller(killerForTests); + Config.CommitFailurePolicy oldPolicy = DatabaseDescriptor.getCommitFailurePolicy(); + try + { + DatabaseDescriptor.setCommitFailurePolicy(Config.CommitFailurePolicy.ignore); + + //now let's create a commit log segment manager and wait for it to fail + new CommitLogSegmentManager(CommitLog.instance); + + //busy wait since commitlogsegmentmanager spawns another thread + int retries = 0; + while (!killerForTests.wasKilled() && retries++ < 5) + Thread.sleep(10); + + //since failure was before CassandraDaemon startup, the JVM must be killed + Assert.assertTrue(killerForTests.wasKilled()); + Assert.assertTrue(killerForTests.wasKilledQuietly()); //killed quietly due to startup failure + } + finally + { + DatabaseDescriptor.setCommitFailurePolicy(oldPolicy); + JVMStabilityInspector.replaceKiller(originalKiller); + commitLogDir.setWritable(true); + } + } + + @Test + public void testCommitLogFailureAfterInitialization_mustRespectFailurePolicy() throws Exception + { + //startup was not completed successfuly (since method completeSetup() was not called) + CassandraDaemon daemon = new CassandraDaemon(); + daemon.completeSetup(); //startup must be completed, otherwise commit log failure must kill JVM regardless of failure policy + StorageService.instance.registerDaemon(daemon); + + //let's make the commit log directory non-writable + File commitLogDir = new File(DatabaseDescriptor.getCommitLogLocation()); + commitLogDir.setWritable(false); + + KillerForTests killerForTests = new KillerForTests(); + JVMStabilityInspector.Killer originalKiller = JVMStabilityInspector.replaceKiller(killerForTests); + Config.CommitFailurePolicy oldPolicy = DatabaseDescriptor.getCommitFailurePolicy(); + try + { + DatabaseDescriptor.setCommitFailurePolicy(Config.CommitFailurePolicy.ignore); + + //now let's create a commit log segment manager and wait for it to fail + new CommitLogSegmentManager(CommitLog.instance); + + //wait commit log segment manager thread to execute + Thread.sleep(50); + + //error policy is set to IGNORE, so JVM must not be killed if error ocurs after startup + Assert.assertFalse(killerForTests.wasKilled()); } finally { DatabaseDescriptor.setCommitFailurePolicy(oldPolicy); JVMStabilityInspector.replaceKiller(originalKiller); + commitLogDir.setWritable(true); } } } http://git-wip-us.apache.org/repos/asf/cassandra/blob/c645b119/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 536f0cb..2764da4 100644 --- a/test/unit/org/apache/cassandra/db/CommitLogTest.java +++ b/test/unit/org/apache/cassandra/db/CommitLogTest.java @@ -319,153 +319,6 @@ public class CommitLogTest } @Test - public void testCommitFailurePolicy_stop() throws ConfigurationException - { - CassandraDaemon daemon = new CassandraDaemon(); - daemon.completeSetup(); //startup must be completed, otherwise commit log failure must kill JVM regardless of failure policy - StorageService.instance.registerDaemon(daemon); - - // Need storage service active so stop policy can shutdown gossip - StorageService.instance.initServer(); - Assert.assertTrue(Gossiper.instance.isEnabled()); - - Config.CommitFailurePolicy oldPolicy = DatabaseDescriptor.getCommitFailurePolicy(); - try - { - DatabaseDescriptor.setCommitFailurePolicy(Config.CommitFailurePolicy.stop); - CommitLog.handleCommitError("Test stop error", new Throwable()); - Assert.assertFalse(Gossiper.instance.isEnabled()); - } - finally - { - DatabaseDescriptor.setCommitFailurePolicy(oldPolicy); - } - } - - @Test - public void testCommitFailurePolicy_die() - { - CassandraDaemon daemon = new CassandraDaemon(); - daemon.completeSetup(); //startup must be completed, otherwise commit log failure must kill JVM regardless of failure policy - StorageService.instance.registerDaemon(daemon); - - KillerForTests killerForTests = new KillerForTests(); - JVMStabilityInspector.Killer originalKiller = JVMStabilityInspector.replaceKiller(killerForTests); - Config.CommitFailurePolicy oldPolicy = DatabaseDescriptor.getCommitFailurePolicy(); - try - { - DatabaseDescriptor.setCommitFailurePolicy(Config.CommitFailurePolicy.die); - CommitLog.handleCommitError("Testing die policy", new Throwable()); - Assert.assertTrue(killerForTests.wasKilled()); - Assert.assertFalse(killerForTests.wasKilledQuietly()); //only killed quietly on startup failure - } - finally - { - DatabaseDescriptor.setCommitFailurePolicy(oldPolicy); - JVMStabilityInspector.replaceKiller(originalKiller); - } - } - - @Test - public void testCommitFailurePolicy_mustDieIfNotStartedUp() - { - //startup was not completed successfuly (since method completeSetup() was not called) - CassandraDaemon daemon = new CassandraDaemon(); - StorageService.instance.registerDaemon(daemon); - - KillerForTests killerForTests = new KillerForTests(); - JVMStabilityInspector.Killer originalKiller = JVMStabilityInspector.replaceKiller(killerForTests); - Config.CommitFailurePolicy oldPolicy = DatabaseDescriptor.getCommitFailurePolicy(); - try - { - //even though policy is ignore, JVM must die because Daemon has not finished initializing - DatabaseDescriptor.setCommitFailurePolicy(Config.CommitFailurePolicy.ignore); - CommitLog.handleCommitError("Testing die policy", new Throwable()); - Assert.assertTrue(killerForTests.wasKilled()); - Assert.assertTrue(killerForTests.wasKilledQuietly()); //killed quietly due to startup failure - } - finally - { - DatabaseDescriptor.setCommitFailurePolicy(oldPolicy); - JVMStabilityInspector.replaceKiller(originalKiller); - } - } - - @Test - public void testCommitLogFailureBeforeInitialization_mustKillJVM() throws Exception - { - //startup was not completed successfuly (since method completeSetup() was not called) - CassandraDaemon daemon = new CassandraDaemon(); - StorageService.instance.registerDaemon(daemon); - - //let's make the commit log directory non-writable - File commitLogDir = new File(DatabaseDescriptor.getCommitLogLocation()); - commitLogDir.setWritable(false); - - KillerForTests killerForTests = new KillerForTests(); - JVMStabilityInspector.Killer originalKiller = JVMStabilityInspector.replaceKiller(killerForTests); - Config.CommitFailurePolicy oldPolicy = DatabaseDescriptor.getCommitFailurePolicy(); - try - { - DatabaseDescriptor.setCommitFailurePolicy(Config.CommitFailurePolicy.ignore); - - //now let's create a commit log segment manager and wait for it to fail - new CommitLogSegmentManager(CommitLog.instance); - - //busy wait since commitlogsegmentmanager spawns another thread - int retries = 0; - while (!killerForTests.wasKilled() && retries++ < 5) - Thread.sleep(10); - - //since failure was before CassandraDaemon startup, the JVM must be killed - Assert.assertTrue(killerForTests.wasKilled()); - Assert.assertTrue(killerForTests.wasKilledQuietly()); //killed quietly due to startup failure - } - finally - { - DatabaseDescriptor.setCommitFailurePolicy(oldPolicy); - JVMStabilityInspector.replaceKiller(originalKiller); - commitLogDir.setWritable(true); - } - } - - @Test - public void testCommitLogFailureAfterInitialization_mustRespectFailurePolicy() throws Exception - { - //startup was not completed successfuly (since method completeSetup() was not called) - CassandraDaemon daemon = new CassandraDaemon(); - daemon.completeSetup(); //startup must be completed, otherwise commit log failure must kill JVM regardless of failure policy - StorageService.instance.registerDaemon(daemon); - - //let's make the commit log directory non-writable - File commitLogDir = new File(DatabaseDescriptor.getCommitLogLocation()); - commitLogDir.setWritable(false); - - KillerForTests killerForTests = new KillerForTests(); - JVMStabilityInspector.Killer originalKiller = JVMStabilityInspector.replaceKiller(killerForTests); - Config.CommitFailurePolicy oldPolicy = DatabaseDescriptor.getCommitFailurePolicy(); - try - { - DatabaseDescriptor.setCommitFailurePolicy(Config.CommitFailurePolicy.ignore); - - //now let's create a commit log segment manager and wait for it to fail - new CommitLogSegmentManager(CommitLog.instance); - - //wait commit log segment manager thread to execute - Thread.sleep(50); - - //error policy is set to IGNORE, so JVM must not be killed if error ocurs after startup - Assert.assertFalse(killerForTests.wasKilled()); - } - finally - { - DatabaseDescriptor.setCommitFailurePolicy(oldPolicy); - JVMStabilityInspector.replaceKiller(originalKiller); - commitLogDir.setWritable(true); - } - } - - @Test public void testTruncateWithoutSnapshot() throws ExecutionException, InterruptedException, IOException { CommitLog.instance.resetUnsafe(true);
