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

Jonathan Ellis commented on CASSANDRA-3203:
-------------------------------------------

If MeteredFlusher + getLiveSize were really doing their job right though, the 
pressure valve wouldn't be getting involved.  (However, I'm not sure it's 
entirely fair to tell it "you can have 1/3 of the heap" and then fill up more 
than 2/3 w/ BF + index.)

I think I'd like to hold off on further tweaking until after 1.0; the current 
fudge factor means we're erring on the side of extra safety, and I'm okay with 
that (especially since, as you noted, leveled compaction isn't very sensitive 
to initial flush size).

> Odd flush behavior
> ------------------
>
>                 Key: CASSANDRA-3203
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-3203
>             Project: Cassandra
>          Issue Type: Bug
>          Components: Core
>    Affects Versions: 1.0.0
>            Reporter: Brandon Williams
>            Assignee: Jonathan Ellis
>            Priority: Critical
>             Fix For: 1.0.0
>
>         Attachments: 3203-prelim.txt, 3203-v2.txt, 3203.png
>
>
> Given the same workload against 0.8, trunk is creating more than twice the 
> amount of sstables.  Even though a uniform stress workload is being 
> generated, flush size degrades quickly:
> {noformat}
>  INFO [NonPeriodicTasks:1] 2011-09-09 18:24:22,878 ColumnFamilyStore.java 
> (line 658) Enqueuing flush of Memtable-Standard1@2058235391(7741
> 035/110172631 serialized/live bytes, 151785 ops)
>  INFO [NonPeriodicTasks:1] 2011-09-09 18:24:24,888 ColumnFamilyStore.java 
> (line 658) Enqueuing flush of Memtable-Standard1@1520390052(3887
> 220/72403158 serialized/live bytes, 76220 ops)
>  INFO [NonPeriodicTasks:1] 2011-09-09 18:24:26,890 ColumnFamilyStore.java 
> (line 658) Enqueuing flush of Memtable-Standard1@1868496516(4097
> 085/76255481 serialized/live bytes, 80335 ops)
>  INFO [NonPeriodicTasks:1] 2011-09-09 18:24:28,893 ColumnFamilyStore.java 
> (line 658) Enqueuing flush of Memtable-Standard1@498232521(43513
> 20/80922269 serialized/live bytes, 85320 ops)
>  INFO [NonPeriodicTasks:1] 2011-09-09 18:24:29,895 ColumnFamilyStore.java 
> (line 658) Enqueuing flush of Memtable-Standard1@1592308290(2310
> 810/44514839 serialized/live bytes, 45310 ops)
>  INFO [NonPeriodicTasks:1] 2011-09-09 18:24:30,897 ColumnFamilyStore.java 
> (line 658) Enqueuing flush of Memtable-Standard1@775439677(22684
> 80/64984390 serialized/live bytes, 44480 ops)
>  INFO [NonPeriodicTasks:1] 2011-09-09 18:24:31,899 ColumnFamilyStore.java 
> (line 658) Enqueuing flush of Memtable-Standard1@928217914(26741
> 85/76231422 serialized/live bytes, 52435 ops)
>  INFO [NonPeriodicTasks:1] 2011-09-09 18:24:32,901 ColumnFamilyStore.java 
> (line 658) Enqueuing flush of Memtable-Standard1@158103119(27511
> 95/77317732 serialized/live bytes, 53945 ops)
>  INFO [NonPeriodicTasks:1] 2011-09-09 18:24:33,903 ColumnFamilyStore.java 
> (line 658) Enqueuing flush of Memtable-Standard1@2035169258(3132
> 420/88934701 serialized/live bytes, 61420 ops)
>  INFO [NonPeriodicTasks:1] 2011-09-09 18:24:34,905 ColumnFamilyStore.java 
> (line 658) Enqueuing flush of Memtable-Standard1@1097314626(2979
> 675/83651699 serialized/live bytes, 58425 ops)
> {noformat}
> The serialized to live size ratio appears completely out of whack.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

Reply via email to