[
https://issues.apache.org/jira/browse/CASSANDRA-847?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12842293#action_12842293
]
Stu Hood commented on CASSANDRA-847:
------------------------------------
> Are we on the same page?
Absolutely: the only point I would add is e) supporting range deletes.
> So let's keep IColumn (I believe the analogue in your patch is Named?) and
> Column.
The analogue in this patch was Slice: a slice has nested Metadata and a nested
ColumnKey, and it supports merging and garbage collection (see
SliceBuffer.merge() and SliceBuffer.garbageCollect).
> Here I think ColumnGroup is a better name than Slice
I'm fine with changing the name of Slice to ColumnGroup, but the problem I was
trying to solve with having a 'begin' and 'end' key in Slice is that it
indicates that you may not be holding the entire ColumnGroup in memory: for
instance, when a ColumnGroup needed to be split into multiple blocks. Also, the
reason we can support SliceBuffer.garbageCollect is because we know that we
have all columns between the two names.
> It's not clear to me how to apply the old ReducingIterator approach to
> multilevel groups. So you are probably
> right that this has to change.
I was thinking about this more last night, and I think you actually convinced
me that Scanner should go back to being an Iterator... heh. When iterating over
Slice objects (because they contain _all_ of the information about a set of
columns), you can build a SuperColumn with a ReducingIterator that collects all
Slices with equal names at depth 1, and you can build a ColumnFamily by
collecting all Slices with equal names at depth 0.
Also, CompactionIterator can go back to being a ReducingIterator that says that
all intersecting Slices are "equal", and reduces them using SliceBuffer.merge()
and garbageCollect(). There would need to be one subtle change to
ReducingIterator, to allow getReduced to return multiple objects (since merge()
does).
> it would be nice to come up with a scheme that doesn't repeat the full path
> in the description of each
> ColumnGroup [i.e., ColumnKey or its analogue], at least not on-disk
We definitely agree on this: a block header storing as much key and metadata
information as possible is definitely a good idea tm.
> Make the reading half of compactions memory-efficient
> -----------------------------------------------------
>
> Key: CASSANDRA-847
> URL: https://issues.apache.org/jira/browse/CASSANDRA-847
> Project: Cassandra
> Issue Type: Improvement
> Reporter: Stu Hood
> Priority: Critical
> Fix For: 0.7
>
> Attachments:
> 0001-Add-structures-that-were-important-to-the-SSTableSca.patch,
> 0002-Implement-most-of-the-new-SSTableScanner-interface.patch,
> 0003-Rename-RowIndexedReader-specific-test.patch,
> 0004-Improve-Scanner-tests-and-separate-SuperCF-handling-.patch,
> 0005-Add-Scanner-interface-and-a-Filtered-implementation-.patch,
> 0006-Add-support-for-compaction-of-super-CFs-and-some-tes.patch
>
>
> This issue is the next on the road to finally fixing CASSANDRA-16. To make
> compactions memory efficient, we have to be able to perform the compaction
> process on the smallest possible chunks that might intersect and contend
> one-another, meaning that we need a better abstraction for reading from
> SSTables.
--
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.