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

Alternative Row Cache Implementation

    XMLWordPrintableJSON

Details

    • Improvement
    • Status: Resolved
    • Normal
    • Resolution: Won't Fix
    • 2.1 beta1
    • None

    Description

      we have been working on an alternative implementation to the existing row cache(s)

      We have 2 main goals:

      • Decrease memory -> get more rows in the cache without suffering a huge performance penalty
      • Reduce gc pressure

      This sounds a lot like we should be using the new serializing cache in 0.8.
      Unfortunately our workload consists of loads of updates which would invalidate the cache all the time.

      Note: Updated Patch Description (Please check history if you're interested where this was comming from)

      Rough Idea

      • Keep serialized row (ByteBuffer) in mem which represents unfiltered but collated columns of all ssts but not memtable columns
      • Writes dont affect the cache at all. They go only to the memtables
      • Reads collect columns from memtables and row cache
      • Serialized Row is re-written (merged) with mem tables when flushed

      Some Implementation Details

      Reads

      • Basically the read logic differ from regular uncached reads only in that a special CollationController which is deserializing columns from in memory bytes
      • In the first version of this cache the serialized in memory format was the same as the fs format but test showed that performance sufferd because a lot of unnecessary deserialization takes place and that columns seeks are O( n ) whithin one block
      • To improve on that a different in memory format was used. It splits length meta info and data of columns so that the names can be binary searched.
      ===========================
      Header (24)                    
      ===========================
      MaxTimestamp:        long  
      LocalDeletionTime:   int   
      MarkedForDeleteAt:   long  
      NumColumns:          int   
      ===========================
      Column Index (num cols * 12)              
      ===========================
      NameOffset:          int   
      ValueOffset:         int   
      ValueLength:         int   
      ===========================
      Column Data                
      ===========================
      Name:                byte[]
      Value:               byte[]
      SerializationFlags:  byte  
      Misc:                ?     
      Timestamp:           long  
      ---------------------------
      Misc Counter Column        
      ---------------------------
      TSOfLastDelete:      long  
      ---------------------------
      Misc Expiring Column       
      ---------------------------
      TimeToLive:          int   
      LocalDeletionTime:   int   
      ===========================
      
      
      • These rows are read by 2 new column interators which correspond to SSTableNamesIterator and SSTableSliceIterator. During filtering only columns that actually match are constructed. The searching / skipping is performed on the raw ByteBuffer and does not create any objects.
      • A special CollationController is used to access and collate via cache and said new iterators. It also supports skipping the cached row by max update timestamp

      Writes

      • Writes dont update or invalidate the cache.
      • In CFS.replaceFlushed memtables are merged before the data view is switched. I fear that this is killing counters because they would be overcounted but my understading of counters is somewhere between weak and non-existing. I guess that counters if one wants to support them here would need an additional unique local identifier in memory and in serialized cache to be able to filter duplicates or something like that.
          void replaceFlushed(Memtable memtable, SSTableReader sstable)
          {
              if (sstCache.getCapacity() > 0) {
                  mergeSSTCache(memtable);
              }
              data.replaceFlushed(memtable, sstable);
              CompactionManager.instance.submitBackground(this);
          }
      

      Test Results: See comments below

      Attachments

        1. 0001-CASSANDRA-2864-w-out-direct-counter-support.patch
          86 kB
          Daniel Doubleday
        2. rowcache-with-snaptree-sketch.patch
          125 kB
          Daniel Doubleday
        3. optimistic-locking.patch
          25 kB
          Daniel Doubleday

        Activity

          People

            doubleday Daniel Doubleday
            doubleday Daniel Doubleday
            Daniel Doubleday
            Sylvain Lebresne
            Votes:
            0 Vote for this issue
            Watchers:
            24 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved: