Uploaded image for project: 'Apache Cassandra'
  1. Apache Cassandra
  2. CASSANDRA-2158

memtable_throughput_in_mb can not support sizes over 2.2 gigs because of an integer overflow.

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Low
    • Resolution: Fixed
    • 0.7.4
    • None
    • None
    • Low

    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.

      Attachments

        1. 2158.txt
          10 kB
          Jonathan Ellis

        Activity

          People

            jbellis Jonathan Ellis
            trisk Eddie
            Jonathan Ellis
            Brandon Williams
            Votes:
            0 Vote for this issue
            Watchers:
            0 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved:

              Time Tracking

                Estimated:
                Original Estimate - 1h
                1h
                Remaining:
                Remaining Estimate - 1h
                1h
                Logged:
                Time Spent - Not Specified
                Not Specified