[ https://issues.apache.org/jira/browse/CASSANDRA-1608?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12920380#action_12920380 ]
Stu Hood commented on CASSANDRA-1608: ------------------------------------- > I guess Cassandra would only need a fixed count of exactly 2, making it a > non-issue. As you said: we would need a counting filter with 2 bits per bucket: if both bits are set, the bucket has collided. If any of the buckets for a key have collided, you can't perform the delete, but that isn't the end of the world here. > Redesigned Compaction > --------------------- > > Key: CASSANDRA-1608 > URL: https://issues.apache.org/jira/browse/CASSANDRA-1608 > Project: Cassandra > Issue Type: Improvement > Components: Core > Reporter: Chris Goffinet > Fix For: 0.7.1 > > > After seeing the I/O issues in CASSANDRA-1470, I've been doing some more > thinking on this subject that I wanted to lay out. > I propose we redo the concept of how compaction works in Cassandra. At the > moment, compaction is kicked off based on a write access pattern, not read > access pattern. In most cases, you want the opposite. You want to be able to > track how well each SSTable is performing in the system. If we were to keep > statistics in-memory of each SSTable, prioritize them based on most accessed, > and bloom filter hit/miss ratios, we could intelligently group sstables that > are being read most often and schedule them for compaction. We could also > schedule lower priority maintenance on SSTable's not often accessed. > I also propose we limit the size of each SSTable to a fix sized, that gives > us the ability to better utilize our bloom filters in a predictable manner. > At the moment after a certain size, the bloom filters become less reliable. > This would also allow us to group data most accessed. Currently the size of > an SSTable can grow to a point where large portions of the data might not > actually be accessed as often. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.