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

Jonathan Ellis commented on CASSANDRA-2988:
-------------------------------------------

bq. when we create BufferredRandomAccessFile, we pass skipCache=true

right, the point of skipCache is to say "don't cache this data because we're 
probably not using it soon, so don't treat it as more fresh than the hot data 
you have already in the page cache." which totally applies to the index 
sampling both after a restart and when opening a new streamed file.

bq. multiple index files of a single column family can be loaded in parallel

i think there's two scenarios: initial server startup (probably want to 
parallelize) and loading a new, streamed sstable (probably don't).

bq. we may also change how we buffer

that sounds like a great idea.

> 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

        

Reply via email to