cassandra-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Melvin Wang (JIRA)" <j...@apache.org>
Subject [jira] [Issue Comment Edited] (CASSANDRA-2988) Improve SSTableReader.load() when loading index files
Date Wed, 03 Aug 2011 23:13:28 GMT

    [ https://issues.apache.org/jira/browse/CASSANDRA-2988?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13079104#comment-13079104
] 

Melvin Wang edited comment on CASSANDRA-2988 at 8/3/11 11:12 PM:
-----------------------------------------------------------------

Some stats data from my experiments. The experiment is about an isolated node. I just measure
the time spent (in ms) in ColumnFamilyStore constructor where we call SSTableReader.load().
We have one keyspace consisting of 3 column families. The total size of index files are marked
in the parenthesis.

Two optimization was tried. I will upload the patches shortly.
1) load all the index file of a single column family in parallel (abbr. as parallel in the
following table)
2) changed the way we buffer the file. Hard to put into words, but it is clear to look at
the patch for it. It can be considered as an improved version of BRAF. (abbr. as modified
bytebuffer in the follow table)

Also these two methods can be easily combined together because they are touching different
classes. (abbr. as combined in the following table)
First line in each method (longer time) is about not using the cache, second line is about
using the cache.


|||cf1(13.5G)|cf2(26.3G)|cf3(8.5G)||
||current|125380|235722|79531||
          || |24399|	59494	|31577||
||parallel   |  94028|	153820|	65966||
             || |15826|	16672|	20773||

||modified bytebuffer|   110476|	197089|	71348||
              || |16455	|38496	|19498||
||combined |     88897|	146750|	62344||
              || |9768	|10518	|12726||


      was (Author: mw):
    Some stats data from my experiments. The experiment is about an isolated node. I just
measure the time spent (in ms) in ColumnFamilyStore constructor where we call SSTableReader.load().
We have one keyspace consisting of 3 column families. The total size of index files are marked
in the parenthesis.

Two optimization was tried. I will upload the patches shortly.
1) load all the index file of a single column family in parallel (abbr. as parallel in the
following table)
2) changed the way we buffer the file. Hard to put into words, but it is clear to look at
the patch for it. It can be considered as an improved version of BRAF. (abbr. as modified
bytebuffer in the follow table)

Also these two methods can be easily combined together because they are touching different
classes. (abbr. as combined in the following table)
First line in each method (longer time) is about not using the cache, second line is about
using the cache.


|||cf1(13.5G)|cf2(26.3G)|cf3(8.5G)||
||current|125380|235722|79531||
          |||24399|	59494	|31577||
||parallel   |  94028|	153820|	65966||
             |||15826|	16672|	20773||

||modified bytebuffer|   110476|	197089|	71348||
              |||16455	|38496	|19498||
||combined |     88897|	146750|	62344||
              |||9768	|10518	|12726||

  
> Improve SSTableReader.load() when loading index files
> -----------------------------------------------------
>
>                 Key: CASSANDRA-2988
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-2988
>             Project: Cassandra
>          Issue Type: Improvement
>            Reporter: Melvin Wang
>
> * when we create BufferredRandomAccessFile, we pass skipCache=true. This hurts the read
performance because we always process the index files sequentially. Simple fix would be set
it to false.
> * multiple index files of a single column family can be loaded in parallel. This buys
a lot when you have multiple super large index files.
> * we may also change how we buffer. By using BufferredRandomAccessFile, for every read,
we need bunch of checking like
>   - do we need to rebuffer?
>   - isEOF()?
>   - assertions
>   These can be simplified to some extent.  We can blindly buffer the index file by chunks
and process the buffer until a key lies across boundary of a chunk. Then we rebuffer and start
from the beginning of the partially read key. Conceptually, this is same as what BRAF does
but w/o the overhead in the read**() methods in BRAF.

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

        

Mime
View raw message