cassandra-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Omer van der Horst Jansen (JIRA)" <j...@apache.org>
Subject [jira] Updated: (CASSANDRA-884) get_range_slice returns multiple copies of each row
Date Tue, 16 Mar 2010 20:30:27 GMT

     [ https://issues.apache.org/jira/browse/CASSANDRA-884?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]

Omer van der Horst Jansen updated CASSANDRA-884:
------------------------------------------------

    Attachment: 0001-RangeSliceResponseResolver.patch

I think I have identified the source of my problem. I'm still new to the Cassandra source
code so I could have this completely wrong. The patch is for the 0.6 version of org.apache.cassandra.service.RangeSliceResponseResolver.

Removing redundant copies of returned rows appears to happen in the RangeSliceResponseResolver
class. This class uses an anonymous innner class that extends ReducingIterator to weed out
the duplicates.

ReducingIterator provides a computeNext() method that compares successive items to see if
they are duplicates. It does this by comparing the current and previous ('last') items to
its isEqual() method.

RangeSliceResponseResolver does not override that isEqual method. That causes Pair<Row,
InetAddress> objects to be compared with each other.  The ReducingIterator.isEqual method
always returns false, because (1)  Row doesn't specify an equals() method and (2) even if
it did, the InetAddresses of rows retrieved from different Cassandra instances would still
be different. This causes each row to be seen as unique.

The attached patch repairs this by providing the ReducingIterator derivative in RangeSliceResponseResolver
with an isEqual() method that compares the key and cf members of the Row objects. It ignores
the InetAddress component of the Pair.

Alternatively I could have added an equals() method to Row which would have simplified the
isEqual() method in RangeSliceResponseResolver.java a bit.



> get_range_slice returns multiple copies of each row
> ---------------------------------------------------
>
>                 Key: CASSANDRA-884
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-884
>             Project: Cassandra
>          Issue Type: Bug
>          Components: Core
>    Affects Versions: 0.5
>         Environment: 4-cluster Gentoo Linux 2.6.18 with a ReplicationFactor of 2
>            Reporter: Omer van der Horst Jansen
>             Fix For: 0.6
>
>         Attachments: 0001-RangeSliceResponseResolver.patch, TestApp2.java
>
>
> I've noticed that both 0.5.1 and 0.6b2 return multiple identical copies of the data stored
in my keyspace whenever I make a call to get_range_slice or get_range_slices using
> ConsistencyLevel.QUORUM and ReplicationFactor is greater than one.
> So with ReplicationFactor set to 2 for my application's KeySpace I get double the number
of KeySlices that I expect to get. When using ConsistencyLevel.ONE I get only one KeySlice
for each row.
> I've seen this happen with Cassandra 0.5.1 and with 0.6 beta 2. The behavior on 0.6 beta
2 is exhibited with both get_range_slice and get_range_slices.
> The attached Java program demonstrates the issue for 0.6 beta 2. The program writes a
series of single-column rows into the Standard1 table, and then uses get_range_slice to receive
a list of all row. The returned number of rows is consistently twice the number of rows written
to the database. I wipe out the database completely before running the test.

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


Mime
View raw message