You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by "Stu Hood (JIRA)" <ji...@apache.org> on 2010/10/01 20:08:34 UTC

[jira] Issue Comment Edited: (CASSANDRA-1526) Make cassandra sampling and startup faster

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

Stu Hood edited comment on CASSANDRA-1526 at 10/1/10 2:08 PM:
--------------------------------------------------------------

1526-v2.txt is likely an improvement, but using the Pair seems unnecessary, since it will lead to autoboxing of the long, for two extra objects per iteration.

> passing an executor to CFS and returning a Future seems a little clunky. Can we make CFS create return a
> Callable<CFS> so Table doesn't have to leak the executor abstraction?
The return type would actually have to be something like Pair<ColumnFamilyStore,List<Callable<SSTableReader>>>, since it is parallelized at the sstable level... more/less clunky?

If ecapriolo is cool with waiting for 0.7.0 for an improvement, I'd propose that we not bother optimizing 0.6 for this case, since the spanned index entries make it inherently more difficult.

      was (Author: stuhood):
    1526-v2.txt is likely an improvement, but using the Pair seems unnecessary, since it will lead to autoboxing of the long, for two extra objects per iteration.

> passing an executor to CFS and returning a Future seems a little clunky. Can we make CFS create return a
> Callable<CFS> so Table doesn't have to leak the executor abstraction?
The return type would actually have to be something like Pair<ColumnFamilyStore,Callable<SSTableReader>>, since it is parallelized at the sstable level... more/less clunky?

If ecapriolo is cool with waiting for 0.7.0 for an improvement, I'd propose that we not bother optimizing 0.6 for this case, since the spanned index entries make it inherently more difficult.
  
> Make cassandra sampling and startup faster
> ------------------------------------------
>
>                 Key: CASSANDRA-1526
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-1526
>             Project: Cassandra
>          Issue Type: Improvement
>          Components: Core
>            Reporter: Edward Capriolo
>            Assignee: Stu Hood
>            Priority: Minor
>             Fix For: 0.6.7, 0.7.0
>
>         Attachments: 0.6-0001-Add-AggregateFuture-to-wait-for-a-batch-of-futures-a.patch, 0.6-0002-Parallelize-SSTable-open.patch, 1526-v2.txt, 1526.txt, cpu.txt, io.txt, snitcherror.txt
>
>
> http://wiki.apache.org/cassandra/CassandraHardware makes mention of very large disks I do not see how that would be possible.
> We have a server class system have 4x processors 16GB RAM a 6 DISK RAID5 (yes RAID0 would be faster but still) 
> {noformat}
> INFO [main] 2010-09-21 12:58:26,348 SSTableReader.java (line 120) Sampling index for /var/lib/cassandra/data/system/LocationInfo-699-Data.db
> ...
> INFO [main] 2010-09-21 13:05:51,333 CassandraDaemon.java (line 124) Binding thrift service to cdbsd07/10.71.71.57:9160
> {noformat}
> This node has 200GB of data in two column families and the time to sample all tables and startup is 7+ minutes. The logging suggests this process is happening a single SSTable at a time. Additionally the normal system vitals mainly DISK and CPU do not look overtaxed.
> * Since SSTables are immutable is there a way the sampling of the tables could be saved?
> * Could this process be done in parallel for speedup?
> * Can multiple column families be processed at once?
> Unless someone has an insanely powerful disk pack making mention of 2TB limitations seem out of place. Unless my calculations are wrong (which they usually are), I have a pretty decent hardware, and if I had 2 TB of data I would have a 95 minute node start up? 
> I hope that maybe sampling multiple ColumnFamilies at once would make nodes of at least a few hundred GB startup reasonably fast.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.