[
https://issues.apache.org/jira/browse/CASSANDRA-1034?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13124946#comment-13124946
]
Sylvain Lebresne commented on CASSANDRA-1034:
---------------------------------------------
I have 2 major problems with that patch:
The first one is I really dislike the idea of merging DK into Token (I disliked
the idea of merging Token into DK and that roughly the same idea). First, I
fail to see how this is of any help in solving what this ticket is trying to
solve. Second, I think it's a very bad use of types. A Token is not a Key. By
merging those together we just weaken our type hierarchy, thus getting less
insurance from types. Typically, with this patch, a function that really want a
DK, could get a Token, i.e getKey() is not guaranteed to return a valid key.
Now I know, we are already using 'false' DK by feeding null as a key sometimes.
Well, that is ugly and error prone. I don't think generalizing this everywhere
while introducing a 300K patch is the right way to go, quite the contrary.
Besides, it's inefficient. All the places were we do use only a Token, we'll
now have a bigger structure with a useless pointer to the EMPTY_BYTE_BUFFER
(granted this has probably little impact, but it's another sign that it's doing
it wrong).
The second problem is this doesn't work. This DK->Token really just muddy the
water but it doesn't solve anything. What we want is to fix the fact that the
code identifies token and keys as a one to one mapping. In particular, this is
forced in DK.compareTo(), which only compare the tokens, ignoring the keys.
Fixing that place is easy, and the patch does it, but it's really just a few
lines change.
The real problem is that the code make the assumption that key <-> token is one
to one in other places. So making DK.compareTo takes key into account breaks
other parts. For instance, in RowIteratorFactory, we have this:
{noformat}
return startWith.compareTo(row.getKey()) <= 0
&& (stopAt.isEmpty() || row.getKey().compareTo(stopAt) <= 0);
{noformat}
and say that startWith and stopAt are token only. The semantic is that this is
supposed to be inclusive on both bound. With the last patch, this would include
keys having the startWith token, but *not* the ones having stopAt as token,
because in the patch, a token compares strictly before all of the key having
this token (concretely, the attached patch skips keys during range queries).
And this is not the only places in the code where this problem manifest,
because this is the symptom of a larger problem. If more than one key can have
the same token, then tokens are a range of keys.
If you ask for the range of tokens [1, 4], then you expect that it will return
all the keys having token 1, 2, 3 and 4. That excludes having a token comparing
strictly before all the keys having this token (or having it compare strictly
after all the keys having it as token for that matter). Merging Token and DK
just doesn't work.
At the risk of sounding cocky, I really encourage people to have another look
at my patch. I do believe that once you've realized what solving this problem
entails, it's a solution that strike a reasonable balance in fixing the problem
without a entire rewrite of Cassandra.
> Remove assumption that Key to Token is one-to-one
> -------------------------------------------------
>
> Key: CASSANDRA-1034
> URL: https://issues.apache.org/jira/browse/CASSANDRA-1034
> Project: Cassandra
> Issue Type: Bug
> Reporter: Stu Hood
> Assignee: Pavel Yaskevich
> Priority: Minor
> Fix For: 1.1
>
> Attachments:
> 0001-Make-range-accept-both-Token-and-DecoratedKey.patch,
> 0002-LengthPartitioner.patch, 1034-1-Generify-AbstractBounds-v3.patch,
> 1034-2-Remove-assumption-that-token-and-keys-are-one-to-one-v3.patch,
> 1034_v1.txt, CASSANDRA-1034.patch
>
>
> get_range_slices assumes that Tokens do not collide and converts a KeyRange
> to an AbstractBounds. For RandomPartitioner, this assumption isn't safe, and
> would lead to a very weird heisenberg.
> Converting AbstractBounds to use a DecoratedKey would solve this, because the
> byte[] key portion of the DecoratedKey can act as a tiebreaker.
> Alternatively, we could make DecoratedKey extend Token, and then use
> DecoratedKeys in places where collisions are unacceptable.
--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators:
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira