Make read repair deterministic to enable unique ID creation function in 
Cassandra
---------------------------------------------------------------------------------

                 Key: CASSANDRA-1039
                 URL: https://issues.apache.org/jira/browse/CASSANDRA-1039
             Project: Cassandra
          Issue Type: Improvement
          Components: Core
    Affects Versions: 0.6.1
            Reporter: Roland Hänel
            Priority: Minor


When deciding to initiate read repair on a certain column, in the special case 
that there are different column values with the same timestamp (this situation 
can occur if values were 'simultaneously' inserted by several clients), the 
behavior should be deterministic (tie-breaker). This prevents ongoing 
back-and-forward read repair when one node repairs towards value X, and another 
one repairs towards value Y.

I propose to deploy the following rule: if we are in doubt whether to repair a 
column with timestamp T (because two values X and Y are present within the 
cluster, both at timestamp T), then we always repair towards X if 
some_func(X)<some_func(Y).  some_func() could be md5() or just plain 
byte-by-byte comparison (maybe the latter is better because it will never 
result in equality).

The major use case for this improvement is to enable Cassandra as a means to 
generate cluste-wide unique identifiers. This could be implemented like this:

  - client wants to grab <ID>, and inserts a column with key <ID>, value 
<some_random_stuff>, timestamp <ID>
  - client reads the just inserted column with consistency level ALL or QUORUM
  - if the read value (<some_random_stuff>) matches the value originally 
inserted by the client, the <ID> is unique for this client

In this algorithm, collisions are prevented by the fact that if to clients try 
to aquire the same <ID> at the same time, the proposed tie-breaking will take 
care that exactly one client wins after the first read repair.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.

Reply via email to