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] Created: (CASSANDRA-2158) memtable_throughput_in_mb can not support sizes over 2.2 gigs because of an integer overflow.

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);
    }


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

        

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

Posted by "Jonathan Ellis (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/CASSANDRA-2158?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

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

    Attachment: 2158.txt

patch to make ints into longs and validate input

> 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
>             Fix For: 0.7.2
>
>         Attachments: 2158.txt
>
>
> 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

        

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

Posted by "Eddie (JIRA)" <ji...@apache.org>.
     [ 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;

...
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);
    }


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.


> 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;
> ...
> 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

        

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

Posted by "Jonathan Ellis (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-2158?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12994105#comment-12994105 ] 

Jonathan Ellis commented on CASSANDRA-2158:
-------------------------------------------

While you are correct, you almost certainly shouldn't have throughput set that high, because if you are tuning things correctly you will hit your operations count limit first for 99.9% of workloads.

> 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;
> ...
> 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

        

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

Posted by "Jonathan Ellis (JIRA)" <ji...@apache.org>.
     [ 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

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

Posted by "Brandon Williams (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-2158?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12995434#comment-12995434 ] 

Brandon Williams commented on CASSANDRA-2158:
---------------------------------------------

+1

> 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
>             Fix For: 0.7.2
>
>         Attachments: 2158.txt
>
>
> 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

        

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

Posted by "Jonathan Ellis (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/CASSANDRA-2158?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

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

    Remaining Estimate: 1h
     Original Estimate: 1h

> 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
>             Fix For: 0.7.3
>
>         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

        

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

Posted by "Hudson (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-2158?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13005921#comment-13005921 ] 

Hudson commented on CASSANDRA-2158:
-----------------------------------

Integrated in Cassandra-0.7 #376 (See [https://hudson.apache.org/hudson/job/Cassandra-0.7/376/])
    fix memtable thresholds better
patch by Erik Foralsrud; reviewed by jbellis for CASSANDRA-2158


> 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

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

Posted by "Hudson (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-2158?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12996632#comment-12996632 ] 

Hudson commented on CASSANDRA-2158:
-----------------------------------

Integrated in Cassandra-0.7 #296 (See [https://hudson.apache.org/hudson/job/Cassandra-0.7/296/])
    update memtable_throughput to be a long
patch by jbellis; reviewed by brandonwilliams for CASSANDRA-2158


> 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
>             Fix For: 0.7.3
>
>         Attachments: 2158.txt
>
>
> 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

        

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

Posted by "Eddie (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-2158?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12994124#comment-12994124 ] 

Eddie commented on CASSANDRA-2158:
----------------------------------

I think I have a usecase where larger memtables would help a lot.  I have a combination of fat columns that can update frequently, and I have lots of memory (currently 96 gb).  I know I could also handle this by putting more boxes in the cluster, but I think I can get a lot more out of the boxes I have.  I am experimenting with breaking up my cf into multiple ones to get the same effect as the bigger sstable.  So far it seems to perform well, but feels hacky.

Even if you decide to have a hard limit on the memtable size, it should probably fail loudly instead of generating hundreds of sstables.  With my understanding of the current defaults, any default install of cassandra with more than 32 gb of memory will default to this state and will be hard for new users to understand (32/2 -> 16 gig heap | 16/8 -> 2gb default CF memtable throughput).  I would much prefer the option than the hard limit though :).

> 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;
> ...
> 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

        

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

Posted by "Brandon Williams (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/CASSANDRA-2158?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Brandon Williams updated CASSANDRA-2158:
----------------------------------------

    Fix Version/s:     (was: 0.7.2)
                   0.7.3

> 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
>             Fix For: 0.7.3
>
>         Attachments: 2158.txt
>
>
> 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