cassandra-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Pavel Yaskevich (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (CASSANDRA-2988) Improve SSTableReader.load() when loading index files
Date Tue, 09 Aug 2011 21:12:31 GMT

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

Pavel Yaskevich commented on CASSANDRA-2988:
--------------------------------------------

First of all I would like to point you to http://wiki.apache.org/cassandra/CodeStyle, please
modify your code according to conventions listed in there.

According to c2988-modified-buffer.patch:

 - please encapsulate your modifications because if you compare how it was and how it is in
your patch it's hard to undertand and just looks like a mess, I would like to suggest moving
those modifications to separate inner class (IndexReader maybe?) and replace only RandomAccessReader
initialization in the SSTableReader.load(...) method...
 - let's add a test comparing "getEstimatedRowSize().count();" and "SSTable.estimateRowsFromIndex(input);"
just to be sure it works correctly.

Also I don't quiet understand logic behind "while (buffer.remaining() > 10) {" in SSTableReader.loadByteBuffer,
let's avoid any hardcoding or at least comment why you did that.

I'm going to take a closer look at patch for parallel index file loading after we will be
done with index reader patch (c2988-modified-buffer.patch).

> Improve SSTableReader.load() when loading index files
> -----------------------------------------------------
>
>                 Key: CASSANDRA-2988
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-2988
>             Project: Cassandra
>          Issue Type: Improvement
>          Components: Core
>            Reporter: Melvin Wang
>            Assignee: Melvin Wang
>            Priority: Minor
>             Fix For: 1.0
>
>         Attachments: c2988-modified-buffer.patch, c2988-parallel-load-sstables.patch
>
>
> * 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