[
https://issues.apache.org/jira/browse/CASSANDRA-1034?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13154293#comment-13154293
]
Sylvain Lebresne commented on CASSANDRA-1034:
---------------------------------------------
bq. Put another way: in my ideal world, DK.token would be purely an
optimization to avoid calling partitioner.getToken(key) over and over.
I understand that, but I think there is two different things and I want to know
exactly where the disagreement/problem is.
The first problem, which is imho the core of this ticket, is that the code
needs to be able somehow to deal with things like (where I use k for keys and t
for tokens, and the term range for either Range or Bounds):
* Is key k in the range [k', t] (or (t', k''])? Because when you do a
range_slice of [k', k''] and there is multiple nodes and [k', k''] spans
multiple replica, we will end up requesting all keys in [k', t] (for some t) or
(t', k''].
* Is key k in range (t, t']? Because we're allowed to range query keys by a
token range, but also a few other reason, like the fact that during validation
compaction we hashes together keys within a token range.
Note that those are not trivial questions, because for instance [k', t], while
we intuitively understand what it represents is a weird beast in that is a
range a point and a segment?!
Or in other words, as much as I'd like the operations on Tokens and the ones on
Keys to be two completely orthogonal sets of operation with no interaction
whatsoever, it is not the case and we have to deal with it.
Dealing with the case where we need tokens and we have keys is trivial (we just
call Key.getToken() and boom, we're back in the case with only tokens).
The problem is when we fundamentally work on keys, but have only token to start
with. Today (i.e. before this ticket), we take a simplification by doing
essentially the same thing that in the 'needs token but got keys' case by
having a sort of Token.getKey() (it's more ugly in practice, we inline calls to
new DecoratedKey(t, null), but that's the same thing). But doing that forces in
itself the fact that key an token are in bijection and we want to lift that.
One solution could be to try to keep Token as long as we can, even in places
where we really need a key and have the code deal with that. I can understand
that on the surface that could look clean, but in practice the code to do that
correctly would a pure nightmare. Just trying to implement a Range that would
mix token and keys (like the [k', t] range above) is a complete mess.
So what this patch does is realizing that you could characterize the set of
keys that a token t represents with only two keys: the smallest key having
token t, and the biggest key having token t.
Now, supposing we agree on what is above, the rest is implementation details
and that's probably a much simpler discussion. Note that above I'm not talking
of DecoratedKey, only key. But the question is, how do you represent those two
new keys (for each token). The patch uses special values of the key field of DK
to deal with those. I can agree this is not the cleanest thing ever and I'm
fine looking for a different encoding, but I just don't have a much better
idea, and frankly I don't find that horrible either.
bq. I think I'd rather have these in the partitioner
Good idea.
bq. his feels odd when we go ahead and construct DKs with null key anyway in
the other constructor.
The goal here is to avoid constructing one of the two 'fake' keys by accident
For that the second constructor is dedicated to their construction and as the
commnet says, you're not even supposed to use this second constructor, but use
Token.{upper|lower}Bound instead. Actually, the assert should check for the
EMPTY_BYTE_BUFFER.
bq. Could we update the methods for whose benefits we're performing the Token
-> DK conversion, to accept RingPosition instead?
Frankly, and as argumented above, no, not without *huge* pain. We only do that
conversion in places where we will have to do it at some point, and trying to
push Tokens deeper would only serve in having operations that make no real
sense for Tokens be able to actually deal with Token. As one example, we would
have to make Range with a mix of Token and Keys, and frankly that will be a
total mess to code.
bq. I don't see a good reason to not use a "real" hashcode implementation
(Objects.hashCode is useful here)
Not sure I follow but ByteBuffer.hashCode() does hash the content of the buffer
if that was what you meant.
bq. There's clearly a lot of places in this patch where it's causing special
case ugliness though, independent of its status as "max."
Again, I would be open to better encoding. But is there really that much
places? The patch tried to make it so that no code outside of DecoratedKey
really have to deal with it. If not perfect, I actually think it's better
contained that before the patch.
bq. I think the positives of making this Generic are outweighed by the negative
of implying that minimum value for partitioner X depends on the RingPosition
that is returning it. I think I'd rather accept the casting ugliness of having
a Partitioner method that does instanceof checks to return the appropriate type.
I think you're right.
bq. Serializer code: How does DK, AB, etc. code deal w/ backwards compatibility
issues?
Basically, old version only understand AbstractBounds of Token, while new
version generates/accept AbstractBounds of either token, or keys. When old
sends to new and keys are expected, new convert the range/bounds of token as
range/bounds of keys. When new sends to old, it converts any range/bounds of
keys to range/bounds of token.
bq. What if we added a Partitioner reference so we could just ask isMinimum()?
Do you mean to have the DK to have a reference to the partioner? If so, I agree
that it's probably something we should, but it's nothing specific to that patch
so I'd rather leave it to another ticket.
> 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-Generify-AbstractBounds.patch,
> 0002-Remove-assumption-that-token-and-keys-are-one-to-one.patch,
> 0003-unit-test.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