move too-large-for-segment check into LogRecordAdder, saving a redundant serializedSize computation
Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/6a7a2885 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/6a7a2885 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/6a7a2885 Branch: refs/heads/cassandra-1.2 Commit: 6a7a28852bee26903dbaf1ca02f0b89a5438898c Parents: 4d29c70 Author: Jonathan Ellis <[email protected]> Authored: Tue Mar 19 08:10:32 2013 -0400 Committer: Jonathan Ellis <[email protected]> Committed: Tue Mar 19 08:10:32 2013 -0400 ---------------------------------------------------------------------- .../apache/cassandra/db/commitlog/CommitLog.java | 37 +++----------- .../cassandra/db/commitlog/CommitLogSegment.java | 8 +-- 2 files changed, 11 insertions(+), 34 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/cassandra/blob/6a7a2885/src/java/org/apache/cassandra/db/commitlog/CommitLog.java ---------------------------------------------------------------------- diff --git a/src/java/org/apache/cassandra/db/commitlog/CommitLog.java b/src/java/org/apache/cassandra/db/commitlog/CommitLog.java index e4e9881..88a9706 100644 --- a/src/java/org/apache/cassandra/db/commitlog/CommitLog.java +++ b/src/java/org/apache/cassandra/db/commitlog/CommitLog.java @@ -188,13 +188,6 @@ public class CommitLog implements CommitLogMBean */ public void add(RowMutation rm) { - long totalSize = RowMutation.serializer.serializedSize(rm, MessagingService.current_version) + CommitLogSegment.ENTRY_OVERHEAD_SIZE; - if (totalSize > DatabaseDescriptor.getCommitLogSegmentSize()) - { - logger.warn("Skipping commitlog append of extremely large mutation ({} bytes)", totalSize); - return; - } - executor.add(new LogRecordAdder(rm)); } @@ -303,27 +296,6 @@ public class CommitLog implements CommitLogMBean } /** - * Forces a new segment file to be allocated and activated. Used mainly by truncate. - */ - public void forceNewSegment() throws ExecutionException, InterruptedException - { - logger.debug("Forcing new segment creation"); - - Callable<?> task = new Callable() - { - public Object call() - { - if (activeSegment.position() > 0) - activateNextSegment(); - - return null; - } - }; - - executor.submit(task).get(); - } - - /** * Fetches a new segment file from the allocator and activates it. * * @return the newly activated segment @@ -371,7 +343,14 @@ public class CommitLog implements CommitLogMBean public void run() { - if (!activeSegment.hasCapacityFor(rowMutation)) + long totalSize = RowMutation.serializer.serializedSize(rowMutation, MessagingService.current_version) + CommitLogSegment.ENTRY_OVERHEAD_SIZE; + if (totalSize > DatabaseDescriptor.getCommitLogSegmentSize()) + { + logger.warn("Skipping commitlog append of extremely large mutation ({} bytes)", totalSize); + return; + } + + if (!activeSegment.hasCapacityFor(totalSize)) { CommitLogSegment oldSegment = activeSegment; activateNextSegment(); http://git-wip-us.apache.org/repos/asf/cassandra/blob/6a7a2885/src/java/org/apache/cassandra/db/commitlog/CommitLogSegment.java ---------------------------------------------------------------------- diff --git a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegment.java b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegment.java index 469ab99..a4805fc 100644 --- a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegment.java +++ b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegment.java @@ -40,7 +40,6 @@ import org.apache.cassandra.db.RowMutation; import org.apache.cassandra.io.FSWriteError; import org.apache.cassandra.io.util.FileUtils; import org.apache.cassandra.net.MessagingService; -import org.apache.cassandra.utils.FBUtilities; import org.apache.cassandra.utils.PureJavaCrc32; /* @@ -172,12 +171,11 @@ public class CommitLogSegment } /** - * @return true if there is room to write() @param mutation to this segment + * @return true if there is room to write() @param size to this segment */ - public boolean hasCapacityFor(RowMutation mutation) + public boolean hasCapacityFor(long size) { - long totalSize = RowMutation.serializer.serializedSize(mutation, MessagingService.current_version) + ENTRY_OVERHEAD_SIZE; - return totalSize <= buffer.remaining(); + return size <= buffer.remaining(); } /**
