[ 
https://issues.apache.org/jira/browse/CASSANDRA-2158?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Jonathan Ellis updated CASSANDRA-2158:
--------------------------------------

         Priority: Minor  (was: Major)
    Fix Version/s:     (was: 0.7.3)
                   0.7.4

Erik Forkalsrud commented on the mailing list,

{noformat}
It looks like the fix isn't entirely correct.  The bug is still in 0.7.3.   In 
Memtable.java, the line:
  THRESHOLD = cfs.getMemtableThroughputInMB() * 1024 * 1024;

should be changed to:
  THRESHOLD = cfs.getMemtableThroughputInMB() * 1024L * 1024L;

Here's some code that illustrates the difference:

   public void testMultiplication() {
       int memtableThroughputInMB = 2300;
       long thresholdA = memtableThroughputInMB * 1024 * 1024;
       long thresholdB = memtableThroughputInMB * 1024L * 1024L;
       System.out.println("a=" + thresholdA + " b=" + thresholdB);
   }
{noformat}

Made this change for 0.7.4

> memtable_throughput_in_mb can not support sizes over 2.2 gigs because of an 
> integer overflow.
> ---------------------------------------------------------------------------------------------
>
>                 Key: CASSANDRA-2158
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-2158
>             Project: Cassandra
>          Issue Type: Bug
>            Reporter: Eddie
>            Assignee: Jonathan Ellis
>            Priority: Minor
>             Fix For: 0.7.4
>
>         Attachments: 2158.txt
>
>   Original Estimate: 1h
>  Remaining Estimate: 1h
>
> If memtable_throughput_in_mb is set past 2.2 gigs, no errors are thrown.  
> However, as soon as data starts being written it is almost immediately being 
> flushed.  Several hundred SSTables are created in minutes.  I am almost 
> positive that the problem is that when memtable_throughput_in_mb is being 
> converted into bytes the result is stored in an integer, which is overflowing.
> From memtable.java:
>     private final int THRESHOLD;
>     private final int THRESHOLD_COUNT;
> ...
> this.THRESHOLD = cfs.getMemtableThroughputInMB() * 1024 * 1024;
> this.THRESHOLD_COUNT = (int) (cfs.getMemtableOperationsInMillions() * 1024 * 
> 1024);
> NOTE:
> I also think currentThroughput also needs to be changed from an int to a 
> long.  I'm not sure if it is as simple as this or if this also is used in 
> other places.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

Reply via email to