cassandra-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Sylvain Lebresne (Commented) (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (CASSANDRA-2988) Improve SSTableReader.load() when loading index files
Date Fri, 23 Dec 2011 16:18:30 GMT

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

Sylvain Lebresne commented on CASSANDRA-2988:
---------------------------------------------

+1 (on 2988-2-v2) with 2 nits:
* It's probably worth caching the value of {{sstableMetadata.estimatedRowSize.count()}} to
avoid the double computation most of the time.
* I think
{noformat}
long current = buckets.get(i);
if (current > 0)
    sum += current;
{noformat}
can be condensed to {{sum += buckets.get\(i);}} (given current can't be negative).
                
> 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.7
>
>         Attachments: 2988-2-cleaned.txt, 2988-2-v2.txt, 2988-parallel-v2.txt, c2988-2-v2,
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.
If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

Mime
View raw message