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

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


bq. Can you please define what do you mean by "pure token"?

In you patch, it's a Token whose key is EMPTY_BYTE_BUFFER (which is *not* a 
valid row key, hence the 'pure' token name).

bq. Aren't we supposed to generate token from key in all situations except 
initial token in config and middle point between tokens?

And? Is that not enough? There is tons of place in the code where we manipulate 
those tokens 'not created from a key' (all the distribution code basically, 
which is a big part of Cassandra). Moreover, there is also range_slice that 
accept a range of token.

bq. So if you do a range slice using tokens instead of keys 
TokenFactory.fromString will force you to use correctly serialized token data 
which will also include key.

To what is this supposed to be an answer ?

bq. If token is generated from key than for me it's natural to have a key as 
member. The thing is that you are enable to create a "pure" token, Partitioner 
will always give you a Token with valid key except for midpoint method so if 
partitioner is used to generate tokens you are guaranteed to have a valid key 
in the resulting token instance.

But it's not always generated from a key! There is nothing natural to a key 
member in all the Token object manipulated by TokenMetadata and other, since 
there is not such key.

bq. The same thing I can say about your set of patches - it's 198 KB. Aren't we 
writing tests to catch such bugs?

Well, in my patch, 148K of those are a type generification only (that's why 
I've separated it). Because generics are erased at runtime, as long as it 
compile, there is *NO* chance this can introduce a bug. As for trusting tests 
to catch bugs, I think it's being overconfident in tests. But in the end, I'm 
happily taking back that objection as this is by far the less important.


Let me try to put things graphically, everyone loves a graph: if I draw the set 
of all keys as this:
{noformat}
[-----------------------------------------------------------------------------[
{noformat}
i.e, the ring but as a line because I'm ignoring wrapping for this.

Now, if I display row keys (decorated or not, that doesn't matter, both are 
keys), I would have for instance:
{noformat}
[---------------------------|-------|---------------|---------------|---------[
                            k1      k2              k3              k4
{noformat}
A key is a point on the ring.

Now if keys and tokens are a 1 to 1 mapping, then it could be ok to say that a 
token is a point on the ring, but once it's not the case, then it looks like 
that:
{noformat}
                                t                     t'              t''
[-------------------------[*|*******|*]----------[**|****]-------[**|******]--[
                            k1      k2              k3              k4
{noformat}
where t is the token for both k1 and k2 (and an infinite number of other keys 
(actually finite because we're working on a computer)), t' the token of k3 (and 
an 'infinite' number of other keys), etc...

A token is intrinsically a range, a segment on the ring. Shoving DK and Token 
into the same class everywhere in the code is saying that we'll use the same 
class for a point and an interval. How can that be a good idea? How can that 
not backfire on us and be hard to work with, making it easy to introduce errors?

                
> 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

        

Reply via email to