Repository: cassandra
Updated Branches:
  refs/heads/cassandra-2.2 830137f1a -> 925e620b4


Dont cast expected bloom filter size to int

Patch by marcuse; reviewed by Jason Brown for CASSANDRA-9959


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

Branch: refs/heads/cassandra-2.2
Commit: 45bd07f6fe2269a8bc68f1c11524b3a3f80f4f23
Parents: 878d616
Author: Marcus Eriksson <marc...@apache.org>
Authored: Mon Aug 3 14:24:23 2015 +0200
Committer: Marcus Eriksson <marc...@apache.org>
Committed: Mon Aug 3 14:42:19 2015 +0200

----------------------------------------------------------------------
 CHANGES.txt                                                    | 1 +
 .../org/apache/cassandra/db/compaction/CompactionManager.java  | 6 +++---
 src/java/org/apache/cassandra/db/compaction/Scrubber.java      | 4 ++--
 3 files changed, 6 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/45bd07f6/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 5ce2cc7..a8cf796 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 2.0.17
+ * Don't cast expected bf size to an int (CASSANDRA-9959)
  * Log when messages are dropped due to cross_node_timeout (CASSANDRA-9793)
  * checkForEndpointCollision fails for legitimate collisions (CASSANDRA-9765)
  * Complete CASSANDRA-8448 fix (CASSANDRA-9519)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/45bd07f6/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionManager.java 
b/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
index 5b5b39e..9d71dc7 100644
--- a/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
@@ -553,8 +553,8 @@ public class CompactionManager implements 
CompactionManagerMBean
 
             long totalkeysWritten = 0;
 
-            int expectedBloomFilterSize = 
Math.max(cfs.metadata.getIndexInterval(),
-                                                   (int) 
(SSTableReader.getApproximateKeyCount(sstableAsSet, cfs.metadata)));
+            long expectedBloomFilterSize = 
Math.max(cfs.metadata.getIndexInterval(),
+                                                    
SSTableReader.getApproximateKeyCount(sstableAsSet, cfs.metadata));
             if (logger.isDebugEnabled())
                 logger.debug("Expected bloom filter size : " + 
expectedBloomFilterSize);
 
@@ -734,7 +734,7 @@ public class CompactionManager implements 
CompactionManagerMBean
 
     public static SSTableWriter createWriter(ColumnFamilyStore cfs,
                                              File compactionFileLocation,
-                                             int expectedBloomFilterSize,
+                                             long expectedBloomFilterSize,
                                              SSTableReader sstable)
     {
         FileUtils.createDirectory(compactionFileLocation);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/45bd07f6/src/java/org/apache/cassandra/db/compaction/Scrubber.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/Scrubber.java 
b/src/java/org/apache/cassandra/db/compaction/Scrubber.java
index dc60efa..d242264 100644
--- a/src/java/org/apache/cassandra/db/compaction/Scrubber.java
+++ b/src/java/org/apache/cassandra/db/compaction/Scrubber.java
@@ -41,7 +41,7 @@ public class Scrubber implements Closeable
 
     private final CompactionController controller;
     private final boolean isCommutative;
-    private final int expectedBloomFilterSize;
+    private final long expectedBloomFilterSize;
 
     private final RandomAccessReader dataFile;
     private final RandomAccessReader indexFile;
@@ -105,7 +105,7 @@ public class Scrubber implements Closeable
         }
 
         this.expectedBloomFilterSize = 
Math.max(cfs.metadata.getIndexInterval(),
-                hasIndexFile ? 
(int)(SSTableReader.getApproximateKeyCount(toScrub,cfs.metadata)) : 0);
+                hasIndexFile ? SSTableReader.getApproximateKeyCount(toScrub, 
cfs.metadata) : 0);
 
         // loop through each row, deserializing to check for damage.
         // we'll also loop through the index at the same time, using the 
position from the index to recover if the

Reply via email to