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

Melvin Wang commented on CASSANDRA-2988:
----------------------------------------

bq. 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.
Sure. This boils down to where to put the curly braces

bq. 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...
This patch is about changing the most part of the load() method. I am not clear 
how we could only change the initialization of RandomAcessReader.

bq. 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.
Sorry for lacking comments. I will add it. However, this is not a hard coding 
in the sense that, Short consists of 2 bytes and Long consists of 8 bytes, the 
sum is 10 bytes. It is just a quick checking if we reach the end.

bq. 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).
FYI, these two patches are completely independent with each other.

> 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

        

Reply via email to