Fix bloom filter sizing with LCS Patch by marcuse; reviewed by Paulo Motta for CASSANDRA-11344
Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/a48b836b Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/a48b836b Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/a48b836b Branch: refs/heads/trunk Commit: a48b836b15fe548334eb302051b34bd5554fa8f9 Parents: 719caa6 Author: Marcus Eriksson <[email protected]> Authored: Fri Mar 11 09:36:05 2016 +0100 Committer: Marcus Eriksson <[email protected]> Committed: Wed Mar 16 09:17:30 2016 +0100 ---------------------------------------------------------------------- CHANGES.txt | 1 + .../writers/MaxSSTableSizeWriter.java | 27 ++++++++++++-------- 2 files changed, 17 insertions(+), 11 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/cassandra/blob/a48b836b/CHANGES.txt ---------------------------------------------------------------------- diff --git a/CHANGES.txt b/CHANGES.txt index b37ef84..a69164e 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 2.2.6 + * Fix bloom filter sizing with LCS (CASSANDRA-11344) * (cqlsh) Fix error when result is 0 rows with EXPAND ON (CASSANDRA-11092) * Fix intra-node serialization issue for multicolumn-restrictions (CASSANDRA-11196) * Non-obsoleting compaction operations over compressed files can impose rate limit on normal reads (CASSANDRA-11301) http://git-wip-us.apache.org/repos/asf/cassandra/blob/a48b836b/src/java/org/apache/cassandra/db/compaction/writers/MaxSSTableSizeWriter.java ---------------------------------------------------------------------- diff --git a/src/java/org/apache/cassandra/db/compaction/writers/MaxSSTableSizeWriter.java b/src/java/org/apache/cassandra/db/compaction/writers/MaxSSTableSizeWriter.java index d30a612..2bae3b8 100644 --- a/src/java/org/apache/cassandra/db/compaction/writers/MaxSSTableSizeWriter.java +++ b/src/java/org/apache/cassandra/db/compaction/writers/MaxSSTableSizeWriter.java @@ -32,7 +32,6 @@ import org.apache.cassandra.io.sstable.metadata.MetadataCollector; public class MaxSSTableSizeWriter extends CompactionAwareWriter { - private final long estimatedTotalKeys; private final long expectedWriteSize; private final long maxSSTableSize; private final int level; @@ -46,10 +45,9 @@ public class MaxSSTableSizeWriter extends CompactionAwareWriter this.allSSTables = txn.originals(); this.level = level; this.maxSSTableSize = maxSSTableSize; - long totalSize = cfs.getExpectedCompactedFileSize(nonExpiredSSTables, compactionType); + long totalSize = getTotalWriteSize(nonExpiredSSTables, estimatedTotalKeys, cfs, compactionType); expectedWriteSize = Math.min(maxSSTableSize, totalSize); - estimatedTotalKeys = SSTableReader.getApproximateKeyCount(nonExpiredSSTables); - estimatedSSTables = Math.max(1, estimatedTotalKeys / maxSSTableSize); + estimatedSSTables = Math.max(1, totalSize / maxSSTableSize); File sstableDirectory = cfs.directories.getLocationForDisk(getWriteDirectory(expectedWriteSize)); @SuppressWarnings("resource") SSTableWriter writer = SSTableWriter.create(Descriptor.fromFilename(cfs.getTempSSTablePath(sstableDirectory)), @@ -61,6 +59,19 @@ public class MaxSSTableSizeWriter extends CompactionAwareWriter sstableWriter.switchWriter(writer); } + /** + * Gets the estimated total amount of data to write during compaction + */ + private static long getTotalWriteSize(Iterable<SSTableReader> nonExpiredSSTables, long estimatedTotalKeys, ColumnFamilyStore cfs, OperationType compactionType) + { + long estimatedKeysBeforeCompaction = 0; + for (SSTableReader sstable : nonExpiredSSTables) + estimatedKeysBeforeCompaction += sstable.estimatedKeys(); + estimatedKeysBeforeCompaction = Math.max(1, estimatedKeysBeforeCompaction); + double estimatedCompactionRatio = (double) estimatedTotalKeys / estimatedKeysBeforeCompaction; + return Math.round(estimatedCompactionRatio * cfs.getExpectedCompactedFileSize(nonExpiredSSTables, compactionType)); + } + @Override public boolean append(AbstractCompactedRow row) { @@ -80,10 +91,4 @@ public class MaxSSTableSizeWriter extends CompactionAwareWriter } return rie != null; } - - @Override - public long estimatedKeys() - { - return estimatedTotalKeys; - } -} \ No newline at end of file +}
