[jira] [Commented] (CASSANDRA-5903) Integer overflow in OffHeapBitSet when bloomfilter 2GB

2013-08-21 Thread Taylan Develioglu (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-5903?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13746120#comment-13746120
 ] 

Taylan Develioglu commented on CASSANDRA-5903:
--

ERROR [SSTableBatchOpen:6] 2013-08-21 15:29:51,799 CassandraDaemon.java (line 
192) Exception in thread Thread[SSTableBatchOpen:6,5,main]
java.lang.IllegalArgumentException
at org.apache.cassandra.io.util.Memory.allocate(Memory.java:58)
at 
org.apache.cassandra.utils.obs.OffHeapBitSet.deserialize(OffHeapBitSet.java:123)
at 
org.apache.cassandra.utils.BloomFilterSerializer.deserialize(BloomFilterSerializer.java:46)
at 
org.apache.cassandra.utils.Murmur2BloomFilter$Murmur2BloomFilterSerializer.deserialize(Murmur2BloomFilter.java:40)
at 
org.apache.cassandra.utils.FilterFactory.deserialize(FilterFactory.java:71)
at 
org.apache.cassandra.io.sstable.SSTableReader.loadBloomFilter(SSTableReader.java:365)
at 
org.apache.cassandra.io.sstable.SSTableReader.open(SSTableReader.java:195)
at 
org.apache.cassandra.io.sstable.SSTableReader.open(SSTableReader.java:153)
at 
org.apache.cassandra.io.sstable.SSTableReader$1.run(SSTableReader.java:258)
at 
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471)
at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:334)
at java.util.concurrent.FutureTask.run(FutureTask.java:166)
at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
at java.lang.Thread.run(Thread.java:724)


 Integer overflow in OffHeapBitSet when bloomfilter  2GB
 

 Key: CASSANDRA-5903
 URL: https://issues.apache.org/jira/browse/CASSANDRA-5903
 Project: Cassandra
  Issue Type: Bug
  Components: Core
Reporter: Taylan Develioglu
Assignee: Vijay
 Fix For: 1.2.9

 Attachments: 0001-CASSANDRA-5903.patch


 In org.apache.cassandra.utils.obs.OffHeapBitSet.
 byteCount overflows and causes an IllegalArgument exception in 
 Memory.allocate when bloomfilter is  2GB.
 Suggest changing byteCount to long.
 {code:title=OffHeapBitSet.java}
 public OffHeapBitSet(long numBits)
 {
 // OpenBitSet.bits2words calculation is there for backward 
 compatibility.
 int byteCount = OpenBitSet.bits2words(numBits) * 8;
 bytes = RefCountedMemory.allocate(byteCount);
 // flush/clear the existing memory.
 clear();
 }
 {code}

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (CASSANDRA-5903) Integer overflow in OffHeapBitSet when bloomfilter 2GB

2013-08-21 Thread Vijay (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-5903?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13746598#comment-13746598
 ] 

Vijay commented on CASSANDRA-5903:
--

Thanks Taylan, I will writeup a test case for it... The patch on 1.2 (0002) 
should handle up to 2GB * 8 over which we might want to serialize and 
deserialize into long for 2.1.

 Integer overflow in OffHeapBitSet when bloomfilter  2GB
 

 Key: CASSANDRA-5903
 URL: https://issues.apache.org/jira/browse/CASSANDRA-5903
 Project: Cassandra
  Issue Type: Bug
  Components: Core
Reporter: Taylan Develioglu
Assignee: Vijay
  Labels: patch
 Fix For: 1.2.9

 Attachments: 0001-CASSANDRA-5903.patch, 0002-CASSANDRA-5903.patch


 In org.apache.cassandra.utils.obs.OffHeapBitSet.
 byteCount overflows and causes an IllegalArgument exception in 
 Memory.allocate when bloomfilter is  2GB.
 Suggest changing byteCount to long.
 {code:title=OffHeapBitSet.java}
 public OffHeapBitSet(long numBits)
 {
 // OpenBitSet.bits2words calculation is there for backward 
 compatibility.
 int byteCount = OpenBitSet.bits2words(numBits) * 8;
 bytes = RefCountedMemory.allocate(byteCount);
 // flush/clear the existing memory.
 clear();
 }
 {code}

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (CASSANDRA-5903) Integer overflow in OffHeapBitSet when bloomfilter 2GB

2013-08-21 Thread Jonathan Ellis (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-5903?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13746612#comment-13746612
 ] 

Jonathan Ellis commented on CASSANDRA-5903:
---

Oops, comment race condition w/ Vijay. :)  Test would still be nice.

16GB filter?  Well, maybe that's where we decide that just making bigger and 
bigger sstables is a bad idea...

 Integer overflow in OffHeapBitSet when bloomfilter  2GB
 

 Key: CASSANDRA-5903
 URL: https://issues.apache.org/jira/browse/CASSANDRA-5903
 Project: Cassandra
  Issue Type: Bug
  Components: Core
Reporter: Taylan Develioglu
Assignee: Vijay
  Labels: patch
 Fix For: 1.2.9

 Attachments: 0001-CASSANDRA-5903.patch, 0002-CASSANDRA-5903.patch


 In org.apache.cassandra.utils.obs.OffHeapBitSet.
 byteCount overflows and causes an IllegalArgument exception in 
 Memory.allocate when bloomfilter is  2GB.
 Suggest changing byteCount to long.
 {code:title=OffHeapBitSet.java}
 public OffHeapBitSet(long numBits)
 {
 // OpenBitSet.bits2words calculation is there for backward 
 compatibility.
 int byteCount = OpenBitSet.bits2words(numBits) * 8;
 bytes = RefCountedMemory.allocate(byteCount);
 // flush/clear the existing memory.
 clear();
 }
 {code}

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (CASSANDRA-5903) Integer overflow in OffHeapBitSet when bloomfilter 2GB

2013-08-21 Thread Jonathan Ellis (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-5903?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13746876#comment-13746876
 ] 

Jonathan Ellis commented on CASSANDRA-5903:
---

+1

 Integer overflow in OffHeapBitSet when bloomfilter  2GB
 

 Key: CASSANDRA-5903
 URL: https://issues.apache.org/jira/browse/CASSANDRA-5903
 Project: Cassandra
  Issue Type: Bug
  Components: Core
Reporter: Taylan Develioglu
Assignee: Vijay
  Labels: patch
 Fix For: 1.2.9

 Attachments: 0001-CASSANDRA-5903-check.patch, 
 0001-CASSANDRA-5903.patch, 0002-CASSANDRA-5903.patch


 In org.apache.cassandra.utils.obs.OffHeapBitSet.
 byteCount overflows and causes an IllegalArgument exception in 
 Memory.allocate when bloomfilter is  2GB.
 Suggest changing byteCount to long.
 {code:title=OffHeapBitSet.java}
 public OffHeapBitSet(long numBits)
 {
 // OpenBitSet.bits2words calculation is there for backward 
 compatibility.
 int byteCount = OpenBitSet.bits2words(numBits) * 8;
 bytes = RefCountedMemory.allocate(byteCount);
 // flush/clear the existing memory.
 clear();
 }
 {code}

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (CASSANDRA-5903) Integer overflow in OffHeapBitSet when bloomfilter 2GB

2013-08-21 Thread Vijay (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-5903?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13747164#comment-13747164
 ] 

Vijay commented on CASSANDRA-5903:
--

Done, Thanks!

 Integer overflow in OffHeapBitSet when bloomfilter  2GB
 

 Key: CASSANDRA-5903
 URL: https://issues.apache.org/jira/browse/CASSANDRA-5903
 Project: Cassandra
  Issue Type: Bug
  Components: Core
Reporter: Taylan Develioglu
Assignee: Vijay
  Labels: patch
 Fix For: 1.2.9

 Attachments: 0001-CASSANDRA-5903-check.patch, 
 0001-CASSANDRA-5903.patch, 0002-CASSANDRA-5903.patch


 In org.apache.cassandra.utils.obs.OffHeapBitSet.
 byteCount overflows and causes an IllegalArgument exception in 
 Memory.allocate when bloomfilter is  2GB.
 Suggest changing byteCount to long.
 {code:title=OffHeapBitSet.java}
 public OffHeapBitSet(long numBits)
 {
 // OpenBitSet.bits2words calculation is there for backward 
 compatibility.
 int byteCount = OpenBitSet.bits2words(numBits) * 8;
 bytes = RefCountedMemory.allocate(byteCount);
 // flush/clear the existing memory.
 clear();
 }
 {code}

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (CASSANDRA-5903) Integer overflow in OffHeapBitSet when bloomfilter 2GB

2013-08-20 Thread Taylan Develioglu (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-5903?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13744858#comment-13744858
 ] 

Taylan Develioglu commented on CASSANDRA-5903:
--

Added relevant stacktrace

DEBUG [CompactionExecutor:116] 2013-08-19 17:06:07,543 CompactionTask.java 
(line 115) Expected bloom filter size : 1830440832
ERROR [CompactionExecutor:116] 2013-08-19 17:06:07,584 CassandraDaemon.java 
(line 192) Exception in thread Thread[CompactionExecutor:116,1,main]
java.lang.IllegalArgumentException
at org.apache.cassandra.io.util.Memory.allocate(Memory.java:58)
at 
org.apache.cassandra.utils.obs.OffHeapBitSet.init(OffHeapBitSet.java:40)
at 
org.apache.cassandra.utils.FilterFactory.createFilter(FilterFactory.java:143)
at 
org.apache.cassandra.utils.FilterFactory.getFilter(FilterFactory.java:137)
at 
org.apache.cassandra.utils.FilterFactory.getFilter(FilterFactory.java:126)
at 
org.apache.cassandra.io.sstable.SSTableWriter$IndexWriter.init(SSTableWriter.java:446)
at 
org.apache.cassandra.io.sstable.SSTableWriter.init(SSTableWriter.java:92)
at 
org.apache.cassandra.db.ColumnFamilyStore.createCompactionWriter(ColumnFamilyStore.java:1983)
at 
org.apache.cassandra.db.compaction.CompactionTask.runWith(CompactionTask.java:143)
at 
org.apache.cassandra.io.util.DiskAwareRunnable.runMayThrow(DiskAwareRunnable.java:48)
at 
org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
at 
org.apache.cassandra.db.compaction.CompactionTask.executeInternal(CompactionTask.java:58)
at 
org.apache.cassandra.db.compaction.AbstractCompactionTask.execute(AbstractCompactionTask.java:60)
at 
org.apache.cassandra.db.compaction.CompactionManager$BackgroundCompactionTask.run(CompactionManager.java:211)
at 
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471)
at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:334)
at java.util.concurrent.FutureTask.run(FutureTask.java:166)
at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
at java.lang.Thread.run(Thread.java:724)


 Integer overflow in OffHeapBitSet when bloomfilter  2GB
 

 Key: CASSANDRA-5903
 URL: https://issues.apache.org/jira/browse/CASSANDRA-5903
 Project: Cassandra
  Issue Type: Bug
  Components: Core
Reporter: Taylan Develioglu
 Fix For: 1.2.9


 In org.apache.cassandra.utils.obs.OffHeapBitSet.
 byteCount overflows and causes an IllegalArgument exception in 
 Memory.allocate when bloomfilter is  2GB.
 Suggest changing byteCount to long.
 {code:title=OffHeapBitSet.java}
 public OffHeapBitSet(long numBits)
 {
 // OpenBitSet.bits2words calculation is there for backward 
 compatibility.
 int byteCount = OpenBitSet.bits2words(numBits) * 8;
 bytes = RefCountedMemory.allocate(byteCount);
 // flush/clear the existing memory.
 clear();
 }
 {code}

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (CASSANDRA-5903) Integer overflow in OffHeapBitSet when bloomfilter 2GB

2013-08-20 Thread Vijay (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-5903?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13745244#comment-13745244
 ] 

Vijay commented on CASSANDRA-5903:
--

I can change the byte count to long, 

As a side note, i am not sure if we are addressing the right issue. From the 
stack trace the byteCount should be 228805104 which is 228 MB 
(OpenBitSet.bits2words(1830440832L) * 8L) which should fit in a integer.


 Integer overflow in OffHeapBitSet when bloomfilter  2GB
 

 Key: CASSANDRA-5903
 URL: https://issues.apache.org/jira/browse/CASSANDRA-5903
 Project: Cassandra
  Issue Type: Bug
  Components: Core
Reporter: Taylan Develioglu
Assignee: Vijay
 Fix For: 1.2.9


 In org.apache.cassandra.utils.obs.OffHeapBitSet.
 byteCount overflows and causes an IllegalArgument exception in 
 Memory.allocate when bloomfilter is  2GB.
 Suggest changing byteCount to long.
 {code:title=OffHeapBitSet.java}
 public OffHeapBitSet(long numBits)
 {
 // OpenBitSet.bits2words calculation is there for backward 
 compatibility.
 int byteCount = OpenBitSet.bits2words(numBits) * 8;
 bytes = RefCountedMemory.allocate(byteCount);
 // flush/clear the existing memory.
 clear();
 }
 {code}

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (CASSANDRA-5903) Integer overflow in OffHeapBitSet when bloomfilter 2GB

2013-08-20 Thread Jonathan Ellis (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-5903?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13745294#comment-13745294
 ] 

Jonathan Ellis commented on CASSANDRA-5903:
---

Hmm.  Is bits2words overflowing somehow?

 Integer overflow in OffHeapBitSet when bloomfilter  2GB
 

 Key: CASSANDRA-5903
 URL: https://issues.apache.org/jira/browse/CASSANDRA-5903
 Project: Cassandra
  Issue Type: Bug
  Components: Core
Reporter: Taylan Develioglu
Assignee: Vijay
 Fix For: 1.2.9


 In org.apache.cassandra.utils.obs.OffHeapBitSet.
 byteCount overflows and causes an IllegalArgument exception in 
 Memory.allocate when bloomfilter is  2GB.
 Suggest changing byteCount to long.
 {code:title=OffHeapBitSet.java}
 public OffHeapBitSet(long numBits)
 {
 // OpenBitSet.bits2words calculation is there for backward 
 compatibility.
 int byteCount = OpenBitSet.bits2words(numBits) * 8;
 bytes = RefCountedMemory.allocate(byteCount);
 // flush/clear the existing memory.
 clear();
 }
 {code}

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (CASSANDRA-5903) Integer overflow in OffHeapBitSet when bloomfilter 2GB

2013-08-20 Thread Vijay (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-5903?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13745320#comment-13745320
 ] 

Vijay commented on CASSANDRA-5903:
--

Not sure yet, still trying to figure it out (Since i am more curious)... A 
simple test shows it might run out after 17B to 18B keys in a single SSTable 
(thats a giant SST) :)

{code}
for (int i = 0; i  30; i++) {
long items = (i * 10L);
System.out.println(Items:  + items +  byteCount:  + 
(OpenBitSet.bits2words(items) * 8));
}
{code}

{noformat}
Items: 0 byteCount: 0
Items: 10 byteCount: 12500
Items: 20 byteCount: 25000
Items: 30 byteCount: 37500
Items: 40 byteCount: 5
Items: 50 byteCount: 62500
Items: 60 byteCount: 75000
Items: 70 byteCount: 87500
Items: 80 byteCount: 10
Items: 90 byteCount: 112500
Items: 100 byteCount: 125000
Items: 110 byteCount: 137500
Items: 120 byteCount: 15
Items: 130 byteCount: 162500
Items: 140 byteCount: 175000
Items: 150 byteCount: 187500
Items: 160 byteCount: 20
Items: 170 byteCount: 212500
Items: 180 byteCount: -2044967296
Items: 190 byteCount: -1919967296
Items: 200 byteCount: -1794967296
...
{noformat}

 Integer overflow in OffHeapBitSet when bloomfilter  2GB
 

 Key: CASSANDRA-5903
 URL: https://issues.apache.org/jira/browse/CASSANDRA-5903
 Project: Cassandra
  Issue Type: Bug
  Components: Core
Reporter: Taylan Develioglu
Assignee: Vijay
 Fix For: 1.2.9


 In org.apache.cassandra.utils.obs.OffHeapBitSet.
 byteCount overflows and causes an IllegalArgument exception in 
 Memory.allocate when bloomfilter is  2GB.
 Suggest changing byteCount to long.
 {code:title=OffHeapBitSet.java}
 public OffHeapBitSet(long numBits)
 {
 // OpenBitSet.bits2words calculation is there for backward 
 compatibility.
 int byteCount = OpenBitSet.bits2words(numBits) * 8;
 bytes = RefCountedMemory.allocate(byteCount);
 // flush/clear the existing memory.
 clear();
 }
 {code}

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (CASSANDRA-5903) Integer overflow in OffHeapBitSet when bloomfilter 2GB

2013-08-20 Thread Vijay (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-5903?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13745328#comment-13745328
 ] 

Vijay commented on CASSANDRA-5903:
--

Actually my calculations where wrong it does use 2 GB for 1830440832

long numElements = 1830440832L;
FilterFactory.getFilter(numElements, 0.01d, true);

fixing it.

 Integer overflow in OffHeapBitSet when bloomfilter  2GB
 

 Key: CASSANDRA-5903
 URL: https://issues.apache.org/jira/browse/CASSANDRA-5903
 Project: Cassandra
  Issue Type: Bug
  Components: Core
Reporter: Taylan Develioglu
Assignee: Vijay
 Fix For: 1.2.9


 In org.apache.cassandra.utils.obs.OffHeapBitSet.
 byteCount overflows and causes an IllegalArgument exception in 
 Memory.allocate when bloomfilter is  2GB.
 Suggest changing byteCount to long.
 {code:title=OffHeapBitSet.java}
 public OffHeapBitSet(long numBits)
 {
 // OpenBitSet.bits2words calculation is there for backward 
 compatibility.
 int byteCount = OpenBitSet.bits2words(numBits) * 8;
 bytes = RefCountedMemory.allocate(byteCount);
 // flush/clear the existing memory.
 clear();
 }
 {code}

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (CASSANDRA-5903) Integer overflow in OffHeapBitSet when bloomfilter 2GB

2013-08-20 Thread Jonathan Ellis (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-5903?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13745392#comment-13745392
 ] 

Jonathan Ellis commented on CASSANDRA-5903:
---

+1

 Integer overflow in OffHeapBitSet when bloomfilter  2GB
 

 Key: CASSANDRA-5903
 URL: https://issues.apache.org/jira/browse/CASSANDRA-5903
 Project: Cassandra
  Issue Type: Bug
  Components: Core
Reporter: Taylan Develioglu
Assignee: Vijay
 Fix For: 1.2.9

 Attachments: 0001-CASSANDRA-5903.patch


 In org.apache.cassandra.utils.obs.OffHeapBitSet.
 byteCount overflows and causes an IllegalArgument exception in 
 Memory.allocate when bloomfilter is  2GB.
 Suggest changing byteCount to long.
 {code:title=OffHeapBitSet.java}
 public OffHeapBitSet(long numBits)
 {
 // OpenBitSet.bits2words calculation is there for backward 
 compatibility.
 int byteCount = OpenBitSet.bits2words(numBits) * 8;
 bytes = RefCountedMemory.allocate(byteCount);
 // flush/clear the existing memory.
 clear();
 }
 {code}

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (CASSANDRA-5903) Integer overflow in OffHeapBitSet when bloomfilter 2GB

2013-08-20 Thread Jonathan Ellis (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-5903?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13745590#comment-13745590
 ] 

Jonathan Ellis commented on CASSANDRA-5903:
---

Can you also add a CHANGES entry?

 Integer overflow in OffHeapBitSet when bloomfilter  2GB
 

 Key: CASSANDRA-5903
 URL: https://issues.apache.org/jira/browse/CASSANDRA-5903
 Project: Cassandra
  Issue Type: Bug
  Components: Core
Reporter: Taylan Develioglu
Assignee: Vijay
 Fix For: 1.2.9

 Attachments: 0001-CASSANDRA-5903.patch


 In org.apache.cassandra.utils.obs.OffHeapBitSet.
 byteCount overflows and causes an IllegalArgument exception in 
 Memory.allocate when bloomfilter is  2GB.
 Suggest changing byteCount to long.
 {code:title=OffHeapBitSet.java}
 public OffHeapBitSet(long numBits)
 {
 // OpenBitSet.bits2words calculation is there for backward 
 compatibility.
 int byteCount = OpenBitSet.bits2words(numBits) * 8;
 bytes = RefCountedMemory.allocate(byteCount);
 // flush/clear the existing memory.
 clear();
 }
 {code}

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (CASSANDRA-5903) Integer overflow in OffHeapBitSet when bloomfilter 2GB

2013-08-20 Thread Vijay (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-5903?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13745622#comment-13745622
 ] 

Vijay commented on CASSANDRA-5903:
--

Done! Thanks.

 Integer overflow in OffHeapBitSet when bloomfilter  2GB
 

 Key: CASSANDRA-5903
 URL: https://issues.apache.org/jira/browse/CASSANDRA-5903
 Project: Cassandra
  Issue Type: Bug
  Components: Core
Reporter: Taylan Develioglu
Assignee: Vijay
 Fix For: 1.2.9

 Attachments: 0001-CASSANDRA-5903.patch


 In org.apache.cassandra.utils.obs.OffHeapBitSet.
 byteCount overflows and causes an IllegalArgument exception in 
 Memory.allocate when bloomfilter is  2GB.
 Suggest changing byteCount to long.
 {code:title=OffHeapBitSet.java}
 public OffHeapBitSet(long numBits)
 {
 // OpenBitSet.bits2words calculation is there for backward 
 compatibility.
 int byteCount = OpenBitSet.bits2words(numBits) * 8;
 bytes = RefCountedMemory.allocate(byteCount);
 // flush/clear the existing memory.
 clear();
 }
 {code}

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira