[
https://issues.apache.org/jira/browse/CASSANDRA-172?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12711937#action_12711937
]
Jun Rao commented on CASSANDRA-172:
-----------------------------------
Overall, 172-addendum-v2 looks fine. Below is a new version of
ReducingIterator.computeNext() that I wrote. It's a bit shorter than your
version.
protected T computeNext()
{
if (last == null && !source.hasNext())
return endOfData();
boolean keyChanged = false;
while (true)
{
if (keyChanged)
break;
if (last != null)
reduce(last);
if (!source.hasNext())
{
last = null;
break;
}
T current = source.next();
if (last != null && !getKey(current).equals(getKey(last)))
keyChanged = true;
last = current;
}
return getReduced();
}
A couple of other minor comments:
* You need to add the following after line 1673 in CFS.java to reverse the
comparator for descending order.
if (!isAscending)
{
comparator = new ReverseComparator(comparator);
}
* SSTableColumnIterator.computeNext(), log the catched IOException.
> A improved and more general version of get_slice
> ------------------------------------------------
>
> Key: CASSANDRA-172
> URL: https://issues.apache.org/jira/browse/CASSANDRA-172
> Project: Cassandra
> Issue Type: New Feature
> Reporter: Jun Rao
> Assignee: Jun Rao
> Fix For: 0.4
>
> Attachments: 172-addendum-SF.patch, 172-addendum-v2.patch,
> 172-addendum.patch, get_slice_from.patchv1, get_slice_from.patchv2,
> get_slice_from.patchv3
>
>
> Today, get_slice has to scan through all columns in every memtable and
> sstable to get a slice of columns. This becomes inefficient when the number
> of columns in a row is large. We need a more efficient API.
--
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.