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

Sylvain Lebresne commented on CASSANDRA-1034:
---------------------------------------------

{quote}
Is this still a fair summary of why we want to fix this?

the problem is that we are using DK both for routing and for local key sorting
{quote}
Hum, I would actually rephrase it with token instead of DK, in the sense that 
we don't really use DK for routing, DK is a key with it's token "cached" to 
speed up computing it, we're using only the token to route. The problem is 
we're also using only the token for local key sorting.

But while we could/should be using token to route and the DK for local key 
sorting, we still need to be able to handle local key *search* by token. And 
that is imho the difficulty of this ticket (if we always had an actual valid 
key to do local key search it would be much easier). And we need local search 
based on tokens because:
  * we allow range_slices on a range of tokens (so this translate ultimately to 
local search by token)
  * even for range_slices by keys, we still end up splitting the key range by a 
token in getRestrictedRanges, hence resulting ultimately to a local search by 
token.
Then the problem is that since a token is a segment and a key (what we're 
searching for) a point, we can't really compare those, in the sense that a key 
is not necessarily either stricly greater, equal, or stricly lesser than a 
token. So you do have to consider both the "bounds" of the token, which are now 
point and that you can compare to keys.
                
> 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: Sylvain Lebresne
>            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

        

Reply via email to