fix CLTest post-#6764 patch by Benedict Elliott Smith; reviewed by jbellis for CASSANDRA-6764
Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/44f4e790 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/44f4e790 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/44f4e790 Branch: refs/heads/trunk Commit: 44f4e790196ff6425255cd12cfd100ddf9415524 Parents: 75c1851 Author: Jonathan Ellis <jbel...@apache.org> Authored: Tue Apr 22 08:47:11 2014 -0500 Committer: Jonathan Ellis <jbel...@apache.org> Committed: Tue Apr 22 08:47:23 2014 -0500 ---------------------------------------------------------------------- .../org/apache/cassandra/db/CommitLogTest.java | 38 ++++++++++++++++++-- 1 file changed, 36 insertions(+), 2 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/cassandra/blob/44f4e790/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 577692d..ddab9ea 100644 --- a/test/unit/org/apache/cassandra/db/CommitLogTest.java +++ b/test/unit/org/apache/cassandra/db/CommitLogTest.java @@ -30,9 +30,11 @@ 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.composites.CellName; import org.apache.cassandra.net.MessagingService; import static org.apache.cassandra.utils.ByteBufferUtil.bytes; @@ -166,17 +168,49 @@ public class CommitLogTest extends SchemaLoader assert CommitLog.instance.activeSegments() == 1 : "Expecting 1 segment, got " + CommitLog.instance.activeSegments(); } + private static int getMaxRecordDataSize(String keyspace, ByteBuffer key, String table, CellName column) + { + Mutation rm = new Mutation("Keyspace1", bytes("k")); + rm.add("Standard1", Util.cellname("c1"), ByteBuffer.allocate(0), 0); + + int max = (DatabaseDescriptor.getCommitLogSegmentSize() / 2); + max -= (4 + 8 + 8); // log entry overhead + return max - (int) Mutation.serializer.serializedSize(rm, MessagingService.current_version); + } + + private static int getMaxRecordDataSize() + { + return getMaxRecordDataSize("Keyspace1", bytes("k"), "Standard1", Util.cellname("c1")); + } + // CASSANDRA-3615 @Test - public void testExceedSegmentSizeWithOverhead() throws Exception + public void testEqualRecordLimit() throws Exception { CommitLog.instance.resetUnsafe(); Mutation rm = new Mutation("Keyspace1", bytes("k")); - rm.add("Standard1", Util.cellname("c1"), ByteBuffer.allocate((DatabaseDescriptor.getCommitLogSegmentSize()) - 83), 0); + rm.add("Standard1", Util.cellname("c1"), ByteBuffer.allocate(getMaxRecordDataSize()), 0); CommitLog.instance.add(rm); } + @Test + public void testExceedRecordLimit() throws Exception + { + CommitLog.instance.resetUnsafe(); + try + { + Mutation rm = new Mutation("Keyspace1", bytes("k")); + rm.add("Standard1", Util.cellname("c1"), ByteBuffer.allocate(1 + getMaxRecordDataSize()), 0); + CommitLog.instance.add(rm); + throw new AssertionError("mutation larger than limit was accepted"); + } + catch (IllegalArgumentException e) + { + // IAE is thrown on too-large mutations + } + } + protected void testRecoveryWithBadSizeArgument(int size, int dataSize) throws Exception { Checksum checksum = new CRC32();