You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by "Eddie (JIRA)" <ji...@apache.org> on 2011/02/13 07:55:57 UTC

[jira] Updated: (CASSANDRA-2158) memtable_throughput_in_mb can not support sizes over 2.2 gigs because of an integer overflow.

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

Eddie updated CASSANDRA-2158:
-----------------------------

    Description: 
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;

    public Memtable(ColumnFamilyStore cfs)
    {

        this.cfs = cfs;
        creationTime = System.currentTimeMillis();
        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.

  was:
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;

    public Memtable(ColumnFamilyStore cfs)
    {

        this.cfs = cfs;
        creationTime = System.currentTimeMillis();
        this.THRESHOLD = cfs.getMemtableThroughputInMB() * 1024 * 1024;
        this.THRESHOLD_COUNT = (int) (cfs.getMemtableOperationsInMillions() * 1024 * 1024);
    }


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.


> 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
>    Affects Versions: 0.7.0
>            Reporter: Eddie
>
> 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;
>     public Memtable(ColumnFamilyStore cfs)
>     {
>         this.cfs = cfs;
>         creationTime = System.currentTimeMillis();
>         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